Hi Mohammad

I expected to find a description of a mechanism for detecting and ignoring 
duplicate events in the documentation, although I got the two-phase commit 
protocol issuing something utterly different.
Flink would not detect and ignore duplicate events when processing them but 
ensure checkpoint could avoid duplicate event stored via exactly-once mechanism 
with replayable source. If we want end-to-end exactly once, Flink need the 
sink-connector could provide the ability to support exactly once visibility and 
the main solution is two-phase commit protocol. Data might be sent to 
exactly-once sink twice but only be committed as visible when two-phase commit 
notified which is triggered by Flink's checkpoint mechanism.

Regarding the event-time processing and watermarking, I have got that if an 
event will be received late, after the allowed lateness time, it will be 
dropped even though I think it is an antithesis of exactly-once semantic.
Yes, allowed lateness is a compromise between exactly-once semantic and 
acceptable delay of streaming application. Flink cannot ensure all data sources 
could generate data without any late which is not the scope of a streaming 
system should do. If you really need to the exactly once in event-time 
processing in this scenario, I suggest to run a batch job later to consume all 
data source and use that result as a credible one. For processing-time data, 
use Flink to generate a credible result is enough.

Best
Yun Tang

________________________________
From: Mohammad NM <mnm1...@gmail.com>
Sent: Monday, December 30, 2019 2:41
To: user@flink.apache.org <user@flink.apache.org>
Subject: Exactly-once ambiguities

Dear Flink team,
I have some ambiguity when it comes to Flink's exactly-once guaranteeing.
1. Based on what I understand, when a failure occurs, some events will be 
replayed which causes them to appear twice in the computations. I cannot 
realize how the two-phase commit protocol can avoid this problem and guarantee 
exactly-once. I expected to find a description of a mechanism for detecting and 
ignoring duplicate events in the documentation, although I got the two-phase 
commit protocol issuing something utterly different.
2. Regarding the event-time processing and watermarking, I have got that if an 
event will be received late, after the allowed lateness time, it will be 
dropped even though I think it is an antithesis of exactly-once semantic.
I will be thankful if I receive your valuable description so that I can remove 
my ambiguities.
Yours faithfully,

Reply via email to