Dmitry.

let me do one step back, to help you better understand the tradeoffs:

A message will only be delivered multiple times in cause of failure --
ie, if a consumer crashed or timed out. For this case, another consumer
will take over the partitions assigned to the failing consumer and start
consumer from the latest committed offsets (committed by the failing
consumer).

Thus, if you commit more often, you get less duplicate reads in case of
failure.

The strongest guarantee you can get here is the case, when you disable
auto-commit, and commit your offsets after each processed message
manually. For this case, if a failure occurs, you will only get a single
message duplicate. Kafka cannot do more for you, as it does not know
(and cannot know) what processing you did with the message. Assume your
message is fully processed, and right before you want to call commit you
consumer fails -- thus the commit is lost, while the message was fully
processed. You would need to build some custom solution to track the
progress of processing in you app -- obviously, Kafka cannot help you
with this.

On the other hand, there is the possibility of "at-most-once" delivery.
If you apply the "commit every single message" scenario for this case,
you would call commit each time __before__ you start processing. Thus,
you would loose the message in case of failure, as the consumer talking
over the work, would not re-read the message as the offset got already
committed.

Does this make sense so far?

Going back to failure scenarios -- either you app crashes for some
"external" reason of a bug -- there is nothing you can do from a
configuration point of view.

For the timeout scenario. there are two timeout you need to consider:
max.session.timeout and max.poll.interval.ms -- this KIP explains the
details of both:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread

Hope this help. Keep in mind, that exactly-once feature will not help
you with you scenario -- and also note, that there is no message brokers
out their, that could give stronger semantics than Kafka -- I hope that
he description about offset commit and fail over explain why it's not
possible to give stronger guarantees.


-Matthias


