[
https://issues.apache.org/jira/browse/FLINK-4027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15332450#comment-15332450
]
ASF GitHub Bot commented on FLINK-4027:
---------------------------------------
GitHub user rmetzger opened a pull request:
https://github.com/apache/flink/pull/2108
[FLINK-4027] Flush FlinkKafkaProducer on checkpoints
A user on the mailing list raised the point that our Kafka producer can be
made at-least-once quite easily.
The current producer code doesn't have any guarantees
We are using the producer's callbacks to account for unacknowledged
records. When a checkpoint barrier reaches the sink, it will confirm the
checkpoint once all pending records have been acked.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/rmetzger/flink flink4027
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/2108.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #2108
----
commit d657ca8be1420a3e73c48bbbf65788fbd0b75c2c
Author: Robert Metzger <[email protected]>
Date: 2016-06-15T15:50:38Z
[FLINK-4027] Flush FlinkKafkaProducer on checkpoints
----
> FlinkKafkaProducer09 sink can lose messages
> -------------------------------------------
>
> Key: FLINK-4027
> URL: https://issues.apache.org/jira/browse/FLINK-4027
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector
> Affects Versions: 1.0.3
> Reporter: Elias Levy
> Priority: Critical
>
> The FlinkKafkaProducer09 sink appears to not offer at-least-once guarantees.
> The producer is publishing messages asynchronously. A callback can record
> publishing errors, which will be raised when detected. But as far as I can
> tell, there is no barrier to wait for async errors from the sink when
> checkpointing or to track the event time of acked messages to inform the
> checkpointing process.
> If a checkpoint occurs while there are pending publish requests, and the
> requests return a failure after the checkpoint occurred, those message will
> be lost as the checkpoint will consider them processed by the sink.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)