You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Tzu-Li (Gordon) Tai (JIRA)" <ji...@apache.org> on 2017/02/08 11:20:41 UTC

[jira] [Commented] (FLINK-5728) FlinkKafkaProducer should flush on checkpoint by default

    [ https://issues.apache.org/jira/browse/FLINK-5728?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15857847#comment-15857847 ] 

Tzu-Li (Gordon) Tai commented on FLINK-5728:
--------------------------------------------

Related:

I've recently gathered thoughts from [~uce] and [~till.rohrmann] that the methods `setLogFailuresOnly` and `setFlushOnCheckpoint` are over-complicating at-least-once guarantees for the Kafka producer. You can see by the docs for it here: https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/connectors/kafka.html#kafka-producers-and-fault-tolerance, rather complicated for the user ;-)

I think there was 2 main reasons why the `setFlushOnCheckpoint` was added in the first place (see https://github.com/apache/flink/pull/2108, I'm basically wrapping up):
- originally there was no flushing, so to avoid breaking user code behaviour, the method was added to allow users who know they need at-least-once to turn it on.
- make sure the producer was flexible and works in different situations.

There's 2 ways to go from here:
1. Once we make flush on checkpoint enabled by default, the user won't have to do anything for at-least-once as long as they turn on checkpointing. Still retain the methods for "advanced users" who need to tweak the settings.
2. Apart from changing default, also remove the methods completely. You'll always have at-least-once if checkpointing is enabled, at-most-once otherwise.

I personally prefer option 2, as I don't see the need to turn flushing off, when you somehow have needs for guarantees and turned on checkpointing in the first place. Also, the API is just simpler for the user and less things to worry about. I may be missing something though.

cc [~rmetzger] I'd also like to hear your take on this!

> FlinkKafkaProducer should flush on checkpoint by default
> --------------------------------------------------------
>
>                 Key: FLINK-5728
>                 URL: https://issues.apache.org/jira/browse/FLINK-5728
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>            Reporter: Tzu-Li (Gordon) Tai
>
> As discussed in FLINK-5702, it might be a good idea to let the FlinkKafkaProducer flush on checkpoints by default. Currently, it is disabled by default.
> It's a very simple change, but we should think about whether or not we want to break user behaviour, or have proper usage migration.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)