On 4/13/17 4:41 PM, Dmitry Goldenberg wrote:
> Thanks, Matthias. Will read the doc you referenced.
> 
> The duplicates are on the consumer side. We've been trying to curtail this
> by increasing the consumer session timeout. Would that potentially help?
> 
> Basically, we're grappling with the causes of the behavior. Why would
> messages be ever delivered multiple times?
> 
> If we have to roll with a lookup table of "already seen" messages, it
> significantly complicates the architecture of our application. In the
> distributed case, we'll have to add something like Redis or memcached and
> the logic for doing the "distributed hashset" of seen messages. We'll also
> need a policy for purging of this hashset periodically.
> 
> I would think that "exactly once" would have to be exactly that. Consumer
> gets a given message just once.
> 
> Basically, we're developing what is a mission critical application, and
> having data loss due to "at most once" or data duplication due to "at least
> once" is pretty much unacceptable. The data loss we can't "sell" to
> application stakeholders.  The data duplication breaks our internal
> bookkeeping of the data processing flows.
> 
> In other words, we would ideally like to see message queueing capabilities
> in Kafka with very high exactly-once delivery guarantees...
> 
> Thanks,
> - Dmitry
> 
> 
> On Thu, Apr 13, 2017 at 7:00 PM, Matthias J. Sax <matth...@confluent.io>
> wrote:
> 
>> Hi,
>>
>> the first question to ask would be, if you get duplicate writes at the
>> producer or duplicate reads at the consumer...
>>
>> For exactly-once: it's work in progress and we aim for 0.11 release
>> (what might still be a beta version).
>>
>> In short, there will be an idempotent producer that will avoid duplicate
>> writes. Furthermore, the will be "transactions" that allow for
>> exactly-once "read-process-write" scenarios -- Kafka Streams will
>> leverage this feature.
>>
>> For reads, exactly-once will allow to only consumer committed messages.
>> But it does not help with duplicate reads.
>>
>> For duplicate reads, you cannot assume that "Kafka just does the right
>> thing" -- however, you can influence the potential number of duplicates
>> heavily. For example, you can reduce commit interval or even commit
>> manually (in the extreme case after each message). But even if you
>> commit after each message, your application needs to "track" the
>> progress of the currently processed message -- if you are in the middle
>> of processing and fail, Kafka cannot know what progress your application
>> made for the current message -- thus, it's up to you to decide on
>> restart, if you want to receive the message again or not... Kafka cannot
>> know this.
>>
>> If you want to get the full details about exactly-once, you can have a
>> look into the KIP:
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
>>
>> Hope this helps.
>>
>>
>> -Matthias
>>
>>
>> On 4/13/17 9:35 AM, Dmitry Goldenberg wrote:
>>> Thanks, Jayesh and Vincent.
>>>
>>> It seems rather extreme that one has to implement a cache of already seen
>>> messages using Redis, memcached or some such.  I would expect Kafka to
>> "do
>>> the right thing".  The data loss is a worse problem, especially for
>> mission
>>> critical applications.  So what is the current "stance" on the
>> exactly-once
>>> delivery semantic?
>>>
>>> - Dmitry
>>>
>>> On Thu, Apr 13, 2017 at 12:07 PM, Thakrar, Jayesh <
>>> jthak...@conversantmedia.com> wrote:
>>>
>>>> Hi Dmitri,
>>>>
>>>> This presentation might help you understand and take appropriate actions
>>>> to deal with data duplication (and data loss)
>>>>
>>>> https://www.slideshare.net/JayeshThakrar/kafka-68540012
>>>>
>>>> Regards,
>>>> Jayesh
>>>>
>>>> On 4/13/17, 10:05 AM, "Vincent Dautremont"
>> <vincent.dautrem...@olamobile.com.INVALID>
>>>> wrote:
>>>>
>>>>     One of the case where you would get a message more than once is if
>> you
>>>> get
>>>>     disconnected / kicked off the consumer group / etc if you fail to
>>>> commit
>>>>     offset for messages you have already read.
>>>>
>>>>     What I do is that I insert the message in a in-memory cache redis
>>>> database.
>>>>     If it fails to insert because of primary key duplication, well that
>>>> means
>>>>     I've already received that message in the past.
>>>>
>>>>     You could even do an insert of the topic+partition+offset of the
>>>> message
>>>>     payload as the insert (instead of the full message) if you know for
>>>> sure
>>>>     that your message payload would not be duplicated in the the kafka
>>>> topic.
>>>>
>>>>     Vincent.
>>>>
>>>>     On Thu, Apr 13, 2017 at 4:52 PM, Dmitry Goldenberg <
>>>> dgoldenb...@hexastax.com
>>>>     > wrote:
>>>>
>>>>     > Hi all,
>>>>     >
>>>>     > I was wondering if someone could list some of the causes which may
>>>> lead to
>>>>     > Kafka delivering the same messages more than once.
>>>>     >
>>>>     > We've looked around and we see no errors to notice, yet
>>>> intermittently, we
>>>>     > see messages being delivered more than once.
>>>>     >
>>>>     > Kafka documentation talks about the below delivery modes:
>>>>     >
>>>>     >    - *At most once*—Messages may be lost but are never
>> redelivered.
>>>>     >    - *At least once*—Messages are never lost but may be
>> redelivered.
>>>>     >    - *Exactly once*—this is what people actually want, each
>> message
>>>> is
>>>>     >    delivered once and only once.
>>>>     >
>>>>     > So the default is 'at least once' and that is what we're running
>>>> with (we
>>>>     > don't want to do "at most once" as that appears to yield some
>>>> potential for
>>>>     > message loss).
>>>>     >
>>>>     > We had not seen duplicated deliveries for a while previously but
>> just
>>>>     > started seeing them quite frequently in our test cluster.
>>>>     >
>>>>     > What are some of the possible causes for this?  What are some of
>> the
>>>>     > available tools for troubleshooting this issue? What are some of
>> the
>>>>     > possible fixes folks have developed or instrumented for this
>> issue?
>>>>     >
>>>>     > Also, is there an effort underway on Kafka side to provide support
>>>> for the
>>>>     > "exactly once" semantic?  That is exactly the semantic we want and
>>>> we're
>>>>     > wondering how that may be achieved.
>>>>     >
>>>>     > Thanks,
>>>>     > - Dmitry
>>>>     >
>>>>
>>>>     --
>>>>     The information transmitted is intended only for the person or
>> entity
>>>> to
>>>>     which it is addressed and may contain confidential and/or privileged
>>>>     material. Any review, retransmission, dissemination or other use
>> of, or
>>>>     taking of any action in reliance upon, this information by persons
>> or
>>>>     entities other than the intended recipient is prohibited. If you
>>>> received
>>>>     this in error, please contact the sender and delete the material
>> from
>>>> any
>>>>     computer.
>>>>
>>>>
>>>>
>>>
>>
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to