Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-35891][cdc-connector][paimon] support dynamic bucket. #3499

Merged
merged 3 commits into from
Aug 12, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -41,4 +41,10 @@ public interface DataSink {
default HashFunctionProvider<DataChangeEvent> getDataChangeEventHashFunctionProvider() {
return new DefaultDataChangeEventHashFunctionProvider();
}

default HashFunctionProvider<DataChangeEvent> getDataChangeEventHashFunctionProvider(
lvyanquan marked this conversation as resolved.
Show resolved Hide resolved
int parallelism) {
return getDataChangeEventHashFunctionProvider(); // fallback to nullary version if it isn't
// overridden
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ public PipelineExecution compose(PipelineDef pipelineDef) {
parallelism,
parallelism,
schemaOperatorIDGenerator.generate(),
dataSink.getDataChangeEventHashFunctionProvider());
dataSink.getDataChangeEventHashFunctionProvider(parallelism));

// Build Sink Operator
sinkTranslator.translate(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,21 @@

package org.apache.flink.cdc.connectors.paimon.sink;

import org.apache.flink.cdc.common.event.DataChangeEvent;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.function.HashFunctionProvider;
import org.apache.flink.cdc.common.sink.DataSink;
import org.apache.flink.cdc.common.sink.EventSinkProvider;
import org.apache.flink.cdc.common.sink.FlinkSinkProvider;
import org.apache.flink.cdc.common.sink.MetadataApplier;
import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonEventSink;
import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonRecordSerializer;
import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonSink;

import org.apache.paimon.options.Options;

import java.io.Serializable;
import java.time.ZoneId;
import java.util.List;
import java.util.Map;

Expand All @@ -47,26 +50,41 @@ public class PaimonDataSink implements DataSink, Serializable {

private final PaimonRecordSerializer<Event> serializer;

private final ZoneId zoneId;

public final String schemaOperatorUid;

public PaimonDataSink(
Options options,
Map<String, String> tableOptions,
String commitUser,
Map<TableId, List<String>> partitionMaps,
PaimonRecordSerializer<Event> serializer) {
PaimonRecordSerializer<Event> serializer,
ZoneId zoneId,
String schemaOperatorUid) {
this.options = options;
this.tableOptions = tableOptions;
this.commitUser = commitUser;
this.partitionMaps = partitionMaps;
this.serializer = serializer;
this.zoneId = zoneId;
this.schemaOperatorUid = schemaOperatorUid;
}

@Override
public EventSinkProvider getEventSinkProvider() {
return FlinkSinkProvider.of(new PaimonSink<>(options, commitUser, serializer));
return FlinkSinkProvider.of(
new PaimonEventSink(options, commitUser, serializer, schemaOperatorUid, zoneId));
}

@Override
public MetadataApplier getMetadataApplier() {
return new PaimonMetadataApplier(options, tableOptions, partitionMaps);
}

@Override
public HashFunctionProvider<DataChangeEvent> getDataChangeEventHashFunctionProvider(
int parallelism) {
return new PaimonHashFunctionProvider(options, zoneId, parallelism);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,17 @@ public DataSink createDataSink(Context context) {
}
}
PaimonRecordSerializer<Event> serializer = new PaimonRecordEventSerializer(zoneId);
return new PaimonDataSink(options, tableOptions, commitUser, partitionMaps, serializer);
String schemaOperatorUid =
context.getPipelineConfiguration()
.get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID);
return new PaimonDataSink(
options,
tableOptions,
commitUser,
partitionMaps,
serializer,
zoneId,
schemaOperatorUid);
}

@Override
Expand Down
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()));
lvyanquan marked this conversation as resolved.
Show resolved Hide resolved
} 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);
}
}
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);
}
}
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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ public class PaimonEvent {

// if true, means that table schema has changed right before this genericRow.
boolean shouldRefreshSchema;
int bucket;

public PaimonEvent(Identifier tableId, GenericRow genericRow) {
this.tableId = tableId;
Expand All @@ -44,6 +45,14 @@ public PaimonEvent(Identifier tableId, GenericRow genericRow, boolean shouldRefr
this.shouldRefreshSchema = shouldRefreshSchema;
}

public PaimonEvent(
Identifier tableId, GenericRow genericRow, boolean shouldRefreshSchema, int bucket) {
this.tableId = tableId;
this.genericRow = genericRow;
this.shouldRefreshSchema = shouldRefreshSchema;
this.bucket = bucket;
}

public Identifier getTableId() {
return tableId;
}
Expand All @@ -67,4 +76,12 @@ public GenericRow getGenericRow() {
public void setGenericRow(GenericRow genericRow) {
this.genericRow = genericRow;
}

public int getBucket() {
return bucket;
}

public void setBucket(int bucket) {
this.bucket = bucket;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.BucketWrapper;
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, commitUser))
.name("Assign Bucket")
// All Events after BucketAssignOperator are decorated with BucketWrapper.
.partitionCustom(
(bucket, numPartitions) -> bucket % numPartitions,
(event) -> ((BucketWrapper) event).getBucket());
}

@Override
public SimpleVersionedSerializer<MultiTableCommittable> getCommittableSerializer() {
CommitMessageSerializer fileSerializer = new CommitMessageSerializer();
return new MultiTableCommittableSerializer(fileSerializer);
}
}
Loading
Loading