Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-4723

Unify behaviour of committed offsets to Kafka / ZK for Kafka 0.8 and 0.9 consumer

    Details

    • Type: Improvement
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.2.0, 1.1.3
    • Component/s: Kafka Connector
    • Labels:
      None

      Description

      The proper "behaviour" of the offsets committed back to Kafka / ZK should be "the next offset that consumers should read (in Kafka terms, the 'position')".

      This is already fixed for the 0.9 consumer by FLINK-4618, by incrementing the committed offsets back to Kafka by the 0.9 by 1, so that the internal KafkaConsumer picks up the correct start position when committed offsets are present. This fix was required because the start position from committed offsets was implicitly determined with Kafka 0.9 APIs.

      However, since the 0.8 consumer handles offset committing and start position using Flink's own ZookeeperOffsetHandler and not Kafka's high-level APIs, the 0.8 consumer did not require a fix.

      I propose to still unify the behaviour of committed offsets across 0.8 and 0.9 to the definition above.

      Otherwise, if users in any case first uses the 0.8 consumer to read data and have Flink-committed offsets in ZK, and then uses a high-level 0.8 Kafka consumer to read the same topic in a non-Flink application, the first record will be duplicate (because, like described above, Kafka high-level consumers expect the committed offsets to be "the next record to process" and not "the last processed record").

      This requires incrementing the committed ZK offsets in 0.8 to also be incremented by 1, and changing how Flink internal offsets are initialized with accordance to the acquired ZK offsets.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the pull request:

          https://github.com/apache/flink/commit/53ed6adac8cbe6b5dcb692dc9b94970f3ec5887c#commitcomment-19258663

          Found some bugs in `KafkaConsumerBaseTest#testSnapshotState()` here, the code happened to workaround and bypass the bugs and asserts
          I'll fix this test as part of FLINK-4723(https://issues.apache.org/jira/browse/FLINK-4723) since I'll need to change this test over there.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the pull request: https://github.com/apache/flink/commit/53ed6adac8cbe6b5dcb692dc9b94970f3ec5887c#commitcomment-19258663 Found some bugs in `KafkaConsumerBaseTest#testSnapshotState()` here, the code happened to workaround and bypass the bugs and asserts I'll fix this test as part of FLINK-4723 ( https://issues.apache.org/jira/browse/FLINK-4723 ) since I'll need to change this test over there.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tzulitai opened a pull request:

          https://github.com/apache/flink/pull/2580

          FLINK-4723 [kafka-connector] Unify committed offsets to Kafka to be the next record to process

          The description within the JIRA ticket (FLINK-4723(https://issues.apache.org/jira/browse/FLINK-4723)) explains the reasoning for this change.

          With this change, offsets committed to Kafka are larger by 1 compared to the internally checkpointed offsets. This is changed at the `FlinkKafkaConsumerBase` level, so that offsets given through the abstract `commitSpecificOffsetsToKafka()` method to the version-specific implementations are already incremented and represent the next record to process. This way, the version-specific implementations simply commit the given offsets without the need to manipulate them.

          This PR also includes major refactoring of the IT tests to add commit offset related IT tests to `FlinkKafkaConsumerTestBase`, and let both the 0.8 and 0.9 consumers run offset committing / initial offset startup tests (previously only the 0.8 consumer had these tests).

          R: @rmetzger what's your take on this?

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tzulitai/flink FLINK-4723

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/2580.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 #2580


          commit cc782ffd4c174f23c45349771b318a08a2be75a3
          Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
          Date: 2016-10-02T08:54:57Z

          FLINK-4723 [kafka-connector] Unify committed offsets to Kafka to be next record to process


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tzulitai opened a pull request: https://github.com/apache/flink/pull/2580 FLINK-4723 [kafka-connector] Unify committed offsets to Kafka to be the next record to process The description within the JIRA ticket ( FLINK-4723 ( https://issues.apache.org/jira/browse/FLINK-4723 )) explains the reasoning for this change. With this change, offsets committed to Kafka are larger by 1 compared to the internally checkpointed offsets. This is changed at the `FlinkKafkaConsumerBase` level, so that offsets given through the abstract `commitSpecificOffsetsToKafka()` method to the version-specific implementations are already incremented and represent the next record to process. This way, the version-specific implementations simply commit the given offsets without the need to manipulate them. This PR also includes major refactoring of the IT tests to add commit offset related IT tests to `FlinkKafkaConsumerTestBase`, and let both the 0.8 and 0.9 consumers run offset committing / initial offset startup tests (previously only the 0.8 consumer had these tests). R: @rmetzger what's your take on this? You can merge this pull request into a Git repository by running: $ git pull https://github.com/tzulitai/flink FLINK-4723 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2580.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 #2580 commit cc782ffd4c174f23c45349771b318a08a2be75a3 Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org> Date: 2016-10-02T08:54:57Z FLINK-4723 [kafka-connector] Unify committed offsets to Kafka to be next record to process
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2580

          Seems like one of the new IT tests is a bit unstable, fixing it ...

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2580 Seems like one of the new IT tests is a bit unstable, fixing it ...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/2580

          Looks quite good. I would suggest one change, though:

          Can we avoid copying the offsets in the checkpoint into a new map (with increment by one) and passing that to the ZooKeeper Offset Committer or the Kafka Offset Committer? I am just not a big fan of copying things back and forth (especially in "prepareSnaoshot()", which we want to keep as lightweight as possible). Instead, can we have the contract that the offset committers always commit "+1" from the value they get (pretty much as it was in the 0.9 committer after FLINK-4618)?

          Concerning the tests, is the stability issue fixed there?
          What I frequently do is push the same commit to 10 different newly created branches to keep Travis busy over night with 10 test runs and see if I see a stability issue.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/2580 Looks quite good. I would suggest one change, though: Can we avoid copying the offsets in the checkpoint into a new map (with increment by one) and passing that to the ZooKeeper Offset Committer or the Kafka Offset Committer? I am just not a big fan of copying things back and forth (especially in "prepareSnaoshot()", which we want to keep as lightweight as possible). Instead, can we have the contract that the offset committers always commit "+1" from the value they get (pretty much as it was in the 0.9 committer after FLINK-4618 )? Concerning the tests, is the stability issue fixed there? What I frequently do is push the same commit to 10 different newly created branches to keep Travis busy over night with 10 test runs and see if I see a stability issue.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2580

          Thanks for the review @StephanEwen.

          Concerning changing the contract for `commitSpecificOffsetsToKafka`:
          Makes sense, I don't really like excessive copying too. With proper tests on both 0.8 and 0.9, I think it's reasonable to change this. I'll update this, and probably also rename `commitSpecficOffsetsToKafka` to reflect the contract behaviour.

          Thanks for the tip and test stability, I'll do that

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2580 Thanks for the review @StephanEwen. Concerning changing the contract for `commitSpecificOffsetsToKafka`: Makes sense, I don't really like excessive copying too. With proper tests on both 0.8 and 0.9, I think it's reasonable to change this. I'll update this, and probably also rename `commitSpecficOffsetsToKafka` to reflect the contract behaviour. Thanks for the tip and test stability, I'll do that
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rmetzger commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2580#discussion_r82577343

          — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java —
          @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception {
          }
          }
          }
          +
          + /**
          + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process"
          + */
          + public void runCommitOffsetsToKafka() throws Exception {
          + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
          + final int parallelism = 3;
          + final int recordsInEachPartition = 50;
          +
          + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
          +
          + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
          + env.getConfig().disableSysoutLogging();
          + env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
          + env.setParallelism(parallelism);
          + env.enableCheckpointing(200);
          +
          + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps));
          + stream.addSink(new DiscardingSink<String>());
          +
          + final AtomicReference<Throwable> errorRef = new AtomicReference<>();
          + final Thread runner = new Thread("runner") {
          + @Override
          + public void run() {
          + try

          { + env.execute(); + }

          + catch (Throwable t) {
          + if (!(t.getCause() instanceof JobCancellationException))

          { + errorRef.set(t); + }

          + }
          + }
          + };
          + runner.start();
          +
          + final Long l50 = 50L; // the final committed offset in Kafka should be 50
          + final long deadline = 30000 + System.currentTimeMillis();
          +
          + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
          +
          + do {
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          +
          + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3))

          { + break; + }

          +
          + Thread.sleep(100);
          + }
          + while (System.currentTimeMillis() < deadline);
          +
          + // cancel the job
          + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
          +
          + final Throwable t = errorRef.get();
          + if (t != null)

          { + throw new RuntimeException("Job failed with an exception", t); + }

          +
          + // final check to see if offsets are correctly in Kafka
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          + Assert.assertEquals(Long.valueOf(50L), o1);
          + Assert.assertEquals(Long.valueOf(50L), o2);
          + Assert.assertEquals(Long.valueOf(50L), o3);
          +
          + kafkaOffsetHandler.close();
          + deleteTestTopic(topicName);
          + }
          +
          + /**
          + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are
          + * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets.
          + * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up
          + * and starts at the correct position.
          + */
          + public void runStartFromKafkaCommitOffsets() throws Exception {
          + final int parallelism = 3;
          + final int recordsInEachPartition = 300;
          +
          + final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1);
          +
          + final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
          + env1.getConfig().disableSysoutLogging();
          + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
          + env1.setParallelism(parallelism);
          + env1.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets
          +
          + env1
          + .addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps))
          + .map(new ThrottledMapper<String>(50))
          + .map(new MapFunction<String, Object>() {
          + int count = 0;
          + @Override
          + public Object map(String value) throws Exception {
          + count++;
          + if (count == 150)

          { + throw new SuccessException(); + }

          + return null;
          + }
          + })
          + .addSink(new DiscardingSink<>());
          +
          + tryExecute(env1, "Read some records to commit offsets to Kafka");
          +
          + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
          +
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          +
          + LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3);
          — End diff –

          I wonder whether it makes sense to check that at least one of o1, o2 and o3 is not 300. If they are all 300 below test

          Show
          githubbot ASF GitHub Bot added a comment - Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/2580#discussion_r82577343 — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java — @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception { } } } + + /** + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process" + */ + public void runCommitOffsetsToKafka() throws Exception { + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50) + final int parallelism = 3; + final int recordsInEachPartition = 50; + + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.getConfig().disableSysoutLogging(); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(parallelism); + env.enableCheckpointing(200); + + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps)); + stream.addSink(new DiscardingSink<String>()); + + final AtomicReference<Throwable> errorRef = new AtomicReference<>(); + final Thread runner = new Thread("runner") { + @Override + public void run() { + try { + env.execute(); + } + catch (Throwable t) { + if (!(t.getCause() instanceof JobCancellationException)) { + errorRef.set(t); + } + } + } + }; + runner.start(); + + final Long l50 = 50L; // the final committed offset in Kafka should be 50 + final long deadline = 30000 + System.currentTimeMillis(); + + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps); + + do { + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) { + break; + } + + Thread.sleep(100); + } + while (System.currentTimeMillis() < deadline); + + // cancel the job + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); + + final Throwable t = errorRef.get(); + if (t != null) { + throw new RuntimeException("Job failed with an exception", t); + } + + // final check to see if offsets are correctly in Kafka + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + Assert.assertEquals(Long.valueOf(50L), o1); + Assert.assertEquals(Long.valueOf(50L), o2); + Assert.assertEquals(Long.valueOf(50L), o3); + + kafkaOffsetHandler.close(); + deleteTestTopic(topicName); + } + + /** + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are + * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets. + * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up + * and starts at the correct position. + */ + public void runStartFromKafkaCommitOffsets() throws Exception { + final int parallelism = 3; + final int recordsInEachPartition = 300; + + final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env1.getConfig().disableSysoutLogging(); + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env1.setParallelism(parallelism); + env1.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets + + env1 + .addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps)) + .map(new ThrottledMapper<String>(50)) + .map(new MapFunction<String, Object>() { + int count = 0; + @Override + public Object map(String value) throws Exception { + count++; + if (count == 150) { + throw new SuccessException(); + } + return null; + } + }) + .addSink(new DiscardingSink<>()); + + tryExecute(env1, "Read some records to commit offsets to Kafka"); + + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps); + + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + + LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3); — End diff – I wonder whether it makes sense to check that at least one of o1, o2 and o3 is not 300. If they are all 300 below test
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rmetzger commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2580#discussion_r82576069

          — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java —
          @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception {
          }
          }
          }
          +
          + /**
          + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process"
          + */
          + public void runCommitOffsetsToKafka() throws Exception {
          + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
          + final int parallelism = 3;
          + final int recordsInEachPartition = 50;
          +
          + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
          +
          + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
          + env.getConfig().disableSysoutLogging();
          + env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
          + env.setParallelism(parallelism);
          + env.enableCheckpointing(200);
          +
          + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps));
          + stream.addSink(new DiscardingSink<String>());
          +
          + final AtomicReference<Throwable> errorRef = new AtomicReference<>();
          + final Thread runner = new Thread("runner") {
          + @Override
          + public void run() {
          + try

          { + env.execute(); + }

          + catch (Throwable t) {
          + if (!(t.getCause() instanceof JobCancellationException))

          { + errorRef.set(t); + }

          + }
          + }
          + };
          + runner.start();
          +
          + final Long l50 = 50L; // the final committed offset in Kafka should be 50
          + final long deadline = 30000 + System.currentTimeMillis();
          +
          + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
          +
          + do {
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          +
          + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3))

          { + break; + }

          +
          + Thread.sleep(100);
          + }
          + while (System.currentTimeMillis() < deadline);
          +
          + // cancel the job
          + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
          +
          + final Throwable t = errorRef.get();
          + if (t != null)

          { + throw new RuntimeException("Job failed with an exception", t); + }

          +
          + // final check to see if offsets are correctly in Kafka
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          + Assert.assertEquals(Long.valueOf(50L), o1);
          + Assert.assertEquals(Long.valueOf(50L), o2);
          + Assert.assertEquals(Long.valueOf(50L), o3);
          +
          + kafkaOffsetHandler.close();
          + deleteTestTopic(topicName);
          + }
          +
          + /**
          + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are
          — End diff –

          300, 150

          Show
          githubbot ASF GitHub Bot added a comment - Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/2580#discussion_r82576069 — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java — @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception { } } } + + /** + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process" + */ + public void runCommitOffsetsToKafka() throws Exception { + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50) + final int parallelism = 3; + final int recordsInEachPartition = 50; + + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.getConfig().disableSysoutLogging(); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(parallelism); + env.enableCheckpointing(200); + + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps)); + stream.addSink(new DiscardingSink<String>()); + + final AtomicReference<Throwable> errorRef = new AtomicReference<>(); + final Thread runner = new Thread("runner") { + @Override + public void run() { + try { + env.execute(); + } + catch (Throwable t) { + if (!(t.getCause() instanceof JobCancellationException)) { + errorRef.set(t); + } + } + } + }; + runner.start(); + + final Long l50 = 50L; // the final committed offset in Kafka should be 50 + final long deadline = 30000 + System.currentTimeMillis(); + + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps); + + do { + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) { + break; + } + + Thread.sleep(100); + } + while (System.currentTimeMillis() < deadline); + + // cancel the job + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); + + final Throwable t = errorRef.get(); + if (t != null) { + throw new RuntimeException("Job failed with an exception", t); + } + + // final check to see if offsets are correctly in Kafka + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + Assert.assertEquals(Long.valueOf(50L), o1); + Assert.assertEquals(Long.valueOf(50L), o2); + Assert.assertEquals(Long.valueOf(50L), o3); + + kafkaOffsetHandler.close(); + deleteTestTopic(topicName); + } + + /** + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are — End diff – 300, 150
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2580

          Thanks for the review @rmetzger. I've created several local branches to test out the new IT tests stability as @StephanEwen suggested, and they seem to be fine.

          I'll rebase this, address the last few comments, and give the changes a final test run before merging.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2580 Thanks for the review @rmetzger. I've created several local branches to test out the new IT tests stability as @StephanEwen suggested, and they seem to be fine. I'll rebase this, address the last few comments, and give the changes a final test run before merging.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user rmetzger commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2580#discussion_r83222904

          — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java —
          @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception {
          }
          }
          }
          +
          + /**
          + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process"
          + */
          + public void runCommitOffsetsToKafka() throws Exception {
          + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50)
          + final int parallelism = 3;
          + final int recordsInEachPartition = 50;
          +
          + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1);
          +
          + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
          + env.getConfig().disableSysoutLogging();
          + env.getConfig().setRestartStrategy(RestartStrategies.noRestart());
          + env.setParallelism(parallelism);
          + env.enableCheckpointing(200);
          +
          + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps));
          + stream.addSink(new DiscardingSink<String>());
          +
          + final AtomicReference<Throwable> errorRef = new AtomicReference<>();
          + final Thread runner = new Thread("runner") {
          + @Override
          + public void run() {
          + try

          { + env.execute(); + }

          + catch (Throwable t) {
          + if (!(t.getCause() instanceof JobCancellationException))

          { + errorRef.set(t); + }

          + }
          + }
          + };
          + runner.start();
          +
          + final Long l50 = 50L; // the final committed offset in Kafka should be 50
          + final long deadline = 30000 + System.currentTimeMillis();
          +
          + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
          +
          + do {
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          +
          + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3))

          { + break; + }

          +
          + Thread.sleep(100);
          + }
          + while (System.currentTimeMillis() < deadline);
          +
          + // cancel the job
          + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout));
          +
          + final Throwable t = errorRef.get();
          + if (t != null)

          { + throw new RuntimeException("Job failed with an exception", t); + }

          +
          + // final check to see if offsets are correctly in Kafka
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          + Assert.assertEquals(Long.valueOf(50L), o1);
          + Assert.assertEquals(Long.valueOf(50L), o2);
          + Assert.assertEquals(Long.valueOf(50L), o3);
          +
          + kafkaOffsetHandler.close();
          + deleteTestTopic(topicName);
          + }
          +
          + /**
          + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are
          + * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets.
          + * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up
          + * and starts at the correct position.
          + */
          + public void runStartFromKafkaCommitOffsets() throws Exception {
          + final int parallelism = 3;
          + final int recordsInEachPartition = 300;
          +
          + final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1);
          +
          + final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort);
          + env1.getConfig().disableSysoutLogging();
          + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart());
          + env1.setParallelism(parallelism);
          + env1.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets
          +
          + env1
          + .addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps))
          + .map(new ThrottledMapper<String>(50))
          + .map(new MapFunction<String, Object>() {
          + int count = 0;
          + @Override
          + public Object map(String value) throws Exception {
          + count++;
          + if (count == 150)

          { + throw new SuccessException(); + }

          + return null;
          + }
          + })
          + .addSink(new DiscardingSink<>());
          +
          + tryExecute(env1, "Read some records to commit offsets to Kafka");
          +
          + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps);
          +
          + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0);
          + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1);
          + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2);
          +
          + LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3);
          — End diff –

          That sounds reasonable.

          Show
          githubbot ASF GitHub Bot added a comment - Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/2580#discussion_r83222904 — Diff: flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java — @@ -208,6 +207,235 @@ public void runFailOnNoBrokerTest() throws Exception { } } } + + /** + * Ensures that the committed offsets to Kafka are the offsets of "the next record to process" + */ + public void runCommitOffsetsToKafka() throws Exception { + // 3 partitions with 50 records each (0-49, so the expected commit offset of each partition should be 50) + final int parallelism = 3; + final int recordsInEachPartition = 50; + + final String topicName = writeSequence("testCommitOffsetsToKafkaTopic", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env.getConfig().disableSysoutLogging(); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env.setParallelism(parallelism); + env.enableCheckpointing(200); + + DataStream<String> stream = env.addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps)); + stream.addSink(new DiscardingSink<String>()); + + final AtomicReference<Throwable> errorRef = new AtomicReference<>(); + final Thread runner = new Thread("runner") { + @Override + public void run() { + try { + env.execute(); + } + catch (Throwable t) { + if (!(t.getCause() instanceof JobCancellationException)) { + errorRef.set(t); + } + } + } + }; + runner.start(); + + final Long l50 = 50L; // the final committed offset in Kafka should be 50 + final long deadline = 30000 + System.currentTimeMillis(); + + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps); + + do { + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + + if (l50.equals(o1) && l50.equals(o2) && l50.equals(o3)) { + break; + } + + Thread.sleep(100); + } + while (System.currentTimeMillis() < deadline); + + // cancel the job + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); + + final Throwable t = errorRef.get(); + if (t != null) { + throw new RuntimeException("Job failed with an exception", t); + } + + // final check to see if offsets are correctly in Kafka + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + Assert.assertEquals(Long.valueOf(50L), o1); + Assert.assertEquals(Long.valueOf(50L), o2); + Assert.assertEquals(Long.valueOf(50L), o3); + + kafkaOffsetHandler.close(); + deleteTestTopic(topicName); + } + + /** + * This test first writes a total of 200 records to a test topic, reads the first 100 so that some offsets are + * committed to Kafka, and then startup the consumer again to read the remaining records starting from the committed offsets. + * The test ensures that whatever offsets were committed to Kafka, the consumer correctly picks them up + * and starts at the correct position. + */ + public void runStartFromKafkaCommitOffsets() throws Exception { + final int parallelism = 3; + final int recordsInEachPartition = 300; + + final String topicName = writeSequence("testStartFromKafkaCommitOffsetsTopic", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env1 = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); + env1.getConfig().disableSysoutLogging(); + env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + env1.setParallelism(parallelism); + env1.enableCheckpointing(20); // fast checkpoints to make sure we commit some offsets + + env1 + .addSource(kafkaServer.getConsumer(topicName, new SimpleStringSchema(), standardProps)) + .map(new ThrottledMapper<String>(50)) + .map(new MapFunction<String, Object>() { + int count = 0; + @Override + public Object map(String value) throws Exception { + count++; + if (count == 150) { + throw new SuccessException(); + } + return null; + } + }) + .addSink(new DiscardingSink<>()); + + tryExecute(env1, "Read some records to commit offsets to Kafka"); + + KafkaTestEnvironment.KafkaOffsetHandler kafkaOffsetHandler = kafkaServer.createOffsetHandler(standardProps); + + Long o1 = kafkaOffsetHandler.getCommittedOffset(topicName, 0); + Long o2 = kafkaOffsetHandler.getCommittedOffset(topicName, 1); + Long o3 = kafkaOffsetHandler.getCommittedOffset(topicName, 2); + + LOG.info("Got final committed offsets from Kafka o1={}, o2={}, o3={}", o1, o2, o3); — End diff – That sounds reasonable.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2580

          Merging this once tests turn green.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2580 Merging this once tests turn green.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2580

          Merging this to master now ...

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2580 Merging this to master now ...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/2580

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/2580
          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - Resolved for master via http://git-wip-us.apache.org/repos/asf/flink/commit/f46ca39
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited

          Robert Metzger Should this fix go into release-1.1 as well?
          It will affect the behaviour of committed offsets for Kafka 0.8 users, so I'm not quite sure.

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - - edited Robert Metzger Should this fix go into release-1.1 as well? It will affect the behaviour of committed offsets for Kafka 0.8 users, so I'm not quite sure.
          Hide
          rmetzger Robert Metzger added a comment -

          I would not merge it because of the offset issues you've mentioned. So far, no 1.1.x user complained about the issue, so there is no immediate need for action.

          Show
          rmetzger Robert Metzger added a comment - I would not merge it because of the offset issues you've mentioned. So far, no 1.1.x user complained about the issue, so there is no immediate need for action.

            People

            • Assignee:
              tzulitai Tzu-Li (Gordon) Tai
              Reporter:
              tzulitai Tzu-Li (Gordon) Tai
            • Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development