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/en/connectors/sink/Kafka.md
+62Lines changed: 62 additions & 0 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -39,6 +39,7 @@ They can be downloaded via install-plugin.sh or from the Maven central repositor
39
39
| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). |
40
40
| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. |
41
41
| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. |
42
+
| kafka_headers_fields | Array | No | - | Configure which fields are used as the headers of the kafka message. The field value will be converted to a string and used as the header value. |
42
43
| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. |
43
44
| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. |
44
45
| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. |
@@ -90,6 +91,23 @@ If not set partition key fields, the null message key will be sent to.
90
91
The format of the message key is json, If name is set as the key, for example '{"name":"Jack"}'.
91
92
The selected field must be an existing field in the upstream.
92
93
94
+
### Kafka Headers Fields
95
+
96
+
For example, if you want to use value of fields from upstream data as kafka message headers, you can assign field names to this property.
Copy file name to clipboardExpand all lines: seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/KafkaSinkOptions.java
+8Lines changed: 8 additions & 0 deletions
Original file line number
Diff line number
Diff line change
@@ -46,6 +46,14 @@ public class KafkaSinkOptions extends KafkaBaseOptions {
46
46
.withDescription(
47
47
"Configure which fields are used as the key of the kafka message.");
Copy file name to clipboardExpand all lines: seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
+106-5Lines changed: 106 additions & 5 deletions
Original file line number
Diff line number
Diff line change
@@ -119,12 +119,23 @@ public static DefaultSeaTunnelRowSerializer create(
119
119
timestampExtractor(),
120
120
keyExtractor(null, rowType, format, delimiter, pluginConfig),
121
121
valueExtractor(rowType, format, delimiter, pluginConfig),
122
-
headersExtractor());
122
+
headersExtractor(null, rowType));
123
+
}
124
+
125
+
publicstaticDefaultSeaTunnelRowSerializercreate(
126
+
Stringtopic,
127
+
Integerpartition,
128
+
SeaTunnelRowTyperowType,
129
+
MessageFormatformat,
130
+
Stringdelimiter,
131
+
ReadonlyConfigpluginConfig) {
132
+
returncreate(topic, partition, null, rowType, format, delimiter, pluginConfig);
123
133
}
124
134
125
135
publicstaticDefaultSeaTunnelRowSerializercreate(
126
136
Stringtopic,
127
137
Integerpartition,
138
+
List<String> headerFields,
128
139
SeaTunnelRowTyperowType,
129
140
MessageFormatformat,
130
141
Stringdelimiter,
@@ -134,13 +145,24 @@ public static DefaultSeaTunnelRowSerializer create(
134
145
partitionExtractor(partition),
135
146
timestampExtractor(),
136
147
keyExtractor(null, rowType, format, delimiter, pluginConfig),
137
-
valueExtractor(rowType, format, delimiter, pluginConfig),
138
-
headersExtractor());
148
+
valueExtractor(headerFields, rowType, format, delimiter, pluginConfig),
149
+
headersExtractor(headerFields, rowType));
150
+
}
151
+
152
+
publicstaticDefaultSeaTunnelRowSerializercreate(
153
+
Stringtopic,
154
+
List<String> keyFields,
155
+
SeaTunnelRowTyperowType,
156
+
MessageFormatformat,
157
+
Stringdelimiter,
158
+
ReadonlyConfigpluginConfig) {
159
+
returncreate(topic, keyFields, null, rowType, format, delimiter, pluginConfig);
139
160
}
140
161
141
162
publicstaticDefaultSeaTunnelRowSerializercreate(
142
163
Stringtopic,
143
164
List<String> keyFields,
165
+
List<String> headerFields,
144
166
SeaTunnelRowTyperowType,
145
167
MessageFormatformat,
146
168
Stringdelimiter,
@@ -150,8 +172,8 @@ public static DefaultSeaTunnelRowSerializer create(
150
172
partitionExtractor(null),
151
173
timestampExtractor(),
152
174
keyExtractor(keyFields, rowType, format, delimiter, pluginConfig),
153
-
valueExtractor(rowType, format, delimiter, pluginConfig),
154
-
headersExtractor());
175
+
valueExtractor(headerFields, rowType, format, delimiter, pluginConfig),
0 commit comments