Skip to content

Commit

Permalink
[FLINK-35891][cdc-connector][paimon] support dynamic bucket.
Browse files Browse the repository at this point in the history
  • Loading branch information
lvyanquan committed Jul 31, 2024
1 parent b15a226 commit 9ad64e1
Show file tree
Hide file tree
Showing 24 changed files with 980 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,8 @@ public interface DataSink {
* Get the {@code HashFunctionProvider<DataChangeEvent>} for calculating hash value if you need
* to partition by data change event before Sink.
*/
default HashFunctionProvider<DataChangeEvent> getDataChangeEventHashFunctionProvider() {
default HashFunctionProvider<DataChangeEvent> getDataChangeEventHashFunctionProvider(
int parallelism) {
return new DefaultDataChangeEventHashFunctionProvider();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ public PipelineExecution compose(PipelineDef pipelineDef) {
parallelism,
parallelism,
schemaOperatorIDGenerator.generate(),
dataSink.getDataChangeEventHashFunctionProvider());
dataSink.getDataChangeEventHashFunctionProvider(parallelism));

// Build Sink Operator
DataSinkTranslator sinkTranslator = new DataSinkTranslator();
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,36 @@ public class PaimonDataSink implements DataSink, Serializable {

private final PaimonRecordSerializer<Event> serializer;

private final ZoneId zoneId;

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

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

@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,8 @@ public DataSink createDataSink(Context context) {
}
}
PaimonRecordSerializer<Event> serializer = new PaimonRecordEventSerializer(zoneId);
return new PaimonDataSink(options, tableOptions, commitUser, partitionMaps, serializer);
return new PaimonDataSink(
options, tableOptions, commitUser, partitionMaps, serializer, zoneId);
}

@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.event.CreateTableEvent;
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.PaimonRecordEventSerializer;

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;

/**
* 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 PaimonRecordEventSerializer eventSerializer;

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);
}
eventSerializer = new PaimonRecordEventSerializer(zoneId);
eventSerializer.serialize(new CreateTableEvent(tableId, schema));
channelComputer = new RowAssignerChannelComputer(table.schema(), parallelism);
channelComputer.setup(parallelism);
}

@Override
public int hashcode(DataChangeEvent event) {
GenericRow genericRow = eventSerializer.serialize(event).getGenericRow();
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
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,60 @@
/*
* 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;

/** A {@link PaimonSink} to process {@link Event}. */
public class PaimonEventSink extends PaimonSink<Event> implements WithPreWriteTopology<Event> {

public PaimonEventSink(
Options catalogOptions, String commitUser, PaimonRecordSerializer<Event> serializer) {
super(catalogOptions, commitUser, serializer);
}

@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))
.name("Assign Bucket")
.partitionCustom(
new BucketWrapperEventPartitioner(), new BucketWrapperEventKeySelector());
}

@Override
public SimpleVersionedSerializer<MultiTableCommittable> getCommittableSerializer() {
CommitMessageSerializer fileSerializer = new CommitMessageSerializer();
return new MultiTableCommittableSerializer(fileSerializer);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketWrapperChangeEvent;

import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.data.GenericRow;
Expand Down Expand Up @@ -51,10 +52,12 @@ public PaimonRecordEventSerializer(ZoneId zoneId) {

@Override
public PaimonEvent serialize(Event event) {
Identifier tableId =
Identifier.create(
((ChangeEvent) event).tableId().getSchemaName(),
((ChangeEvent) event).tableId().getTableName());
int bucket = 0;
if (event instanceof BucketWrapperChangeEvent) {
bucket = ((BucketWrapperChangeEvent) event).getBucket();
event = ((BucketWrapperChangeEvent) event).getInnerEvent();
}
Identifier tableId = Identifier.fromString(((ChangeEvent) event).tableId().toString());
if (event instanceof SchemaChangeEvent) {
if (event instanceof CreateTableEvent) {
CreateTableEvent createTableEvent = (CreateTableEvent) event;
Expand All @@ -78,7 +81,7 @@ public PaimonEvent serialize(Event event) {
PaimonWriterHelper.convertEventToGenericRow(
dataChangeEvent,
schemaMaps.get(dataChangeEvent.tableId()).getFieldGetters());
return new PaimonEvent(tableId, genericRow);
return new PaimonEvent(tableId, genericRow, false, bucket);
} else {
throw new IllegalArgumentException(
"failed to convert Input into PaimonEvent, unsupported event: " + event);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,9 +40,9 @@ public class PaimonSink<InputT> implements WithPreCommitTopology<InputT, MultiTa
// provided a default commit user.
public static final String DEFAULT_COMMIT_USER = "admin";

private final Options catalogOptions;
protected final Options catalogOptions;

private final String commitUser;
protected final String commitUser;

private final PaimonRecordSerializer<InputT> serializer;

Expand Down
Loading

0 comments on commit 9ad64e1

Please sign in to comment.