-
Notifications
You must be signed in to change notification settings - Fork 1.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-35891][cdc-connect][paimon] support dynamic bucket mode.
- Loading branch information
Showing
22 changed files
with
1,113 additions
and
32 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
70 changes: 70 additions & 0 deletions
70
...-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonHashFunction.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cdc.connectors.paimon.sink; | ||
|
||
import org.apache.flink.cdc.common.data.RecordData; | ||
import org.apache.flink.cdc.common.event.DataChangeEvent; | ||
import org.apache.flink.cdc.common.event.TableId; | ||
import org.apache.flink.cdc.common.function.HashFunction; | ||
import org.apache.flink.cdc.common.schema.Schema; | ||
import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonWriterHelper; | ||
|
||
import org.apache.paimon.catalog.Catalog; | ||
import org.apache.paimon.catalog.Identifier; | ||
import org.apache.paimon.data.GenericRow; | ||
import org.apache.paimon.flink.FlinkCatalogFactory; | ||
import org.apache.paimon.flink.sink.RowAssignerChannelComputer; | ||
import org.apache.paimon.options.Options; | ||
import org.apache.paimon.table.FileStoreTable; | ||
|
||
import java.io.Serializable; | ||
import java.time.ZoneId; | ||
import java.util.List; | ||
|
||
/** | ||
* A {@link HashFunction} implementation for {@link PaimonDataSink}. Shuffle {@link DataChangeEvent} | ||
* by hash of PrimaryKey. | ||
*/ | ||
public class PaimonHashFunction implements HashFunction<DataChangeEvent>, Serializable { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private final List<RecordData.FieldGetter> fieldGetters; | ||
|
||
private final RowAssignerChannelComputer channelComputer; | ||
|
||
public PaimonHashFunction( | ||
Options options, TableId tableId, Schema schema, ZoneId zoneId, int parallelism) { | ||
Catalog catalog = FlinkCatalogFactory.createPaimonCatalog(options); | ||
FileStoreTable table; | ||
try { | ||
table = (FileStoreTable) catalog.getTable(Identifier.fromString(tableId.toString())); | ||
} catch (Catalog.TableNotExistException e) { | ||
throw new RuntimeException(e); | ||
} | ||
this.fieldGetters = PaimonWriterHelper.createFieldGetters(schema, zoneId); | ||
channelComputer = new RowAssignerChannelComputer(table.schema(), parallelism); | ||
channelComputer.setup(parallelism); | ||
} | ||
|
||
@Override | ||
public int hashcode(DataChangeEvent event) { | ||
GenericRow genericRow = PaimonWriterHelper.convertEventToGenericRow(event, fieldGetters); | ||
return channelComputer.channel(genericRow); | ||
} | ||
} |
51 changes: 51 additions & 0 deletions
51
...src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonHashFunctionProvider.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,51 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cdc.connectors.paimon.sink; | ||
|
||
import org.apache.flink.cdc.common.event.DataChangeEvent; | ||
import org.apache.flink.cdc.common.event.TableId; | ||
import org.apache.flink.cdc.common.function.HashFunction; | ||
import org.apache.flink.cdc.common.function.HashFunctionProvider; | ||
import org.apache.flink.cdc.common.schema.Schema; | ||
|
||
import org.apache.paimon.options.Options; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.time.ZoneId; | ||
|
||
/** A {@link HashFunctionProvider} implementation for {@link PaimonDataSink}. */ | ||
public class PaimonHashFunctionProvider implements HashFunctionProvider<DataChangeEvent> { | ||
|
||
private final Options options; | ||
|
||
private final ZoneId zoneId; | ||
|
||
private final int parallelism; | ||
|
||
public PaimonHashFunctionProvider(Options options, ZoneId zoneId, int parallelism) { | ||
this.options = options; | ||
this.zoneId = zoneId; | ||
this.parallelism = parallelism; | ||
} | ||
|
||
@Override | ||
public HashFunction<DataChangeEvent> getHashFunction(@Nullable TableId tableId, Schema schema) { | ||
return new PaimonHashFunction(options, tableId, schema, zoneId, parallelism); | ||
} | ||
} |
57 changes: 57 additions & 0 deletions
57
...mon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/OperatorIDGenerator.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cdc.connectors.paimon.sink.v2; | ||
|
||
import org.apache.flink.cdc.common.annotation.Internal; | ||
import org.apache.flink.runtime.jobgraph.OperatorID; | ||
|
||
import org.apache.flink.shaded.guava31.com.google.common.hash.Hashing; | ||
|
||
import static java.nio.charset.StandardCharsets.UTF_8; | ||
|
||
/** Generating {@link OperatorID} for communication between Flink operators. */ | ||
@Internal | ||
public class OperatorIDGenerator { | ||
private final String transformationUid; | ||
|
||
public OperatorIDGenerator(String transformationUid) { | ||
this.transformationUid = transformationUid; | ||
} | ||
|
||
/** | ||
* Generate {@link OperatorID}. | ||
* | ||
* <p>Operator ID generation is an internal implementation inside Flink, happening during the | ||
* stream graph generating phase, so our algorithm of generating operator ID should be exactly | ||
* the same as in Flink, in order to make sure that operators can reach out each other on the | ||
* cluster. | ||
* | ||
* @see | ||
* org.apache.flink.streaming.api.graph.StreamGraphHasherV2#traverseStreamGraphAndGenerateHashes | ||
* the algorithm of generating operator ID in Flink | ||
*/ | ||
public OperatorID generate() { | ||
byte[] hash = | ||
Hashing.murmur3_128(0) | ||
.newHasher() | ||
.putString(transformationUid, UTF_8) | ||
.hash() | ||
.asBytes(); | ||
return new OperatorID(hash); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
72 changes: 72 additions & 0 deletions
72
...-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonEventSink.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cdc.connectors.paimon.sink.v2; | ||
|
||
import org.apache.flink.cdc.common.event.Event; | ||
import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketAssignOperator; | ||
import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketWrapperEventKeySelector; | ||
import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketWrapperEventPartitioner; | ||
import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketWrapperEventTypeInfo; | ||
import org.apache.flink.core.io.SimpleVersionedSerializer; | ||
import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; | ||
import org.apache.flink.streaming.api.datastream.DataStream; | ||
|
||
import org.apache.paimon.flink.sink.MultiTableCommittable; | ||
import org.apache.paimon.flink.sink.MultiTableCommittableSerializer; | ||
import org.apache.paimon.options.Options; | ||
import org.apache.paimon.table.sink.CommitMessageSerializer; | ||
|
||
import java.time.ZoneId; | ||
|
||
/** A {@link PaimonSink} to process {@link Event}. */ | ||
public class PaimonEventSink extends PaimonSink<Event> implements WithPreWriteTopology<Event> { | ||
|
||
public final String schemaOperatorUid; | ||
|
||
public final ZoneId zoneId; | ||
|
||
public PaimonEventSink( | ||
Options catalogOptions, | ||
String commitUser, | ||
PaimonRecordSerializer<Event> serializer, | ||
String schemaOperatorUid, | ||
ZoneId zoneId) { | ||
super(catalogOptions, commitUser, serializer); | ||
this.schemaOperatorUid = schemaOperatorUid; | ||
this.zoneId = zoneId; | ||
} | ||
|
||
@Override | ||
public DataStream<Event> addPreWriteTopology(DataStream<Event> dataStream) { | ||
// Shuffle by key hash => Assign bucket => Shuffle by bucket. | ||
return dataStream | ||
.transform( | ||
"BucketAssign", | ||
new BucketWrapperEventTypeInfo(), | ||
new BucketAssignOperator(catalogOptions, schemaOperatorUid, zoneId)) | ||
.name("Assign Bucket") | ||
.partitionCustom( | ||
new BucketWrapperEventPartitioner(), new BucketWrapperEventKeySelector()); | ||
} | ||
|
||
@Override | ||
public SimpleVersionedSerializer<MultiTableCommittable> getCommittableSerializer() { | ||
CommitMessageSerializer fileSerializer = new CommitMessageSerializer(); | ||
return new MultiTableCommittableSerializer(fileSerializer); | ||
} | ||
} |
Oops, something went wrong.