[
https://issues.apache.org/jira/browse/FLINK-6006?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15923874#comment-15923874
]
ASF GitHub Bot commented on FLINK-6006:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/3505#discussion_r105860440
--- Diff:
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
---
@@ -489,16 +486,15 @@ public void notifyCheckpointComplete(long
checkpointId) throws Exception {
// Utilities
//
------------------------------------------------------------------------
- private void assignTopicPartitions(List<KafkaTopicPartition>
kafkaTopicPartitions) {
- subscribedPartitions = new ArrayList<>();
-
+ private void assignTopicPartitions() {
if (restoreToOffset != null) {
- for (KafkaTopicPartition kafkaTopicPartition :
kafkaTopicPartitions) {
- if
(restoreToOffset.containsKey(kafkaTopicPartition)) {
-
subscribedPartitions.add(kafkaTopicPartition);
- }
+ subscribedPartitions = new
ArrayList<>(restoreToOffset.size());
+ for (Map.Entry<KafkaTopicPartition, Long>
restoredPartitionState : restoreToOffset.entrySet()) {
+
subscribedPartitions.add(restoredPartitionState.getKey());
--- End diff --
(note about your comment --->) subscribed partitions will always be
completely identical to the restored state, if there is any.
I should actually just change this to `subscribedPartitions = new
ArrayList<>(restoreToOffset.keySet);`, to be more clearer.
> Kafka Consumer can lose state if queried partition list is incomplete on
> restore
> --------------------------------------------------------------------------------
>
> Key: FLINK-6006
> URL: https://issues.apache.org/jira/browse/FLINK-6006
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector, Streaming Connectors
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Tzu-Li (Gordon) Tai
> Priority: Blocker
> Fix For: 1.1.5, 1.2.1
>
>
> In 1.1.x and 1.2.x, the FlinkKafkaConsumer performs partition list querying
> on restore. Then, only restored state of partitions that exists in the
> queried list is used to initialize the fetcher's state holders.
> If in any case the returned partition list is incomplete (i.e. missing
> partitions that existed before, perhaps due to temporary ZK / broker
> downtime), then the state of the missing partitions is dropped and cannot be
> recovered anymore.
> In 1.3-SNAPSHOT, this is fixed by changes in FLINK-4280, so only 1.1 and 1.2
> is affected.
> We can backport some of the behavioural changes there to 1.1 and 1.2.
> Generally, we should not depend on the current partition list in Kafka when
> restoring, but just restore all previous state into the fetcher's state
> holders.
> This would therefore also require some checking on how the consumer threads /
> Kafka clients behave when its assigned partitions cannot be reached.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)