[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-04-08 Thread shikhar
Github user shikhar commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-207563659 Another thing is that presence of offsets in ZK (0.8) / Kafka (0.9) also affects what happens when a job is starting without a checkpoint / savepoint. If they are

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-04-07 Thread wanderingbort
Github user wanderingbort commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-207065222 I just wanted to echo @StephanEwen 's sentiment. Flink's management of offsets seems to supersede the Kafka concept of committed offsets (whether to ZK

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-04-05 Thread StephanEwen
Github user StephanEwen commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-205819299 I think the functionality is very desirable. In many cases, you really don't want Flink to write something to Zookeeper for some random GroupId. This floods

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-03-21 Thread rmetzger
Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-199316872 Let me summarize the change: ``` auto.commit.enable=true checkpointing=trueoffsets go to ZK/Broker on each completed checkpoint

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-03-18 Thread shikhar
Github user shikhar commented on a diff in the pull request: https://github.com/apache/flink/pull/1690#discussion_r56450314 --- Diff: flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java --- @@

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-03-14 Thread rmetzger
Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-196338135 Can you add a test case ensuring the functionality? If I'm not mistaken, the change would have introduced a pretty bad bug. --- If your project is set up for it, you

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-03-14 Thread rmetzger
Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/1690#discussion_r56009318 --- Diff: flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java --- @@

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-03-11 Thread rmetzger
Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-195374583 Thank you for working on this. I've been very busy with the 1.0 release, vacation and some talks, but now I'm back for my regular work. I try to review the PR today

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-02-25 Thread shikhar
Github user shikhar commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-188818665 Just fixed the `Kafka09ITCase.testCheckpointing()` fail as well, hopefully CI is green with that. --- If your project is set up for it, you can reply to this email and

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-02-23 Thread shikhar
Github user shikhar commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-188002588 @rmetzger the test failures were because we had `auto.commit.enable=false` in the standard `KafkaTestEnvironmentImpl` standard props and it wasn't respecting that

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-02-23 Thread rmetzger
Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/1690#issuecomment-187710646 The change is breaking some tests. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does

[GitHub] flink pull request: FLINK-3398: Allow for opting-out from Kafka of...

2016-02-22 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/flink/pull/1690 FLINK-3398: Allow for opting-out from Kafka offset auto-commit You can merge this pull request into a Git repository by running: $ git pull https://github.com/shikhar/flink