You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Copy file name to clipboardExpand all lines: docs/content/docs/connectors/pipeline-connectors/kafka.md
+7Lines changed: 7 additions & 0 deletions
Original file line number
Diff line number
Diff line change
@@ -141,6 +141,13 @@ Pipeline Connector Options
141
141
<td>String</td>
142
142
<td>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'. </td>
143
143
</tr>
144
+
<tr>
145
+
<td>sink.tableId-to-topic.mapping</td>
146
+
<td>optional</td>
147
+
<td style="word-wrap: break-word;">(none)</td>
148
+
<td>String</td>
149
+
<td>Custom table mappings for each table from upstream tableId to downstream Kafka topic. Each mapping is separated by `;`, separate upstream tableId and downstream Kafka topic by `:`, For example, we can set `sink.tableId-to-topic.mapping` like `mydb.mytable1:topic1;mydb.mytable2:topic2`. </td>
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java
+7-2Lines changed: 7 additions & 2 deletions
Original file line number
Diff line number
Diff line change
@@ -53,6 +53,8 @@ public class KafkaDataSink implements DataSink {
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactory.java
Copy file name to clipboardExpand all lines: 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 {
79
84
.defaultValue("")
80
85
.withDescription(
81
86
"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'.");
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/PipelineKafkaRecordSerializationSchema.java
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactoryTest.java
Copy file name to clipboardExpand all lines: flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkITCase.java
0 commit comments