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

    https://github.com/apache/flink/pull/3746#discussion_r116000971
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ---
    @@ -503,23 +644,30 @@ public void close() throws Exception {
        public void initializeState(FunctionInitializationContext context) 
throws Exception {
     
                OperatorStateStore stateStore = context.getOperatorStateStore();
    -           offsetsStateForCheckpoint = 
stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
     
    -           if (context.isRestored()) {
    -                   if (restoredState == null) {
    -                           restoredState = new HashMap<>();
    -                           for (Tuple2<KafkaTopicPartition, Long> 
kafkaOffset : offsetsStateForCheckpoint.get()) {
    -                                   restoredState.put(kafkaOffset.f0, 
kafkaOffset.f1);
    -                           }
    +           ListState<Tuple2<KafkaTopicPartition, Long>> 
oldRoundRobinListState =
    +                   
stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME);
     
    -                           LOG.info("Setting restore state in the 
FlinkKafkaConsumer.");
    -                           if (LOG.isDebugEnabled()) {
    -                                   LOG.debug("Using the following offsets: 
{}", restoredState);
    -                           }
    +           this.unionOffsetStates = stateStore.getUnionListState(new 
ListStateDescriptor<>(
    +                           OFFSETS_STATE_NAME,
    +                           TypeInformation.of(new 
TypeHint<Tuple2<KafkaTopicPartition, Long>>() {})));
    +
    +           if (context.isRestored() && !restoredFromOldState) {
    +                   restoredState = new TreeMap<>(new 
KafkaTopicPartition.Comparator());
    +
    +                   // migrate from 1.2 state, if there is any
    +                   for (Tuple2<KafkaTopicPartition, Long> kafkaOffset : 
oldRoundRobinListState.get()) {
    +                           restoredFromOldState = true;
    --- End diff --
    
    Could it be that we restore from an old 1.2 snapshot and don't get anything 
here because we simply weren't assigned any state. (For example because the 
parallelism is higher than before.)


---
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.
---

Reply via email to