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
[FLINK-31363] Do not checkpoint a KafkaCommittable if the transaction was empty #15
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
c93c59b
[FLINK-31363] Add hasDataInTransaction flag in FlinkKafkaInternalProd…
tzulitai 73a6612
[FLINK-31363] Only produce a KafkaCommittable on pre-commit if data w…
tzulitai b930efa
[FLINK-31363] Add tests for committing resumed transactions
tzulitai 503a75f
[FLINK-31363] Add KafkaWriterTest for prepareCommit() on an empty tra…
tzulitai a7cdc55
[review] Use assertThatThrownBy
tzulitai File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
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 |
---|---|---|
|
@@ -63,16 +63,15 @@ class FlinkKafkaInternalProducerITCase { | |
private static final KafkaContainer KAFKA_CONTAINER = | ||
createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); | ||
|
||
private static final String TRANSACTION_PREFIX = "test-transaction-"; | ||
|
||
@Test | ||
void testInitTransactionId() { | ||
final String topic = "test-init-transactions"; | ||
final String transactionIdPrefix = "testInitTransactionId-"; | ||
try (FlinkKafkaInternalProducer<String, String> reuse = | ||
new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { | ||
int numTransactions = 20; | ||
for (int i = 1; i <= numTransactions; i++) { | ||
reuse.initTransactionId(TRANSACTION_PREFIX + i); | ||
reuse.initTransactionId(transactionIdPrefix + i); | ||
reuse.beginTransaction(); | ||
reuse.send(new ProducerRecord<>(topic, "test-value-" + i)); | ||
if (i % 2 == 0) { | ||
|
@@ -81,12 +80,58 @@ void testInitTransactionId() { | |
reuse.flush(); | ||
reuse.abortTransaction(); | ||
} | ||
assertNumTransactions(i); | ||
assertNumTransactions(i, transactionIdPrefix); | ||
assertThat(readRecords(topic).count()).isEqualTo(i / 2); | ||
} | ||
} | ||
} | ||
|
||
@Test | ||
void testCommitResumedTransaction() { | ||
final String topic = "test-commit-resumed-transaction"; | ||
final String transactionIdPrefix = "testCommitResumedTransaction-"; | ||
final String transactionalId = transactionIdPrefix + "id"; | ||
|
||
KafkaCommittable snapshottedCommittable; | ||
try (FlinkKafkaInternalProducer<String, String> producer = | ||
new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { | ||
producer.initTransactions(); | ||
producer.beginTransaction(); | ||
producer.send(new ProducerRecord<>(topic, "test-value")); | ||
producer.flush(); | ||
snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); | ||
} | ||
|
||
try (FlinkKafkaInternalProducer<String, String> resumedProducer = | ||
new FlinkKafkaInternalProducer<>(getProperties(), transactionalId)) { | ||
resumedProducer.resumeTransaction( | ||
snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); | ||
resumedProducer.commitTransaction(); | ||
} | ||
|
||
assertNumTransactions(1, transactionIdPrefix); | ||
assertThat(readRecords(topic).count()).isEqualTo(1); | ||
} | ||
|
||
@Test | ||
void testCommitResumedEmptyTransactionShouldFail() { | ||
KafkaCommittable snapshottedCommittable; | ||
try (FlinkKafkaInternalProducer<String, String> producer = | ||
new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { | ||
producer.initTransactions(); | ||
producer.beginTransaction(); | ||
snapshottedCommittable = KafkaCommittable.of(producer, ignored -> {}); | ||
} | ||
|
||
try (FlinkKafkaInternalProducer<String, String> resumedProducer = | ||
new FlinkKafkaInternalProducer<>(getProperties(), "dummy")) { | ||
resumedProducer.resumeTransaction( | ||
snapshottedCommittable.getProducerId(), snapshottedCommittable.getEpoch()); | ||
|
||
assertThatThrownBy(resumedProducer::commitTransaction); | ||
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. nit: Can we check the exception type also using |
||
} | ||
} | ||
|
||
@ParameterizedTest | ||
@MethodSource("provideTransactionsFinalizer") | ||
void testResetInnerTransactionIfFinalizingTransactionFailed( | ||
|
@@ -131,10 +176,10 @@ private static Properties getProperties() { | |
FlinkKafkaInternalProducer::abortTransaction); | ||
} | ||
|
||
private void assertNumTransactions(int numTransactions) { | ||
private void assertNumTransactions(int numTransactions, String transactionIdPrefix) { | ||
List<KafkaTransactionLog.TransactionRecord> transactions = | ||
new KafkaTransactionLog(getProperties()) | ||
.getTransactions(id -> id.startsWith(TRANSACTION_PREFIX)); | ||
.getTransactions(id -> id.startsWith(transactionIdPrefix)); | ||
assertThat( | ||
transactions.stream() | ||
.map(KafkaTransactionLog.TransactionRecord::getTransactionId) | ||
|
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
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.
Should this boolean be set in the callback, in the successful send scenario?
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.
hmm, that's a good point. I think the question to ask is: is it incorrect to set this flag (to allow a
KafkaCommittable
to be generated for the txn at pre-commit time) preemptively, instead of only setting it when data has actually been successfully written?I think the answer is that it is not incorrect, so it is ok to leave this as is. Reasoning is as follows:
At pre-commit time and performing flush, if some data failed to be flushed, the pre-commit will fail so a
KafkaCommittable
will not be checkpointed for the txn anyways. In this scenario, thehasRecordsInTransaction
flag is irrelevant no matter its value.If all records are correctly flushed, then good; a
KafkaCommittable
should be generated for the txn. We're good here because we've alraedy preemptively set thehasRecordsInTransaction
flag.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.
Speaking of which, it might give a more complete picture of this interaction after I rebase this PR branch on top of the latest changes (to include the fix that adds
checkAsyncExceptions
).