-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
[Hotfix][Connector-V2][kafka] fix kafka sink config exactly-once exception #7857
base: dev
Are you sure you want to change the base?
Changes from all commits
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 |
---|---|---|
|
@@ -26,6 +26,7 @@ | |
import com.google.common.collect.Lists; | ||
import lombok.extern.slf4j.Slf4j; | ||
|
||
import java.time.Duration; | ||
import java.util.List; | ||
import java.util.Optional; | ||
import java.util.Properties; | ||
|
@@ -45,6 +46,7 @@ public class KafkaTransactionSender<K, V> implements KafkaProduceSender<K, V> { | |
private String transactionId; | ||
private final String transactionPrefix; | ||
private final Properties kafkaProperties; | ||
private int recordNumInTransaction = 0; | ||
|
||
public KafkaTransactionSender(String transactionPrefix, Properties kafkaProperties) { | ||
this.transactionPrefix = transactionPrefix; | ||
|
@@ -54,13 +56,15 @@ public KafkaTransactionSender(String transactionPrefix, Properties kafkaProperti | |
@Override | ||
public void send(ProducerRecord<K, V> producerRecord) { | ||
kafkaProducer.send(producerRecord); | ||
recordNumInTransaction++; | ||
} | ||
|
||
@Override | ||
public void beginTransaction(String transactionId) { | ||
this.transactionId = transactionId; | ||
this.kafkaProducer = getTransactionProducer(kafkaProperties, transactionId); | ||
kafkaProducer.beginTransaction(); | ||
recordNumInTransaction = 0; | ||
} | ||
|
||
@Override | ||
|
@@ -70,7 +74,8 @@ public Optional<KafkaCommitInfo> prepareCommit() { | |
transactionId, | ||
kafkaProperties, | ||
this.kafkaProducer.getProducerId(), | ||
this.kafkaProducer.getEpoch()); | ||
this.kafkaProducer.getEpoch(), | ||
this.kafkaProducer.isTxnStarted()); | ||
return Optional.of(kafkaCommitInfo); | ||
} | ||
|
||
|
@@ -107,6 +112,10 @@ public void abortTransaction(long checkpointId) { | |
|
||
@Override | ||
public List<KafkaSinkState> snapshotState(long checkpointId) { | ||
if (recordNumInTransaction == 0) { | ||
// KafkaSinkCommitter does not support emptyTransaction, so we commit here. | ||
kafkaProducer.commitTransaction(); | ||
} | ||
return Lists.newArrayList( | ||
new KafkaSinkState( | ||
transactionId, transactionPrefix, checkpointId, kafkaProperties)); | ||
|
@@ -116,7 +125,9 @@ public List<KafkaSinkState> snapshotState(long checkpointId) { | |
public void close() { | ||
if (kafkaProducer != null) { | ||
kafkaProducer.flush(); | ||
kafkaProducer.close(); | ||
// kafkaProducer will abort the transaction if you call close() without a duration arg | ||
// which will cause an exception when Committer commit the transaction later. | ||
kafkaProducer.close(Duration.ZERO); | ||
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. I suggest we move him to KafkaInternalProducer 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. @fcb-xiaobo Is it missing here |
||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -92,6 +92,7 @@ | |
import java.time.LocalDateTime; | ||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
|
@@ -752,6 +753,26 @@ public void testKafkaProtobufToAssert(TestContainer container) | |
}); | ||
} | ||
|
||
@TestTemplate | ||
public void testKafkaToKafkaExactlyOnce(TestContainer container) throws Exception { | ||
String sourceData = "{\"key\":\"SeaTunnel\",\"value\":\"kafka\"}"; | ||
for (int i = 0; i < 10; i++) { | ||
ProducerRecord<byte[], byte[]> record = | ||
new ProducerRecord<>("kafka_topic_exactly_once", null, sourceData.getBytes()); | ||
producer.send(record); | ||
} | ||
container.executeJob("/kafka/kafka_to_kafka_exactly_once.conf"); | ||
String topicName = "kafka_topic_exactly_once"; | ||
Map<String, String> data = getKafkaConsumerData(topicName); | ||
Assertions.assertEquals(10, data.size()); | ||
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. It's better to compare the data content 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. ok,I will update the test cases later |
||
Collection<String> values = data.values(); | ||
for (String value : values) { | ||
Map<String, String> node = JsonUtils.toMap(value); | ||
Assertions.assertEquals("SeaTunnel", node.get("key")); | ||
Assertions.assertEquals("kafka", node.get("value")); | ||
} | ||
} | ||
|
||
private @NotNull DefaultSeaTunnelRowSerializer getDefaultSeaTunnelRowSerializer( | ||
String topic, SeaTunnelRowType seaTunnelRowType, ReadonlyConfig readonlyConfig) { | ||
// Create serializer | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
# | ||
# 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. | ||
# | ||
env { | ||
execution.parallelism = 1 | ||
job.mode = "STREAMING" | ||
} | ||
|
||
source { | ||
Kafka { | ||
bootstrap.servers = "kafkaCluster:9092" | ||
topic = "kafka_topic_exactly_once" | ||
result_table_name = "kafka_topic_exactly_once" | ||
# The default format is json, which is optional | ||
format = json | ||
start_mode = earliest | ||
} | ||
|
||
} | ||
transform {} | ||
|
||
|
||
sink{ | ||
kafka { | ||
format = JSON | ||
topic = "kafka_topic_exactly_once" | ||
bootstrap.servers = "kafkaCluster:9092" | ||
semantics = EXACTLY_ONCE | ||
} | ||
} |
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.
It seems like it's always equal to 0