tzulitai commented on code in PR #15:
URL:
https://github.com/apache/flink-connector-kafka/pull/15#discussion_r1151177809
##########
flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/FlinkKafkaInternalProducer.java:
##########
@@ -67,6 +72,14 @@ private static Properties withTransactionalId(
return props;
}
+ @Override
+ public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback
callback) {
+ if (inTransaction) {
+ hasRecordsInTransaction = true;
Review Comment:
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 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, the
`hasRecordsInTransaction` flag is irrelevant anyways.
- 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 the `hasRecordsInTransaction` flag.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]