dawidwys commented on a change in pull request #16184:
URL: https://github.com/apache/flink/pull/16184#discussion_r668785212



##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskChainedSourcesCheckpointingTest.java
##########
@@ -330,6 +338,64 @@ public void testRpcTriggerCheckpointWithSourceChain() 
throws Exception {
         }
     }
 
+    @Test
+    public void testSkipExecutionsIfFinishedOnRestoreWithSourceChained() 
throws Exception {
+        OperatorID firstSourceOperatorId = new OperatorID();
+        OperatorID secondSourceOperatorId = new OperatorID();
+        OperatorID nonSourceOperatorId = new OperatorID();
+
+        try (StreamTaskMailboxTestHarness<String> testHarness =
+                new StreamTaskMailboxTestHarnessBuilder<>(
+                                MultipleInputStreamTask::new, 
BasicTypeInfo.STRING_TYPE_INFO)
+                        .modifyStreamConfig(config -> 
config.setCheckpointingEnabled(true))
+                        
.modifyExecutionConfig(ExecutionConfig::enableObjectReuse)
+                        .addInput(BasicTypeInfo.INT_TYPE_INFO)
+                        .addSourceInput(
+                                firstSourceOperatorId,
+                                new SourceOperatorFactory<>(
+                                        new 
SourceOperatorStreamTaskTest.LifeCycleMonitorSource(
+                                                Boundedness.BOUNDED, 1),

Review comment:
       nit: as a safety precaution let's use the `CONTINUOUS_UNBOUNDED` here, 
theoretically for bounded sources we might not have checkpointing
   
   I know at this point we will, but let's do it for my peaces of mind ;)

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
##########
@@ -158,7 +158,16 @@ protected void 
processInput(MailboxDefaultAction.Controller controller) throws E
         // compatibility reasons with the current source interface (source 
functions run as a loop,
         // not in steps).
         sourceThread.setTaskDescription(getName());
-        sourceThread.start();
+
+        if (operatorChain.isFinishedOnRestore()) {
+            LOG.debug(
+                    "Legacy source {} skip execution since the task is 
finished on restore",
+                    getTaskNameWithSubtaskAndId());
+            sourceThread.getCompletionFuture().complete(null);

Review comment:
       Shouldn't we suspend the `mailboxProcessor` here? It was being done 
before when the future completed.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinishedOnRestoreSourceInput.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.streaming.runtime.tasks;
+
+import org.apache.flink.core.io.InputStatus;
+import org.apache.flink.streaming.api.operators.SourceOperator;
+import org.apache.flink.streaming.runtime.io.StreamTaskSourceInput;
+
+import java.util.concurrent.CompletableFuture;
+
+/** Guess what I'm going to do. */

Review comment:
       Let's maybe just add the answer ;)

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskChainedSourcesCheckpointingTest.java
##########
@@ -380,4 +446,57 @@ private void processSingleStepUntil(
         }
         assertTrue(condition.get());
     }
+
+    static class LifeCycleMonitorMultipleInputOperator
+            extends 
MultipleInputStreamTaskTest.MapToStringMultipleInputOperator {
+
+        private final StreamTaskTest.LifeCycleMonitor lifeCycleMonitor =
+                new StreamTaskTest.LifeCycleMonitor();

Review comment:
       nit: Let's move the `LifeCycleMonitor` to a top level file.

##########
File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskChainedSourcesCheckpointingTest.java
##########
@@ -380,4 +446,57 @@ private void processSingleStepUntil(
         }
         assertTrue(condition.get());
     }
+
+    static class LifeCycleMonitorMultipleInputOperator
+            extends 
MultipleInputStreamTaskTest.MapToStringMultipleInputOperator {
+
+        private final StreamTaskTest.LifeCycleMonitor lifeCycleMonitor =
+                new StreamTaskTest.LifeCycleMonitor();
+
+        public 
LifeCycleMonitorMultipleInputOperator(StreamOperatorParameters<String> 
parameters) {
+            super(parameters, 3);
+        }
+
+        @Override
+        public void open() throws Exception {
+            super.open();
+            
lifeCycleMonitor.incrementCallTime(StreamTaskTest.LifeCyclePhase.OPEN);
+        }
+
+        @Override
+        public void initializeState(StateInitializationContext context) throws 
Exception {
+            super.initializeState(context);
+            
lifeCycleMonitor.incrementCallTime(StreamTaskTest.LifeCyclePhase.INITIALIZE_STATE);
+        }
+
+        @Override
+        public void finish() throws Exception {
+            
lifeCycleMonitor.incrementCallTime(StreamTaskTest.LifeCyclePhase.FINISH);

Review comment:
       nit: super.finish();




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