Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/2509#discussion_r83165323
--- Diff:
flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java
---
@@ -202,13 +204,53 @@ public void run() {
}
}
- // seek the consumer to the initial offsets
+ List<KafkaTopicPartition> partitionsWithNoOffset = new
ArrayList<>();
for (KafkaTopicPartitionState<TopicPartition> partition
: subscribedPartitions()) {
- if (partition.isOffsetDefined()) {
+ if (!partition.isOffsetDefined()) {
+
partitionsWithNoOffset.add(partition.getKafkaTopicPartition());
+ } else {
consumer.seek(partition.getKafkaPartitionHandle(), partition.getOffset() + 1);
}
}
+ if (partitionsWithNoOffset.size() ==
subscribedPartitions().length) {
+ // if all partitions have no initial offsets,
that means we're starting fresh
+ switch (startupMode) {
+ case EARLIEST:
+ LOG.info("Setting starting
point as earliest offset for partitions {}", partitionsWithNoOffset);
+
+ for
(KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions()) {
+
consumer.seekToBeginning(partition.getKafkaPartitionHandle());
+ }
+ break;
+ case LATEST:
+ LOG.info("Setting starting
point as latest offset for partitions {}", partitionsWithNoOffset);
+
+ for
(KafkaTopicPartitionState<TopicPartition> partition : subscribedPartitions()) {
+
consumer.seekToEnd(partition.getKafkaPartitionHandle());
+ }
+ break;
+ default:
+ case GROUP_OFFSETS:
+ LOG.info("Using group offsets
in Kafka of group.id {} as starting point for partitions {}",
+
kafkaProperties.getProperty("group.id"), partitionsWithNoOffset);
+ // don't need to do anything;
the KafkaConsumer by default finds group offsets from Kafka brokers
+ }
+ } else if (partitionsWithNoOffset.size() > 0 &&
partitionsWithNoOffset.size() < subscribedPartitions().length) {
+ // we are restoring from a
checkpoint/savepoint, but there are some new partitions that weren't
+ // subscribed by the consumer on the previous
execution; in this case, we set the starting offset
+ // of all new partitions to the earliest offset
+ LOG.info("Setting starting point as earliest
offset for newly created partitions after startup: {}", partitionsWithNoOffset);
+
+ for (KafkaTopicPartitionState<TopicPartition>
partition : subscribedPartitions()) {
+ if
(partitionsWithNoOffset.contains(partition.getKafkaTopicPartition())) {
+
consumer.seekToBeginning(partition.getKafkaPartitionHandle());
+ }
+ }
+ } else {
+ // restored from a checkpoint/savepoint, and
all partitions have starting offsets; don't need to do anything
+ }
+
--- End diff --
@rmetzger after looking at
https://issues.apache.org/jira/browse/FLINK-3037, do you think the proposed
changes here actually fixes that issue?
---
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 [email protected] or file a JIRA ticket
with INFRA.
---