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
--- 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 not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---