Re: Exactly-once semantics with kakfa CanCommitOffsets.commitAsync?

2017-04-28 Thread Cody Koeninger
It's asynchronous. If your job stopped before the commit happened, then of course it's not guaranteed to succeed. But even if those commits were somehow guaranteed to succeed even if your job stopped... you still need idempotent output operations. The point of transactionality isn't that it's

Re: Exactly-once semantics with kakfa CanCommitOffsets.commitAsync?

2017-04-28 Thread David Rosenstrauch
Yes, I saw that sentence too. But it's rather short and not very explanatory, and there doesn't seem to be any further info available anywhere that expands on it. When I parse out that sentence: 1) "Kafka is not transactional" - i.e., the commits are done asynchronously, not synchronously. 2)

Re: Exactly-once semantics with kakfa CanCommitOffsets.commitAsync?

2017-04-28 Thread Cody Koeninger
>From that doc: " However, Kafka is not transactional, so your outputs must still be idempotent. " On Fri, Apr 28, 2017 at 10:29 AM, David Rosenstrauch wrote: > I'm doing a POC to test recovery with spark streaming from Kafka. I'm using > the technique for storing the

Exactly-once semantics with kakfa CanCommitOffsets.commitAsync?

2017-04-28 Thread David Rosenstrauch
I'm doing a POC to test recovery with spark streaming from Kafka. I'm using the technique for storing the offsets in Kafka, as described at: https://spark.apache.org/docs/2.1.0/streaming-kafka-0-10-integration.html#kafka-itself I.e., grabbing the list of offsets before I start processing a