-
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
[Hotfix][Connector-V2][kafka] fix kafka sink config exactly-once exception #7857
Merged
Merged
Changes from all commits
Commits
Show all changes
22 commits
Select commit
Hold shift + click to select a range
3e84e2c
[Hotfix][Connector-V2][kafka] fix kafka sink config exactly-once exc…
fcb-xiaobo b1a314a
[Hotfix][Connector-V2][kafka] code review
fcb-xiaobo 238c6e2
update test
fcb-xiaobo 3fa9136
update test
fcb-xiaobo d346d5b
update test
fcb-xiaobo 2bdaf61
update test
fcb-xiaobo 6badc5a
update test
fcb-xiaobo 6209c92
update test
fcb-xiaobo 1ee5689
update test
fcb-xiaobo 2798051
update test
fcb-xiaobo 66ab98f
update test
fcb-xiaobo 8402937
update test
fcb-xiaobo 74b9a4e
update test
fcb-xiaobo 7c06ab6
update test
fcb-xiaobo 4c6d7ce
update test
fcb-xiaobo 3807088
update test
fcb-xiaobo 96abf3a
update test
fcb-xiaobo ef26c7b
update test
fcb-xiaobo 374e9f8
update test
fcb-xiaobo 4b8a78e
update test
fcb-xiaobo b67ac6c
update test
fcb-xiaobo e2f19e5
update test
fcb-xiaobo File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 |
||
} | ||
} | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
42 changes: 42 additions & 0 deletions
42
...2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafka_to_kafka_exactly_once_batch.conf
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
# | ||
# 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 { | ||
parallelism = 1 | ||
job.mode = "BATCH" | ||
} | ||
|
||
source { | ||
Kafka { | ||
bootstrap.servers = "kafkaCluster:9092" | ||
topic = "kafka_topic_exactly_once_1" | ||
# The default format is json, which is optional | ||
format = text | ||
start_mode = earliest | ||
} | ||
|
||
} | ||
transform {} | ||
|
||
|
||
sink{ | ||
kafka { | ||
format = text | ||
topic = "kafka_topic_exactly_once_2" | ||
bootstrap.servers = "kafkaCluster:9092" | ||
semantics = EXACTLY_ONCE | ||
} | ||
} |
44 changes: 44 additions & 0 deletions
44
...e/connector-kafka-e2e/src/test/resources/kafka/kafka_to_kafka_exactly_once_streaming.conf
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,44 @@ | ||
# | ||
# 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 { | ||
parallelism = 1 | ||
job.mode = "STREAMING" | ||
checkpoint.interval = 5000 | ||
checkpoint.timeout = 60000 | ||
} | ||
|
||
source { | ||
Kafka { | ||
bootstrap.servers = "kafkaCluster:9092" | ||
topic = "kafka_topic_exactly_once_1" | ||
# The default format is json, which is optional | ||
format = text | ||
start_mode = earliest | ||
} | ||
|
||
} | ||
transform {} | ||
|
||
|
||
sink{ | ||
kafka { | ||
format = text | ||
topic = "kafka_topic_exactly_once_2" | ||
bootstrap.servers = "kafkaCluster:9092" | ||
semantics = EXACTLY_ONCE | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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