Thank you very much,
Would you think that Kafka-Streams with exactly_once flag enabled would perform 
better than kafka client with individual commit per message as timed below? 
Perhaps the implementation of exactly-once read-process-write is using other 
methods and its performance is better.
Indeed, incrementing a counter per processed message key in our database would 
be one way of accounting for duplicate processing, but I am not sure how can I 
do this in an efficient way (not querying all table).
Until now I concentrated on accounting for duplicate keys in topics via a 
kafka-streams job. That might be enough only if we transform a code that we 
have for main business logic in a pure function, and create the effect of 
writing to the database via a kafka connector. Since I understand both streams 
and connectors support exactly once, it would be a possibility to eliminate the 
possibility of duplicate processing downstream of a topic.

Thanks for your help,
Nicu
________________________________________
From: Sönke Liebau [soenke.lie...@opencore.com.INVALID]
Sent: Wednesday, February 21, 2018 4:59 PM
To: users@kafka.apache.org
Subject: Re: commiting consumed offsets synchronously (every message)

Hi Nicu,

committing after every message and thus retrieving them with a batch size
of 1 will definitely make a huge difference in performance!
I've rigged a quick (and totally non academic) test which came up with the
following numbers:

Batching consumer - Consumed 1000490 records in 5 seconds
Non Batching, commiting consumer - Consumed 1000000 records in 3023 seconds

The first line was a consumer with default settings and auto.offset.commit,
the second one retrieved messages one per poll and called commitSync after
every message.


I am not sure if you actually need this though, wouldn't your deduplication
process be able to check the downstream system, whether that specific
message was already processed and use that to identify duplicates?
Or are you not sending the actual records downstream but just doing
something like summing, counting, ... them?

It's tough to be more specific without knowing more specifics, but maybe
that helps a bit already?

Best regards,
Sönke

On Wed, Feb 21, 2018 at 11:57 AM, Marasoiu, Nicu <
nicu.maras...@metrosystems.net> wrote:
> Hi,
> In order to obtain an exactly-once semantics, we are thinking of doing
at-least-once processing, and then have a compensation mechanism to fix the
results in few minutes by correcting them by substracting the effects of
the duplicates. However, in order to do that, it seems that at least this
compensation mechanism needs to read from a topic and commit offsets every
message, so that when failover happens, it would not interpret as
duplicates the events from the latest commit until present. What are the
performance implications of this, and what advice would you have for
exactly-once behavior (at least with controllable error)?
> Thank you,
> Nicu Marasoiu
> Geschäftsanschrift/Business address: METRO SYSTEMS GmbH, Metro-Straße 12,
40235 Düsseldorf, Germany
> Aufsichtsrat/Supervisory Board: Heiko Hutmacher (Vorsitzender/ Chairman)
> Geschäftsführung/Management Board: Dr. Dirk Toepfer (Vorsitzender/CEO),
Wim van Herwijnen
> Sitz Düsseldorf, Amtsgericht Düsseldorf, HRB 18232/Registered Office
Düsseldorf, Commercial Register of the Düsseldorf Local Court, HRB 18232
>
> Betreffend Mails von *@metrosystems.net
> Die in dieser E-Mail enthaltenen Nachrichten und Anhänge sind
ausschließlich für den bezeichneten Adressaten bestimmt. Sie können
rechtlich geschützte, vertrauliche Informationen enthalten. Falls Sie nicht
der bezeichnete Empfänger oder zum Empfang dieser E-Mail nicht berechtigt
sind, ist die Verwendung, Vervielfältigung oder Weitergabe der Nachrichten
und Anhänge untersagt. Falls Sie diese E-Mail irrtümlich erhalten haben,
informieren Sie bitte unverzüglich den Absender und vernichten Sie die
E-Mail.
>
> Regarding mails from *@metrosystems.net
> This e-mail message and any attachment are intended exclusively for the
named addressee. They may contain confidential information which may also
be protected by professional secrecy. Unless you are the named addressee
(or authorised to receive for the addressee) you may not copy or use this
message or any attachment or disclose the contents to anyone else. If this
e-mail was sent to you by mistake please notify the sender immediately and
delete this e-mail.
>



--
Sönke Liebau
Partner
Tel. +49 179 7940878
OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel - Germany
Geschäftsanschrift/Business address: METRO SYSTEMS GmbH, Metro-Straße 12, 40235 
Düsseldorf, Germany
Aufsichtsrat/Supervisory Board: Heiko Hutmacher (Vorsitzender/ Chairman)
Geschäftsführung/Management Board: Dr. Dirk Toepfer (Vorsitzender/CEO), Wim van 
Herwijnen
Sitz Düsseldorf, Amtsgericht Düsseldorf, HRB 18232/Registered Office 
Düsseldorf, Commercial Register of the Düsseldorf Local Court, HRB 18232

Betreffend Mails von *@metrosystems.net
Die in dieser E-Mail enthaltenen Nachrichten und Anhänge sind ausschließlich 
für den bezeichneten Adressaten bestimmt. Sie können rechtlich geschützte, 
vertrauliche Informationen enthalten. Falls Sie nicht der bezeichnete Empfänger 
oder zum Empfang dieser E-Mail nicht berechtigt sind, ist die Verwendung, 
Vervielfältigung oder Weitergabe der Nachrichten und Anhänge untersagt. Falls 
Sie diese E-Mail irrtümlich erhalten haben, informieren Sie bitte unverzüglich 
den Absender und vernichten Sie die E-Mail.

Regarding mails from *@metrosystems.net
This e-mail message and any attachment are intended exclusively for the named 
addressee. They may contain confidential information which may also be 
protected by professional secrecy. Unless you are the named addressee (or 
authorised to receive for the addressee) you may not copy or use this message 
or any attachment or disclose the contents to anyone else. If this e-mail was 
sent to you by mistake please notify the sender immediately and delete this 
e-mail.

Reply via email to