yuxiqian commented on code in PR #3805:
URL: https://github.com/apache/flink-cdc/pull/3805#discussion_r1890051914


##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkOptions.java:
##########
@@ -79,4 +84,20 @@ public class KafkaDataSinkOptions {
                     .defaultValue("")
                     .withDescription(
                             "custom headers for each kafka record. Each header 
are separated by ',', separate key and value by ':'. For example, we can set 
headers like 'key1:value1,key2:value2'.");
+
+    public static final ConfigOption<String> SINK_TABLE_MAPPING =
+            key("sink.table.mapping")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "Custom table mappings for each 
table from upstream tableId to downstream Kafka topic. Each mapping is 
separated by ")
+                                    .text(DILIMITER_TABLE_MAPPINGS)
+                                    .text(
+                                            ", separate upstream tableId and 
downstream Kafka topic by ")
+                                    .text(DILIMITER_SELECTOR_TABLEID)

Review Comment:
   Delimiter is more common



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java:
##########
@@ -560,6 +560,68 @@ void testTopicAndHeaderOption() throws Exception {
         checkProducerLeak();
     }
 
+    @Test
+    void testSINKTABLEMAPPING() throws Exception {

Review Comment:
   ```suggestion
       void testSinkTableMapping() throws Exception {
   ```



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/utils/KafkaSinkUtils.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.kafka.utils;
+
+import org.apache.flink.cdc.common.event.TableId;
+import org.apache.flink.cdc.common.schema.Selectors;
+import org.apache.flink.cdc.common.utils.Preconditions;
+import org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static 
org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.DILIMITER_SELECTOR_TABLEID;
+import static 
org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSinkOptions.DILIMITER_TABLE_MAPPINGS;
+
+/** Util class for {@link 
org.apache.flink.cdc.connectors.kafka.sink.KafkaDataSink}. */
+public class KafkaSinkUtils {
+
+    /** Parse the mapping text to a map from Selectors to TableId. */
+    public static Map<Selectors, TableId> 
parseSelectorsToTableIdMapping(String tableMappings) {

Review Comment:
   Any reason we're yielding `Map<Selectors, TableId>` instead of 
`Map<Selectors, String>`? Seems there's no limitation that written-into Kafka 
topic must be a TableId in previous docs specification.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java:
##########
@@ -560,6 +560,68 @@ void testTopicAndHeaderOption() throws Exception {
         checkProducerLeak();
     }
 
+    @Test
+    void testSINKTABLEMAPPING() throws Exception {
+        final StreamExecutionEnvironment env = new LocalStreamEnvironment();
+        env.enableCheckpointing(1000L);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        final DataStream<Event> source =
+                env.fromCollection(createSourceEvents(), new EventTypeInfo());
+        Map<String, String> config = new HashMap<>();
+        config.put(
+                KafkaDataSinkOptions.SINK_TABLE_MAPPING.key(),
+                
"default_namespace.default_schema_copy.\\.*:test_topic_mapping_copy;default_namespace.default_schema.\\.*:test_topic_mapping");
+        Properties properties = getKafkaClientConfiguration();
+        properties.forEach(
+                (key, value) ->
+                        config.put(
+                                KafkaDataSinkOptions.PROPERTIES_PREFIX + 
key.toString(),
+                                value.toString()));
+        source.sinkTo(
+                ((FlinkSinkProvider)
+                                (new KafkaDataSinkFactory()
+                                        .createDataSink(
+                                                new 
FactoryHelper.DefaultContext(
+                                                        
Configuration.fromMap(config),
+                                                        
Configuration.fromMap(new HashMap<>()),
+                                                        
this.getClass().getClassLoader()))
+                                        .getEventSinkProvider()))
+                        .getSink());
+        env.execute();
+
+        final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
+                drainAllRecordsFromTopic("test_topic_mapping", false, 0);
+        final long recordsCount = 5;
+        assertThat(recordsCount).isEqualTo(collectedRecords.size());

Review Comment:
   ```suggestion
           assertThat(collectedRecords).hasSize(recordsCount);
   ```
   
   This might print better error message when assertion fails.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java:
##########
@@ -57,6 +59,13 @@ public class PipelineKafkaRecordSerializationSchema
     // key value pairs to be put into Kafka Record Header.
     public final Map<String, String> customHeaders;
 
+    private final String tableMapping;
+
+    private Map<Selectors, TableId> tableMappings;

Review Comment:
   `tableMappings` could be final. Also can we change names of the "mapping 
rule string" and "table mapping selectors map"? Their names are a little 
confusing.



##########
flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java:
##########
@@ -560,6 +560,68 @@ void testTopicAndHeaderOption() throws Exception {
         checkProducerLeak();
     }
 
+    @Test
+    void testSINKTABLEMAPPING() throws Exception {

Review Comment:
   Seems we can make this method more generic by making `tableMappingRule` and 
`expectedResults` as arguments. Thus, we can test more cases including regex / 
exact matching, partial mapping and fallback mapping.



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