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


##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleWithMixedExchangesITCase.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.checkpointing;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.Types;
+import 
org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExternalizedCheckpointRetention;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateRecoveryOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.connector.datagen.source.DataGeneratorSource;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Integration test for rescaling jobs with mixed (UC-supported and 
UC-unsupported) exchanges from
+ * an unaligned checkpoint.
+ */
+@RunWith(Parameterized.class)
+public class UnalignedCheckpointRescaleWithMixedExchangesITCase extends 
TestLogger {
+
+    private static final int NUM_TASK_MANAGERS = 1;
+    private static final int SLOTS_PER_TASK_MANAGER = 10;
+    private static final int MAX_SLOTS = NUM_TASK_MANAGERS * 
SLOTS_PER_TASK_MANAGER;
+    private static final Random RANDOM = new Random();
+
+    private static MiniClusterWithClientResource cluster;
+    @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+    @Parameterized.Parameter public ExecuteJobViaEnv executeJobViaEnv;
+
+    @Parameterized.Parameters(name = "Test case {index}")
+    public static Collection<ExecuteJobViaEnv> parameter() {
+        return List.of(
+                
UnalignedCheckpointRescaleWithMixedExchangesITCase::createMultiOutputDAG,
+                
UnalignedCheckpointRescaleWithMixedExchangesITCase::createMultiInputDAG,
+                
UnalignedCheckpointRescaleWithMixedExchangesITCase::createRescalePartitionerDAG,
+                
UnalignedCheckpointRescaleWithMixedExchangesITCase::createMixedComplexityDAG);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        cluster =
+                new MiniClusterWithClientResource(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setConfiguration(new Configuration())
+                                .setNumberTaskManagers(NUM_TASK_MANAGERS)
+                                
.setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER)
+                                .build());
+        cluster.before();
+    }
+
+    @After
+    public void shutDownExistingCluster() {
+        if (cluster != null) {
+            cluster.after();
+            cluster = null;
+        }
+    }
+
+    /**
+     * Tests rescaling from an unaligned checkpoint with different job 
structures that have mixed
+     * (UC-supported and UC-unsupported) exchanges.
+     */
+    @Test
+    public void testRescaleFromUnalignedCheckpoint() throws Exception {
+        final MiniCluster miniCluster = cluster.getMiniCluster();
+
+        // Step 1: Run the job with initial parallelism and take a checkpoint
+        JobClient jobClient1 = 
executeJobViaEnv.executeJob(getUnalignedCheckpointEnv(null));
+
+        CommonTestUtils.waitForJobStatus(jobClient1, 
Collections.singletonList(JobStatus.RUNNING));
+        CommonTestUtils.waitForAllTaskRunning(miniCluster, 
jobClient1.getJobID(), false);
+        String checkpointPath =
+                CommonTestUtils.waitForCheckpointWithInflightBuffers(
+                        jobClient1.getJobID(), miniCluster);
+        jobClient1.cancel().get();
+
+        // Step 2: Restore the job with a different parallelism
+        JobClient jobClient2 =
+                
executeJobViaEnv.executeJob(getUnalignedCheckpointEnv(checkpointPath));
+
+        CommonTestUtils.waitForJobStatus(jobClient2, 
Collections.singletonList(JobStatus.RUNNING));
+        CommonTestUtils.waitForAllTaskRunning(miniCluster, 
jobClient2.getJobID(), false);
+        
CommonTestUtils.waitForCheckpointWithInflightBuffers(jobClient2.getJobID(), 
miniCluster);
+        jobClient2.cancel().get();
+    }
+
+    private StreamExecutionEnvironment getUnalignedCheckpointEnv(@Nullable 
String recoveryPath)
+            throws IOException {
+        Configuration conf = new Configuration();
+        conf.set(CheckpointingOptions.CHECKPOINTING_INTERVAL, 
Duration.ofSeconds(1));
+        conf.set(
+                CheckpointingOptions.EXTERNALIZED_CHECKPOINT_RETENTION,
+                ExternalizedCheckpointRetention.RETAIN_ON_CANCELLATION);
+        conf.set(
+                CheckpointingOptions.CHECKPOINTS_DIRECTORY,
+                temporaryFolder.newFolder().toURI().toString());
+        conf.set(CheckpointingOptions.ENABLE_UNALIGNED, true);
+        // Decrease the memory segment size to avoid the test is so slow for 
some reasons:
+        // 1. Recovery phase needs to consume all inflight buffers
+        // 2. Forward or rescale exchange does not support unaligned 
checkpoint.
+        conf.set(TaskManagerOptions.MEMORY_SEGMENT_SIZE, MemorySize.parse("1 
kb"));

Review Comment:
   Not yet understand these comments, could you elaborate on this? On my side 
if I remove it then it makes tests just flaky.



##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleWithMixedExchangesITCase.java:
##########
@@ -0,0 +1,352 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.checkpointing;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.Types;
+import 
org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExternalizedCheckpointRetention;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.StateRecoveryOptions;
+import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.connector.datagen.source.DataGeneratorSource;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoMapFunction;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Integration test for rescaling jobs with mixed (UC-supported and 
UC-unsupported) exchanges from
+ * an unaligned checkpoint.
+ */
+@RunWith(Parameterized.class)
+public class UnalignedCheckpointRescaleWithMixedExchangesITCase extends 
TestLogger {

Review Comment:
   I've gone through the existing and these new UC tests which are providing 
excellent coverage. What I've not yet found is cases which skip rescaling when 
an edge has no in-flight/channel state.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateAssignment.java:
##########
@@ -418,6 +495,50 @@ public SubtasksRescaleMapping getInputMapping(int 
gateIndex) {
                         checkSubtaskMapping(oldMapping, mapping, 
mapper.isAmbiguous()));
     }
 
+    public boolean hasInFlightDataForInputGate(int gateIndex) {
+        // Check own input state for this gate
+        if (inputStateGates.contains(gateIndex)) {
+            return true;
+        }
+
+        // Check upstream output state for this gate
+        TaskStateAssignment upstreamAssignment = 
getUpstreamAssignments()[gateIndex];
+        if (upstreamAssignment != null && upstreamAssignment.hasOutputState()) 
{
+            IntermediateResult inputResult = 
executionJobVertex.getInputs().get(gateIndex);
+            int partitionIndex =
+                    
Arrays.asList(inputResult.getProducer().getProducedDataSets())
+                            .indexOf(inputResult);
+
+            if (partitionIndex != -1) {
+                return 
upstreamAssignment.outputStatePartitions.contains(partitionIndex);
+            }
+        }
+
+        return false;
+    }
+
+    public boolean hasInFlightDataForResultPartition(int partitionIndex) {
+        // Check own output state for this partition
+        if (outputStatePartitions.contains(partitionIndex)) {
+            return true;
+        }
+
+        // Check downstream input state for this partition
+        TaskStateAssignment downstreamAssignment = 
getDownstreamAssignments()[partitionIndex];
+
+        if (downstreamAssignment != null && 
downstreamAssignment.hasInputState()) {
+            IntermediateResult producedResult =
+                    executionJobVertex.getProducedDataSets()[partitionIndex];
+            int gateIndex =
+                    
downstreamAssignment.executionJobVertex.getInputs().indexOf(producedResult);

Review Comment:
   Same here.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/InflightDataRescalingDescriptor.java:
##########
@@ -112,6 +112,28 @@ public String toString() {
      */
     public static class InflightDataGateOrPartitionRescalingDescriptor 
implements Serializable {
 
+        public static final InflightDataGateOrPartitionRescalingDescriptor 
NO_STATE =
+                new InflightDataGateOrPartitionRescalingDescriptor(
+                        new int[0],
+                        RescaleMappings.identity(0, 0),
+                        java.util.Collections.emptySet(),

Review Comment:
   Nit: Maybe it can be imported.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateAssignment.java:
##########
@@ -418,6 +495,50 @@ public SubtasksRescaleMapping getInputMapping(int 
gateIndex) {
                         checkSubtaskMapping(oldMapping, mapping, 
mapper.isAmbiguous()));
     }
 
+    public boolean hasInFlightDataForInputGate(int gateIndex) {
+        // Check own input state for this gate
+        if (inputStateGates.contains(gateIndex)) {
+            return true;
+        }
+
+        // Check upstream output state for this gate
+        TaskStateAssignment upstreamAssignment = 
getUpstreamAssignments()[gateIndex];
+        if (upstreamAssignment != null && upstreamAssignment.hasOutputState()) 
{
+            IntermediateResult inputResult = 
executionJobVertex.getInputs().get(gateIndex);
+            int partitionIndex =
+                    
Arrays.asList(inputResult.getProducer().getProducedDataSets())
+                            .indexOf(inputResult);

Review Comment:
   Are we sure that this is working all the time because `IntermediateResult` 
has no valid `equals` implementation and just uses `==` on the references?



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to