Hello Chris, That is an interesting KIP. I have a couple of questions:
(1) in section of pseudo-code, what if the failure happens between 4(b) and 5(a), meaning after the producer commit the transaction, and before task.commitRecord(). (2) in section "source task life time", what is the difference between "commit offset" and "offsets to commit"? Given that the offset storage can be a Kafka topic (/KafkaOffsetBackingStore.java) and producer could only produce to a kafka topic, are / is the topic(s) the same ? (the topic that producer writes offsets to and the topic task.commit() to) (3) for JDBC source task, it relies on `context.offsetStorageReader()` (https://github.com/confluentinc/kafka-connect-jdbc/blob/master/src/main/java/io/confluent/connect/jdbc/source/JdbcSourceTask.java#L140) to retrieve the previously committed offset (if from a fresh start or resume from failure). so it seems that the single-source-of-truth of where to consume from last known / committed position stored in offset storage (e.g. kafka topic) managed by the periodic task.commit()? On 2020/05/22 06:20:51, Chris Egerton <chr...@confluent.io> wrote: > Hi all, > > I know it's a busy time with the upcoming 2.6 release and I don't expect > this to get a lot of traction until that's done, but I've published a KIP > for allowing atomic commit of offsets and records for source connectors and > would appreciate your feedback: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-618%3A+Atomic+commit+of+source+connector+records+and+offsets > > This feature should make it possible to implement source connectors with > exactly-once delivery guarantees, and even allow a wide range of existing > source connectors to provide exactly-once delivery guarantees with no > changes required. > > Cheers, > > Chris >