Nico Kruber created FLINK-7732: ---------------------------------- Summary: test instability in Kafa end-to-end test (invalid Kafka offset) Key: FLINK-7732 URL: https://issues.apache.org/jira/browse/FLINK-7732 Project: Flink Issue Type: Bug Components: Kafka Connector, Tests Affects Versions: 1.4.0 Reporter: Nico Kruber Priority: Critical
In a test run with unrelated changes in the network stack, the Kafa end-to-end test was failing with an invalid offset: {code} 2017-09-28 06:34:10,736 INFO org.apache.kafka.common.utils.AppInfoParser - Kafka version : 0.10.2.1 2017-09-28 06:34:10,744 INFO org.apache.kafka.common.utils.AppInfoParser - Kafka commitId : e89bffd6b2eff799 2017-09-28 06:34:14,549 INFO org.apache.kafka.clients.consumer.internals.AbstractCoordinator - Discovered coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 2147483647 rack: null) for group myconsumer. 2017-09-28 06:34:14,573 INFO org.apache.kafka.clients.consumer.internals.AbstractCoordinator - Marking the coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 2147483647 rack: null) dead for group myconsumer 2017-09-28 06:34:14,686 INFO org.apache.kafka.clients.consumer.internals.AbstractCoordinator - Discovered coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 2147483647 rack: null) for group myconsumer. 2017-09-28 06:34:14,687 INFO org.apache.kafka.clients.consumer.internals.AbstractCoordinator - Marking the coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 2147483647 rack: null) dead for group myconsumer 2017-09-28 06:34:14,792 INFO org.apache.kafka.clients.consumer.internals.AbstractCoordinator - Discovered coordinator testing-gce-f67551b8-8867-41c5-b36e-4196d0f11e30:9092 (id: 2147483647 rack: null) for group myconsumer. 2017-09-28 06:34:15,068 INFO org.apache.flink.runtime.state.DefaultOperatorStateBackend - DefaultOperatorStateBackend snapshot (In-Memory Stream Factory, synchronous part) in thread Thread[Async calls on Source: Custom Source -> Map -> Sink: Unnamed (1/1),5,Flink Task Threads] took 948 ms. 2017-09-28 06:34:15,164 WARN org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher - Committing offsets to Kafka failed. This does not compromise Flink's checkpoints. java.lang.IllegalArgumentException: Invalid offset: -915623761772 at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:687) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.doCommitOffsetsAsync(ConsumerCoordinator.java:531) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsAsync(ConsumerCoordinator.java:499) at org.apache.kafka.clients.consumer.KafkaConsumer.commitAsync(KafkaConsumer.java:1181) at org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread.run(KafkaConsumerThread.java:217) 2017-09-28 06:34:15,171 ERROR org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase - Async Kafka commit failed. java.lang.IllegalArgumentException: Invalid offset: -915623761772 at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:687) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.doCommitOffsetsAsync(ConsumerCoordinator.java:531) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsAsync(ConsumerCoordinator.java:499) at org.apache.kafka.clients.consumer.KafkaConsumer.commitAsync(KafkaConsumer.java:1181) at org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread.run(KafkaConsumerThread.java:217) {code} https://travis-ci.org/apache/flink/jobs/280722829 [~pnowojski] did a first analysis that revealed this: In org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java:229 this is being sent: {{long offsetToCommit = lastProcessedOffset + 1;}} {{lastProcessedOffset}} comes from: {{org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase#snapshotState}} either lines 741 or 749 The value that we see is strangely similiar to {{org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel#GROUP_OFFSET}} {code} /** * Magic number that defines the partition should start from its committed group offset in Kafka. * * <p>This is used as a placeholder so that the actual committed group offset can be evaluated lazily * when the partition will actually start to be read by the consumer. */ public static final long GROUP_OFFSET = -915623761773L; {code} This {{GROUP_OFFSET}} clearly was not intended to be committed, however probably there is some race between this "lazy evaluation" and checkpoints (which trigger committing offsets to kafka). -- This message was sent by Atlassian JIRA (v6.4.14#64029)