-
Notifications
You must be signed in to change notification settings - Fork 1.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Improve][Connector-V2][Kafka Sink]custom partition #2889
Changes from 5 commits
5b1ee04
ea581e0
572a9af
b967635
8442fd7
078aabb
8614fcc
a461055
de2fe6b
2f09ed7
4f8ec2c
c0a54c8
0e11275
3dfbd08
32ca0b3
9c3d016
90dc08f
eb50ffd
3527313
51b9229
189e987
0f92c00
9e3a082
dea8966
b145fa9
8cb6087
dee3a8c
8ddb6dc
9e83250
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
# Kafka | ||
|
||
> Kafka sink connector | ||
|
||
## Description | ||
|
||
Write Rows to a Kafka topic. | ||
|
||
## Key features | ||
|
||
- [x] [exactly-once](../../concept/connector-v2-features.md) | ||
|
||
By default, we will use 2pc to guarantee the message is sent to kafka exactly once. | ||
|
||
- [ ] [schema projection](../../concept/connector-v2-features.md) | ||
|
||
## Options | ||
|
||
| name | type | required | default value | | ||
| ------------------ | ------ | -------- | ------------- | | ||
| topic | string | yes | - | | ||
| bootstrap.servers | string | yes | - | | ||
| producer.* | string | no | - | | ||
| semantic | string | no | NON | | ||
| partition | int | no | - | | ||
| assign_partitions | list | no | - | | ||
| transaction_prefix | string | no | - | | ||
| common-options | string | no | - | | ||
|
||
### topic [string] | ||
|
||
Kafka Topic. | ||
|
||
### bootstrap.servers [string] | ||
|
||
Kafka Brokers List. | ||
|
||
### producer [string] | ||
|
||
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). | ||
|
||
The way to specify the parameter is to add the prefix `producer.` to the original parameter name. For example, the way to specify `request.timeout.ms` is: `producer.request.timeout.ms = 60000` . If these non-essential parameters are not specified, they will use the default values given in the official Kafka documentation. | ||
|
||
### semantic [string] | ||
|
||
Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | ||
|
||
In EXACTLY_ONCE, producer will write all messages in a Kafka transaction that will be committed to Kafka on a checkpoint. | ||
|
||
In AT_LEAST_ONCE, producer will wait for all outstanding messages in the Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. | ||
|
||
NON does not provide any guarantees: messages may be lost in case of issues on the Kafka broker and messages may be duplicated. | ||
|
||
### partition [int] | ||
|
||
We can specify the partition, all messages will be sent to this partition. | ||
|
||
### assign_partitions [list] | ||
|
||
We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | ||
|
||
For example, there are 5 partitions in total, and the assign_partitions field in config is as follows: | ||
assignpartitions = ["shoe", "clothing"] | ||
|
||
Then the message containing "shoe" will be sent to partition 0 (because "shoe" is subscripted as 0 in assign_partitions), and the message containing "clothing" will be sent to partition 1.For other messages, the hash algorithm will be used to divide them into the remaining partitions. | ||
|
||
### transaction_prefix [string] | ||
|
||
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. | ||
|
||
### common options [string] | ||
|
||
Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details. | ||
|
||
## Examples | ||
|
||
```hocon | ||
sink { | ||
|
||
kafka { | ||
topic = "seatunnel" | ||
bootstrap.servers = "localhost:9092" | ||
partition = 3 | ||
producer.acks = 1 | ||
producer.request.timeout.ms = 60000 | ||
semantics = EXACTLY_ONCE | ||
} | ||
|
||
} | ||
``` |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -40,7 +40,6 @@ public class Config { | |
*/ | ||
public static final String CONSUMER_GROUP = "consumer.group"; | ||
|
||
|
||
/** | ||
* consumer group of kafka client consume message. | ||
*/ | ||
|
@@ -50,4 +49,14 @@ public class Config { | |
* The prefix of kafka's transactionId, make sure different job use different prefix. | ||
*/ | ||
public static final String TRANSACTION_PREFIX = "transaction_prefix"; | ||
|
||
/** | ||
* Send information according to the specified partition. | ||
*/ | ||
public static final String PARTITION = "partition"; | ||
|
||
/** | ||
* Determine the partition to send based on the content of the message. | ||
*/ | ||
public static final String ASSIGN_PARTITIONS = "assign_partitions"; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove |
||
} |
Original file line number | Diff line number | Diff line change | ||||||||||
---|---|---|---|---|---|---|---|---|---|---|---|---|
|
@@ -25,6 +25,7 @@ | |||||||||||
|
||||||||||||
public class DefaultSeaTunnelRowSerializer implements SeaTunnelRowSerializer<byte[], byte[]> { | ||||||||||||
|
||||||||||||
private Integer partation = null; | ||||||||||||
private final String topic; | ||||||||||||
private final JsonSerializationSchema jsonSerializationSchema; | ||||||||||||
|
||||||||||||
|
@@ -33,8 +34,19 @@ public DefaultSeaTunnelRowSerializer(String topic, SeaTunnelRowType seaTunnelRow | |||||||||||
this.jsonSerializationSchema = new JsonSerializationSchema(seaTunnelRowType); | ||||||||||||
} | ||||||||||||
|
||||||||||||
public DefaultSeaTunnelRowSerializer(String topic, int partation, SeaTunnelRowType seaTunnelRowType) { | ||||||||||||
this.topic = topic; | ||||||||||||
this.partation = partation; | ||||||||||||
this.jsonSerializationSchema = new JsonSerializationSchema(seaTunnelRowType); | ||||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||||||||
} | ||||||||||||
|
||||||||||||
@Override | ||||||||||||
public ProducerRecord<byte[], byte[]> serializeRow(SeaTunnelRow row) { | ||||||||||||
return new ProducerRecord<>(topic, null, jsonSerializationSchema.serialize(row)); | ||||||||||||
if (this.partation != null) { | ||||||||||||
return new ProducerRecord<>(topic, this.partation, null, jsonSerializationSchema.serialize(row)); | ||||||||||||
} | ||||||||||||
else { | ||||||||||||
return new ProducerRecord<>(topic, null, jsonSerializationSchema.serialize(row)); | ||||||||||||
} | ||||||||||||
} | ||||||||||||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,54 @@ | ||
/* | ||
* 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.seatunnel.connectors.seatunnel.kafka.sink; | ||
|
||
import org.apache.kafka.clients.producer.Partitioner; | ||
import org.apache.kafka.common.Cluster; | ||
import org.apache.kafka.common.PartitionInfo; | ||
|
||
import java.util.List; | ||
import java.util.Map; | ||
|
||
public class CustomPartitioner implements Partitioner { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. rename |
||
List<String> assignPartitions = KafkaSinkWriter.getASSIGNPARTATIONS(); | ||
@Override | ||
public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) { | ||
List<PartitionInfo> partitions = cluster.partitionsForTopic(topic); | ||
int numPartitions = partitions.size(); | ||
|
||
int assignPartitionsSize = assignPartitions.size(); | ||
String message = new String(valueBytes); | ||
for (int i = 0; i < assignPartitionsSize; i++) { | ||
if (message.contains(assignPartitions.get(i))) { | ||
return i; | ||
} | ||
} | ||
//Choose one of the remaining partitions according to the hashcode. | ||
return (Math.abs(message.hashCode()) % (numPartitions - assignPartitionsSize)) + assignPartitionsSize; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Math.abs(message.hashCode()) -> message.hashCode() & Integer.MAX_VALUE There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. reference |
||
} | ||
|
||
@Override | ||
public void close() { | ||
|
||
} | ||
|
||
@Override | ||
public void configure(Map<String, ?> map) { | ||
|
||
} | ||
} |
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -17,6 +17,8 @@ | |||||
|
||||||
package org.apache.seatunnel.connectors.seatunnel.kafka.sink; | ||||||
|
||||||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.ASSIGN_PARTITIONS; | ||||||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PARTITION; | ||||||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC; | ||||||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TRANSACTION_PREFIX; | ||||||
|
||||||
|
@@ -51,11 +53,17 @@ public class KafkaSinkWriter implements SinkWriter<SeaTunnelRow, KafkaCommitInfo | |||||
|
||||||
private String transactionPrefix; | ||||||
private long lastCheckpointId = 0; | ||||||
private int partition; | ||||||
|
||||||
private final KafkaProduceSender<byte[], byte[]> kafkaProducerSender; | ||||||
private final SeaTunnelRowSerializer<byte[], byte[]> seaTunnelRowSerializer; | ||||||
|
||||||
private static final int PREFIX_RANGE = 10000; | ||||||
private static List<String> ASSIGNPARTATIONS; | ||||||
|
||||||
public static List<String> getASSIGNPARTATIONS() { | ||||||
return ASSIGNPARTATIONS; | ||||||
} | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. remove |
||||||
|
||||||
// check config | ||||||
@Override | ||||||
|
@@ -71,6 +79,12 @@ public KafkaSinkWriter( | |||||
List<KafkaSinkState> kafkaStates) { | ||||||
this.context = context; | ||||||
this.pluginConfig = pluginConfig; | ||||||
if (pluginConfig.hasPath(PARTITION)) { | ||||||
this.partition = pluginConfig.getInt(PARTITION); | ||||||
} | ||||||
if (pluginConfig.hasPath(ASSIGN_PARTITIONS)) { | ||||||
ASSIGNPARTATIONS = pluginConfig.getStringList(ASSIGN_PARTITIONS); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
} | ||||||
if (pluginConfig.hasPath(TRANSACTION_PREFIX)) { | ||||||
this.transactionPrefix = pluginConfig.getString(TRANSACTION_PREFIX); | ||||||
} else { | ||||||
|
@@ -129,6 +143,9 @@ private Properties getKafkaProperties(Config pluginConfig) { | |||||
kafkaConfig.entrySet().forEach(entry -> { | ||||||
kafkaProperties.put(entry.getKey(), entry.getValue().unwrapped()); | ||||||
}); | ||||||
if (pluginConfig.hasPath(ASSIGN_PARTITIONS)) { | ||||||
kafkaProperties.put(ProducerConfig.PARTITIONER_CLASS_CONFIG, "org.apache.seatunnel.connectors.seatunnel.kafka.sink.CustomPartitioner"); | ||||||
} | ||||||
TaoZex marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||
kafkaProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, pluginConfig.getString(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); | ||||||
kafkaProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); | ||||||
kafkaProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); | ||||||
|
@@ -137,7 +154,12 @@ private Properties getKafkaProperties(Config pluginConfig) { | |||||
|
||||||
// todo: parse the target field from config | ||||||
private SeaTunnelRowSerializer<byte[], byte[]> getSerializer(Config pluginConfig, SeaTunnelRowType seaTunnelRowType) { | ||||||
return new DefaultSeaTunnelRowSerializer(pluginConfig.getString(TOPIC), seaTunnelRowType); | ||||||
if (pluginConfig.hasPath(PARTITION)){ | ||||||
return new DefaultSeaTunnelRowSerializer(pluginConfig.getString(TOPIC), this.partition, seaTunnelRowType); | ||||||
} | ||||||
else { | ||||||
return new DefaultSeaTunnelRowSerializer(pluginConfig.getString(TOPIC), seaTunnelRowType); | ||||||
} | ||||||
} | ||||||
|
||||||
private KafkaSemantics getKafkaSemantics(Config pluginConfig) { | ||||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Replace with
### kafka.* [kafka producer config]
.