gaborgsomogyi commented on code in PR #26931:
URL: https://github.com/apache/flink/pull/26931#discussion_r2291438290


##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateAssignment.java:
##########
@@ -131,12 +141,63 @@ public TaskStateAssignment(
         outputOperatorID = operatorIDs.get(0).getGeneratedOperatorID();
         inputOperatorID = operatorIDs.get(operatorIDs.size() - 
1).getGeneratedOperatorID();
 
-        hasInputState =
-                oldState.get(inputOperatorID).getStates().stream()
-                        .anyMatch(subState -> 
!subState.getInputChannelState().isEmpty());
-        hasOutputState =
-                oldState.get(outputOperatorID).getStates().stream()
-                        .anyMatch(subState -> 
!subState.getResultSubpartitionState().isEmpty());
+        inputStateGates = 
extractInputStateGates(oldState.get(inputOperatorID));
+        outputStatePartitions = 
extractOutputStatePartitions(oldState.get(outputOperatorID));
+    }
+
+    private static Set<Integer> extractInputStateGates(OperatorState 
operatorState) {
+        return operatorState.getStates().stream()
+                .map(OperatorSubtaskState::getInputChannelState)
+                .flatMap(Collection::stream)
+                .flatMapToInt(
+                        handle -> {
+                            if (handle instanceof InputChannelStateHandle) {
+                                return IntStream.of(
+                                        ((InputChannelStateHandle) 
handle).getInfo().getGateIdx());
+                            } else if (handle instanceof 
MergedInputChannelStateHandle) {
+                                return ((MergedInputChannelStateHandle) handle)
+                                        
.getInfos().stream().mapToInt(InputChannelInfo::getGateIdx);
+                            } else {
+                                throw new IllegalStateException(
+                                        "Invalid input channel state : " + 
handle.getClass());
+                            }
+                        })
+                .distinct()
+                .boxed()
+                .collect(Collectors.toSet());
+    }
+
+    private static Set<Integer> extractOutputStatePartitions(OperatorState 
operatorState) {
+        return operatorState.getStates().stream()
+                .map(OperatorSubtaskState::getResultSubpartitionState)
+                .flatMap(Collection::stream)
+                .flatMapToInt(
+                        handle -> {
+                            if (handle instanceof 
ResultSubpartitionStateHandle) {
+                                return IntStream.of(
+                                        ((ResultSubpartitionStateHandle) 
handle)
+                                                .getInfo()
+                                                .getPartitionIdx());
+                            } else if (handle instanceof 
MergedResultSubpartitionStateHandle) {
+                                return ((MergedResultSubpartitionStateHandle) 
handle)
+                                        .getInfos().stream()
+                                                
.mapToInt(ResultSubpartitionInfo::getPartitionIdx);
+                            } else {
+                                throw new IllegalStateException(

Review Comment:
   There is no uncovered implementation so can't be tested now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to