JingsongLi commented on code in PR #1261:
URL: https://github.com/apache/incubator-paimon/pull/1261#discussion_r1233987297


##########
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(

Review Comment:
   remove 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.

Review Comment:
   routed into new operators? new operator? just unified operator?



##########
paimon-flink/paimon-flink-common/src/main/resources/META-INF/licenses/LICENSE.jdom:
##########
@@ -0,0 +1,51 @@
+ Copyright (C) 2000-2004 Jason Hunter & Brett McLaughlin.
+ All rights reserved.
+
+This project bundles the following dependencies under the following license.
+

Review Comment:
   remove this file



##########
paimon-flink/paimon-flink-common/src/main/resources/META-INF/licenses/LICENSE.jacoco:
##########
@@ -0,0 +1,14 @@
+License

Review Comment:
   remove this



##########
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) {

Review Comment:
   can you rebase latest master to contain sink for dynamic bucket mode.



-- 
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]

Reply via email to