JingsongLi commented on code in PR #1261:
URL: https://github.com/apache/incubator-paimon/pull/1261#discussion_r1234011019
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java:
##########
@@ -85,55 +99,94 @@ public void build() {
Preconditions.checkNotNull(parserFactory);
StreamExecutionEnvironment env = input.getExecutionEnvironment();
+ if (mode == MySqlDatabaseSyncMode.DYNAMIC) {
+ buildDynamicCdcSink(env);
+ } else {
+ buildStaticCdcSink(env);
+ }
+ }
+ private void buildDynamicCdcSink(StreamExecutionEnvironment env) {
+ SingleOutputStreamOperator<Void> parsed =
+ input.forward()
+ .process(
+ new CdcDynamicTableParsingProcessFunction<>(
+ database, catalogLoader, tables,
parserFactory, mode))
+ .setParallelism(input.getParallelism());
+
+ // for newly-added tables, create a multiplexing operator that handles
all their records
+ // and writes to multiple tables
+ DataStream<CdcMultiplexRecord> newlyAddedTableStream =
+ SingleOutputStreamOperatorUtils.getSideOutput(
+ parsed,
CdcDynamicTableParsingProcessFunction.DYNAMIC_OUTPUT_TAG);
+ // handles schema change for newly added tables
+ SingleOutputStreamOperatorUtils.getSideOutput(
+ parsed,
+
CdcDynamicTableParsingProcessFunction.DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG)
+ .process(new
MultiTableUpdatedDataFieldsProcessFunction(catalogLoader));
+
+ FlinkStreamPartitioner<CdcMultiplexRecord> partitioner =
+ new FlinkStreamPartitioner<>(new
CdcMultiplexRecordChannelComputer(catalogLoader));
+ PartitionTransformation<CdcMultiplexRecord> partitioned =
Review Comment:
no one use `partitioned`.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputer.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.flink.sink.ChannelComputer;
+import org.apache.paimon.table.FileStoreTable;
+
+import java.util.Map;
+import java.util.Objects;
+
+/** {@link ChannelComputer} for {@link CdcRecord}. */
+public class CdcMultiplexRecordChannelComputer implements
ChannelComputer<CdcMultiplexRecord> {
+
+ private static final long serialVersionUID = 1L;
+ private final Catalog.Loader catalogLoader;
+
+ private transient int numChannels;
+
+ private Map<Identifier, CdcRecordChannelComputer> channelComputers;
+ private Catalog catalog;
+
+ public CdcMultiplexRecordChannelComputer(Catalog.Loader catalogLoader) {
+ this.catalogLoader = catalogLoader;
+ }
+
+ @Override
+ public void setup(int numChannels) {
+ this.numChannels = numChannels;
+ this.catalog = catalogLoader.load();
+ }
+
+ @Override
+ public int channel(CdcMultiplexRecord multiplexRecord) {
+ return Objects.hash(
+ multiplexRecord.databaseName(),
+ multiplexRecord.tableName(),
+
computeChannelComputer(multiplexRecord).channel(multiplexRecord.record()))
+ % numChannels;
+ }
+
+ private ChannelComputer<CdcRecord>
computeChannelComputer(CdcMultiplexRecord record) {
+ return channelComputers.computeIfAbsent(
Review Comment:
NPE here.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.flink.action.cdc.mysql.MySqlDatabaseSyncMode;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.types.DataField;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * A {@link ProcessFunction} to parse CDC change event to either a list of
{@link DataField}s or
+ * {@link CdcRecord} and send them to different side outputs according to
table name. This process
+ * function will capture newly added tables when syncing entire database and
in cases where the
+ * newly added tables are including by attesting table filters.
+ *
+ * <p>This {@link ProcessFunction} can handle records for different tables at
the same time.
+ *
+ * @param <T> CDC change event type
+ */
+public class CdcDynamicTableParsingProcessFunction<T> extends
ProcessFunction<T, Void> {
+
+ private final EventParser.Factory<T> parserFactory;
Review Comment:
```
public static final OutputTag<CdcMultiplexRecord> DYNAMIC_OUTPUT_TAG =
new OutputTag<>("paimon-dynamic-table",
TypeInformation.of(CdcMultiplexRecord.class));
public static final OutputTag<Tuple2<Identifier, List<DataField>>>
DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG =
new OutputTag<>(
"paimon-dynamic-table-schema-change",
TypeInformation.of(
new TypeHint<Tuple2<Identifier,
List<DataField>>>() {}));
private final EventParser.Factory<T> parserFactory;
private final String database;
private final Catalog.Loader catalogLoader;
private transient EventParser<T> parser;
private transient Catalog catalog;
```
Is enough
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.flink.VersionedSerializerWrapper;
+import org.apache.paimon.flink.sink.CommittableStateManager;
+import org.apache.paimon.flink.sink.Committer;
+import org.apache.paimon.flink.sink.CommitterOperator;
+import org.apache.paimon.flink.sink.FlinkSink;
+import org.apache.paimon.flink.sink.MultiTableCommittable;
+import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo;
+import org.apache.paimon.flink.sink.RestoreAndFailCommittableStateManager;
+import org.apache.paimon.flink.sink.StoreMultiCommitter;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.flink.sink.StoreSinkWriteImpl;
+import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer;
+import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
+import org.apache.paimon.manifest.WrappedManifestCommittable;
+import org.apache.paimon.operation.Lock;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.SerializableFunction;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * A {@link FlinkSink} which accepts {@link CdcRecord} and waits for a schema
change if necessary.
+ */
+public class FlinkCdcMultiTableSink implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private static final String WRITER_NAME = "CDC MultiplexWriter";
+ private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global
Committer";
+ private final Lock.Factory lockFactory;
Review Comment:
static final and final, should have a empty line.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.schema.SchemaChange;
+import org.apache.paimon.schema.SchemaManager;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataTypeChecks;
+import org.apache.paimon.types.DataTypeRoot;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.Preconditions;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * A {@link ProcessFunction} to handle schema changes. New schema is
represented by a list of {@link
+ * DataField}s.
+ *
+ * <p>NOTE: To avoid concurrent schema changes, the parallelism of this {@link
ProcessFunction} must
+ * be 1.
+ */
+public class MultiTableUpdatedDataFieldsProcessFunction
+ extends ProcessFunction<Tuple2<Identifier, List<DataField>>, Void> {
+
+ private static final Logger LOG =
+
LoggerFactory.getLogger(MultiTableUpdatedDataFieldsProcessFunction.class);
+
+ private final Map<Identifier, SchemaManager> schemaManagers = new
HashMap<>();
+ private Catalog catalog;
+ private final Catalog.Loader catalogLoader;
+
+ public MultiTableUpdatedDataFieldsProcessFunction(Catalog.Loader
catalogLoader) {
+ this.catalogLoader = catalogLoader;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ this.catalog = catalogLoader.load();
+ }
+
+ @Override
+ public void processElement(
+ Tuple2<Identifier, List<DataField>> updatedDataFields,
+ Context context,
+ Collector<Void> collector)
+ throws Exception {
+ Identifier tableId = updatedDataFields.f0;
+ SchemaManager schemaManager =
+ schemaManagers.computeIfAbsent(
+ tableId,
+ id -> {
+ FileStoreTable table;
+ try {
+ table = (FileStoreTable)
catalog.getTable(tableId);
+ } catch (Catalog.TableNotExistException e) {
+ return null;
+ }
+ return new SchemaManager(table.fileIO(),
table.location());
+ });
+
+ if (Objects.isNull(schemaManager)) {
+ throw new IOException("Failed to get schema manager for table " +
tableId);
+ }
+
+ for (SchemaChange schemaChange :
+ extractSchemaChanges(schemaManager, updatedDataFields.f1)) {
+ applySchemaChange(schemaManager, schemaChange);
+ }
+ }
+
+ private List<SchemaChange> extractSchemaChanges(
+ SchemaManager schemaManager, List<DataField> updatedDataFields) {
+ RowType oldRowType = schemaManager.latest().get().logicalRowType();
+ Map<String, DataField> oldFields = new HashMap<>();
+ for (DataField oldField : oldRowType.getFields()) {
+ oldFields.put(oldField.name(), oldField);
+ }
+
+ List<SchemaChange> result = new ArrayList<>();
+ for (DataField newField : updatedDataFields) {
+ if (oldFields.containsKey(newField.name())) {
+ DataField oldField = oldFields.get(newField.name());
+ // we compare by ignoring nullable, because partition keys and
primary keys might be
+ // nullable in source database, but they can't be null in
Paimon
+ if (oldField.type().equalsIgnoreNullable(newField.type())) {
+ if (!Objects.equals(oldField.description(),
newField.description())) {
+ result.add(
+ SchemaChange.updateColumnComment(
+ new String[] {newField.name()},
newField.description()));
+ }
+ } else {
+ result.add(SchemaChange.updateColumnType(newField.name(),
newField.type()));
+ if (newField.description() != null) {
+ result.add(
+ SchemaChange.updateColumnComment(
+ new String[] {newField.name()},
newField.description()));
+ }
+ }
+ } else {
+ result.add(
+ SchemaChange.addColumn(
+ newField.name(), newField.type(),
newField.description(), null));
+ }
+ }
+ return result;
+ }
+
+ private void applySchemaChange(SchemaManager schemaManager, SchemaChange
schemaChange)
+ throws Exception {
+ if (schemaChange instanceof SchemaChange.AddColumn) {
Review Comment:
duplicate codes.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java:
##########
@@ -85,55 +99,94 @@ public void build() {
Preconditions.checkNotNull(parserFactory);
StreamExecutionEnvironment env = input.getExecutionEnvironment();
+ if (mode == MySqlDatabaseSyncMode.DYNAMIC) {
+ buildDynamicCdcSink(env);
+ } else {
+ buildStaticCdcSink(env);
+ }
+ }
+ private void buildDynamicCdcSink(StreamExecutionEnvironment env) {
+ SingleOutputStreamOperator<Void> parsed =
+ input.forward()
+ .process(
+ new CdcDynamicTableParsingProcessFunction<>(
+ database, catalogLoader, tables,
parserFactory, mode))
+ .setParallelism(input.getParallelism());
+
+ // for newly-added tables, create a multiplexing operator that handles
all their records
+ // and writes to multiple tables
+ DataStream<CdcMultiplexRecord> newlyAddedTableStream =
+ SingleOutputStreamOperatorUtils.getSideOutput(
+ parsed,
CdcDynamicTableParsingProcessFunction.DYNAMIC_OUTPUT_TAG);
+ // handles schema change for newly added tables
+ SingleOutputStreamOperatorUtils.getSideOutput(
+ parsed,
+
CdcDynamicTableParsingProcessFunction.DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG)
+ .process(new
MultiTableUpdatedDataFieldsProcessFunction(catalogLoader));
+
+ FlinkStreamPartitioner<CdcMultiplexRecord> partitioner =
Review Comment:
FlinkStreamPartitioner.partition
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.flink.VersionedSerializerWrapper;
+import org.apache.paimon.flink.sink.CommittableStateManager;
+import org.apache.paimon.flink.sink.Committer;
+import org.apache.paimon.flink.sink.CommitterOperator;
+import org.apache.paimon.flink.sink.FlinkSink;
+import org.apache.paimon.flink.sink.MultiTableCommittable;
+import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo;
+import org.apache.paimon.flink.sink.RestoreAndFailCommittableStateManager;
+import org.apache.paimon.flink.sink.StoreMultiCommitter;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.flink.sink.StoreSinkWriteImpl;
+import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer;
+import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
+import org.apache.paimon.manifest.WrappedManifestCommittable;
+import org.apache.paimon.operation.Lock;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.SerializableFunction;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * A {@link FlinkSink} which accepts {@link CdcRecord} and waits for a schema
change if necessary.
+ */
+public class FlinkCdcMultiTableSink implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private static final String WRITER_NAME = "CDC MultiplexWriter";
+ private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global
Committer";
+ private final Lock.Factory lockFactory;
+ private final boolean isOverwrite = false;
+ private final Catalog.Loader catalogLoader;
+
+ public FlinkCdcMultiTableSink(Catalog.Loader catalogLoader, Lock.Factory
lockFactory) {
+ this.catalogLoader = catalogLoader;
+ this.lockFactory = lockFactory;
+ }
+
+ private StoreSinkWrite.Provider createWriteProvider(CheckpointConfig
checkpointConfig) {
Review Comment:
checkpointConfig useless
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlDatabaseSyncMode.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.paimon.flink.action.cdc.mysql;
+
+import java.io.Serializable;
+
+/**
+ * There are two modes for database sync. 1) STATIC mode, only write record
from static tables.
+ * Newly added tables during runtime are not synced. 2) DYNAMIC mode, all
records from static tables
+ * and newly added tables are routed into new operators.
+ */
+public enum MySqlDatabaseSyncMode implements Serializable {
+ STATIC,
+ DYNAMIC
Review Comment:
Can we find better names to these two? Dynamic should be very confused with
Dynamic bucket mode... (rebase master to see bucket mode in sink)
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.flink.action.cdc.mysql.MySqlDatabaseSyncMode;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.types.DataField;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.OutputTag;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * A {@link ProcessFunction} to parse CDC change event to either a list of
{@link DataField}s or
+ * {@link CdcRecord} and send them to different side outputs according to
table name. This process
+ * function will capture newly added tables when syncing entire database and
in cases where the
+ * newly added tables are including by attesting table filters.
+ *
+ * <p>This {@link ProcessFunction} can handle records for different tables at
the same time.
+ *
+ * @param <T> CDC change event type
+ */
+public class CdcDynamicTableParsingProcessFunction<T> extends
ProcessFunction<T, Void> {
+
+ private final EventParser.Factory<T> parserFactory;
+ private final Set<String> initialTables;
+ private final String database;
+ private final Catalog.Loader catalogLoader;
+ private final MySqlDatabaseSyncMode mode;
+
+ private transient EventParser<T> parser;
+ private transient Catalog catalog;
+ private transient Map<String, OutputTag<List<DataField>>>
updatedDataFieldsOutputTags;
+ private transient Map<String, OutputTag<CdcRecord>> recordOutputTags;
+ public static final OutputTag<CdcMultiplexRecord> DYNAMIC_OUTPUT_TAG =
+ new OutputTag<>("paimon-dynamic-table",
TypeInformation.of(CdcMultiplexRecord.class));
+ public static final OutputTag<Tuple2<Identifier, List<DataField>>>
+ DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG =
+ new OutputTag<>(
+ "paimon-dynamic-table-schema-change",
+ TypeInformation.of(
+ new TypeHint<Tuple2<Identifier,
List<DataField>>>() {}));
+
+ public CdcDynamicTableParsingProcessFunction(
+ String database,
+ Catalog.Loader catalogLoader,
+ List<FileStoreTable> tables,
+ EventParser.Factory<T> parserFactory) {
+ this(database, catalogLoader, tables, parserFactory,
MySqlDatabaseSyncMode.STATIC);
+ }
+
+ public CdcDynamicTableParsingProcessFunction(
+ String database,
+ Catalog.Loader catalogLoader,
+ List<FileStoreTable> tables,
+ EventParser.Factory<T> parserFactory,
+ MySqlDatabaseSyncMode mode) {
+ // for now, only support single database
+ this.database = database;
+ this.catalogLoader = catalogLoader;
+ this.initialTables =
tables.stream().map(FileStoreTable::name).collect(Collectors.toSet());
+ this.parserFactory = parserFactory;
+ this.mode = mode;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ parser = parserFactory.create();
+ updatedDataFieldsOutputTags = new HashMap<>();
+ recordOutputTags = new HashMap<>();
+ catalog = catalogLoader.load();
+ }
+
+ @Override
+ public void processElement(T raw, Context context, Collector<Void>
collector) throws Exception {
+ parser.setRawEvent(raw);
+
+ // CDC Ingestion only supports single database at this time being.
+ // In the future, there will be a mapping between source databases
+ // and target paimon databases
+ // TODO: support multiple databases
+ // String databaseName = parser.parseDatabaseName();
+ String tableName = parser.parseTableName();
+
+ // check for newly added table
+ parser.parseNewTable(database)
+ .ifPresent(
+ schema -> {
+ Identifier identifier =
+ new Identifier(database,
parser.parseTableName());
+ try {
+ catalog.createTable(identifier, schema, true);
+ } catch (Throwable ignored) {
+ }
+ });
+
+ List<DataField> schemaChange = parser.parseSchemaChange();
+ if (schemaChange.size() > 0) {
+ context.output(
+ DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG,
+ Tuple2.of(Identifier.create(database, tableName),
schemaChange));
+ }
+
+ parser.parseRecords()
+ .forEach(
+ record ->
+ context.output(
+ DYNAMIC_OUTPUT_TAG,
+ wrapRecord(database, tableName,
record)));
+ }
+
+ private CdcMultiplexRecord wrapRecord(String databaseName, String
tableName, CdcRecord record) {
+ return CdcMultiplexRecord.fromCdcRecord(databaseName, tableName,
record);
+ }
+
+ private OutputTag<List<DataField>> getUpdatedDataFieldsOutputTag(String
tableName) {
+ return updatedDataFieldsOutputTags.computeIfAbsent(
+ tableName,
CdcDynamicTableParsingProcessFunction::createUpdatedDataFieldsOutputTag);
+ }
+
+ public static OutputTag<List<DataField>>
createUpdatedDataFieldsOutputTag(String tableName) {
+ return new OutputTag<>(
+ "new-data-field-list-" + tableName, new
ListTypeInfo<>(DataField.class));
+ }
+
+ private OutputTag<CdcRecord> getRecordOutputTag(String tableName) {
Review Comment:
remove lots of useless methods.
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.flink.VersionedSerializerWrapper;
+import org.apache.paimon.flink.sink.CommittableStateManager;
+import org.apache.paimon.flink.sink.Committer;
+import org.apache.paimon.flink.sink.CommitterOperator;
+import org.apache.paimon.flink.sink.FlinkSink;
+import org.apache.paimon.flink.sink.MultiTableCommittable;
+import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo;
+import org.apache.paimon.flink.sink.RestoreAndFailCommittableStateManager;
+import org.apache.paimon.flink.sink.StoreMultiCommitter;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.flink.sink.StoreSinkWriteImpl;
+import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer;
+import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
+import org.apache.paimon.manifest.WrappedManifestCommittable;
+import org.apache.paimon.operation.Lock;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.SerializableFunction;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * A {@link FlinkSink} which accepts {@link CdcRecord} and waits for a schema
change if necessary.
+ */
+public class FlinkCdcMultiTableSink implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private static final String WRITER_NAME = "CDC MultiplexWriter";
+ private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global
Committer";
+ private final Lock.Factory lockFactory;
+ private final boolean isOverwrite = false;
+ private final Catalog.Loader catalogLoader;
+
+ public FlinkCdcMultiTableSink(Catalog.Loader catalogLoader, Lock.Factory
lockFactory) {
+ this.catalogLoader = catalogLoader;
+ this.lockFactory = lockFactory;
+ }
+
+ private StoreSinkWrite.Provider createWriteProvider(CheckpointConfig
checkpointConfig) {
+ // for now, no compaction for multiplexed sink
+ return (table, commitUser, state, ioManager, memoryPool) ->
+ new StoreSinkWriteImpl(
+ table, commitUser, state, ioManager, isOverwrite,
false, memoryPool);
+ }
+
+ public DataStreamSink<?> sinkFrom(DataStream<CdcMultiplexRecord> input) {
+ // This commitUser is valid only for new jobs.
+ // After the job starts, this commitUser will be recorded into the
states of write and
+ // commit operators.
+ // When the job restarts, commitUser will be recovered from states and
this value is
+ // ignored.
+ String initialCommitUser = UUID.randomUUID().toString();
+ return sinkFrom(
+ input,
+ initialCommitUser,
+
createWriteProvider(input.getExecutionEnvironment().getCheckpointConfig()));
+ }
+
+ public DataStreamSink<?> sinkFrom(
+ DataStream<CdcMultiplexRecord> input,
+ String commitUser,
+ StoreSinkWrite.Provider sinkProvider) {
+ StreamExecutionEnvironment env = input.getExecutionEnvironment();
+ ReadableConfig conf =
StreamExecutionEnvironmentUtils.getConfiguration(env);
+ CheckpointConfig checkpointConfig = env.getCheckpointConfig();
+
+ boolean isStreaming =
+ conf.get(ExecutionOptions.RUNTIME_MODE) ==
RuntimeExecutionMode.STREAMING;
+ boolean streamingCheckpointEnabled =
+ isStreaming && checkpointConfig.isCheckpointingEnabled();
+ if (streamingCheckpointEnabled) {
+ assertCheckpointConfiguration(env);
+ }
+
+ MultiTableCommittableTypeInfo typeInfo = new
MultiTableCommittableTypeInfo();
+ SingleOutputStreamOperator<MultiTableCommittable> written =
+ input.transform(
+ WRITER_NAME,
+ typeInfo,
+ createWriteOperator(sinkProvider, isStreaming,
commitUser))
+ .setParallelism(input.getParallelism());
+
+ SingleOutputStreamOperator<?> committed =
+ written.transform(
+ GLOBAL_COMMITTER_NAME,
+ typeInfo,
+ new CommitterOperator<>(
+ streamingCheckpointEnabled,
+ commitUser,
+
createCommitterFactory(streamingCheckpointEnabled),
+ createCommittableStateManager()))
+ .setParallelism(1)
+ .setMaxParallelism(1);
+ return committed.addSink(new
DiscardingSink<>()).name("end").setParallelism(1);
+ }
+
+ private void assertCheckpointConfiguration(StreamExecutionEnvironment env)
{
+ Preconditions.checkArgument(
+ !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(),
+ "Paimon sink currently does not support unaligned checkpoints.
Please set "
+ + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key()
+ + " to false.");
+ Preconditions.checkArgument(
+ env.getCheckpointConfig().getCheckpointingMode() ==
CheckpointingMode.EXACTLY_ONCE,
+ "Paimon sink currently only supports EXACTLY_ONCE checkpoint
mode. Please set "
+ +
ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key()
+ + " to exactly-once");
+ }
+
+ protected OneInputStreamOperator<CdcMultiplexRecord,
MultiTableCommittable> createWriteOperator(
+ StoreSinkWrite.Provider writeProvider, boolean isStreaming, String
commitUser) {
Review Comment:
isStreaming useless
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.paimon.flink.sink.cdc;
+
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.flink.VersionedSerializerWrapper;
+import org.apache.paimon.flink.sink.CommittableStateManager;
+import org.apache.paimon.flink.sink.Committer;
+import org.apache.paimon.flink.sink.CommitterOperator;
+import org.apache.paimon.flink.sink.FlinkSink;
+import org.apache.paimon.flink.sink.MultiTableCommittable;
+import org.apache.paimon.flink.sink.MultiTableCommittableTypeInfo;
+import org.apache.paimon.flink.sink.RestoreAndFailCommittableStateManager;
+import org.apache.paimon.flink.sink.StoreMultiCommitter;
+import org.apache.paimon.flink.sink.StoreSinkWrite;
+import org.apache.paimon.flink.sink.StoreSinkWriteImpl;
+import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer;
+import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
+import org.apache.paimon.manifest.WrappedManifestCommittable;
+import org.apache.paimon.operation.Lock;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.SerializableFunction;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+/**
+ * A {@link FlinkSink} which accepts {@link CdcRecord} and waits for a schema
change if necessary.
+ */
+public class FlinkCdcMultiTableSink implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private static final String WRITER_NAME = "CDC MultiplexWriter";
+ private static final String GLOBAL_COMMITTER_NAME = "Multiplex Global
Committer";
+ private final Lock.Factory lockFactory;
+ private final boolean isOverwrite = false;
+ private final Catalog.Loader catalogLoader;
+
+ public FlinkCdcMultiTableSink(Catalog.Loader catalogLoader, Lock.Factory
lockFactory) {
+ this.catalogLoader = catalogLoader;
+ this.lockFactory = lockFactory;
+ }
+
+ private StoreSinkWrite.Provider createWriteProvider(CheckpointConfig
checkpointConfig) {
+ // for now, no compaction for multiplexed sink
+ return (table, commitUser, state, ioManager, memoryPool) ->
+ new StoreSinkWriteImpl(
+ table, commitUser, state, ioManager, isOverwrite,
false, memoryPool);
+ }
+
+ public DataStreamSink<?> sinkFrom(DataStream<CdcMultiplexRecord> input) {
+ // This commitUser is valid only for new jobs.
+ // After the job starts, this commitUser will be recorded into the
states of write and
+ // commit operators.
+ // When the job restarts, commitUser will be recovered from states and
this value is
+ // ignored.
+ String initialCommitUser = UUID.randomUUID().toString();
+ return sinkFrom(
+ input,
+ initialCommitUser,
+
createWriteProvider(input.getExecutionEnvironment().getCheckpointConfig()));
+ }
+
+ public DataStreamSink<?> sinkFrom(
+ DataStream<CdcMultiplexRecord> input,
+ String commitUser,
+ StoreSinkWrite.Provider sinkProvider) {
+ StreamExecutionEnvironment env = input.getExecutionEnvironment();
+ ReadableConfig conf =
StreamExecutionEnvironmentUtils.getConfiguration(env);
+ CheckpointConfig checkpointConfig = env.getCheckpointConfig();
+
+ boolean isStreaming =
+ conf.get(ExecutionOptions.RUNTIME_MODE) ==
RuntimeExecutionMode.STREAMING;
+ boolean streamingCheckpointEnabled =
+ isStreaming && checkpointConfig.isCheckpointingEnabled();
+ if (streamingCheckpointEnabled) {
+ assertCheckpointConfiguration(env);
+ }
+
+ MultiTableCommittableTypeInfo typeInfo = new
MultiTableCommittableTypeInfo();
+ SingleOutputStreamOperator<MultiTableCommittable> written =
+ input.transform(
+ WRITER_NAME,
+ typeInfo,
+ createWriteOperator(sinkProvider, isStreaming,
commitUser))
+ .setParallelism(input.getParallelism());
+
+ SingleOutputStreamOperator<?> committed =
+ written.transform(
+ GLOBAL_COMMITTER_NAME,
+ typeInfo,
+ new CommitterOperator<>(
+ streamingCheckpointEnabled,
+ commitUser,
+
createCommitterFactory(streamingCheckpointEnabled),
+ createCommittableStateManager()))
+ .setParallelism(1)
+ .setMaxParallelism(1);
+ return committed.addSink(new
DiscardingSink<>()).name("end").setParallelism(1);
+ }
+
+ private void assertCheckpointConfiguration(StreamExecutionEnvironment env)
{
+ Preconditions.checkArgument(
+ !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(),
+ "Paimon sink currently does not support unaligned checkpoints.
Please set "
+ + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key()
+ + " to false.");
+ Preconditions.checkArgument(
+ env.getCheckpointConfig().getCheckpointingMode() ==
CheckpointingMode.EXACTLY_ONCE,
+ "Paimon sink currently only supports EXACTLY_ONCE checkpoint
mode. Please set "
+ +
ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key()
+ + " to exactly-once");
+ }
+
+ protected OneInputStreamOperator<CdcMultiplexRecord,
MultiTableCommittable> createWriteOperator(
+ StoreSinkWrite.Provider writeProvider, boolean isStreaming, String
commitUser) {
+ return new CdcRecordStoreMultiWriteOperator(
+ catalogLoader, writeProvider, commitUser, new Options());
+ }
+
+ // Table committers are dynamically created at runtime
+ protected SerializableFunction<
+ String, Committer<MultiTableCommittable,
WrappedManifestCommittable>>
+ createCommitterFactory(boolean streamingCheckpointEnabled) {
Review Comment:
streamingCheckpointEnabled useless
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]