gaoyunhaii commented on a change in pull request #14820: URL: https://github.com/apache/flink/pull/14820#discussion_r570009052
########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/CheckpointableOneInputStreamTask.java ########## @@ -0,0 +1,49 @@ +/* + * 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.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetricsBuilder; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.execution.Environment; + +/** A test stream task that also response to the checkpoint trigger requirement. */ +public class CheckpointableOneInputStreamTask<IN, OUT> extends OneInputStreamTask<IN, OUT> { Review comment: This is due to the logic for this test OneInputStream is different from the formal one in that it would trigger a snapshot (like source stream task) instead of go through the CheckpointBarrierHandler. They are mainly used in some test cases related to taking checkpoint on TM side. After some rethink I think for these test they would better to directly call `triggerCheckpointOnBarrier` to simulate the case that the alignment has finished. Thus I removed `CheckpointableOneInputStreamTask` and modifies the related tests to call `triggerCheckpointOnBarrier` directly. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -877,12 +875,6 @@ public StreamStatusMaintainer getStreamStatusMaintainer() { CompletableFuture<Boolean> result = new CompletableFuture<>(); mainMailboxExecutor.execute( () -> { - latestAsyncCheckpointStartDelayNanos = Review comment: This variable is only used for the true source tasks which do not attached with a `CheckpointBarrierHandler`, thus initially I think it could be moved to the base class for the source tasks would make it more clear. The variable mainly measures the delay between JM triggering checkpoint and (source) tasks start to processing this checkpoint (namely `checkpointStartDelayNanos`), since move it to be inside `triggerCheckpoint` would only differs in the time of one method call, thus I think it would not cause too large difference, and we could rename the variable to `latestACheckpointStartDelayNanos`. Do you think this option would also be ok? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -901,54 +893,12 @@ public StreamStatusMaintainer getStreamStatusMaintainer() { return result; } - private boolean triggerCheckpoint( + protected boolean triggerCheckpoint( Review comment: Currently not making it abstract is mainly because `MultipleInputStreamTask` has override `triggerCheckpointAsync`, if we make ` triggerCheckpoint` abstract, then it has implemented an empty method which it does not use. If we want to also make `MultipleInputStreamTask` do not need to override `triggerCheckpointAsync` directly, we have to make the completable future a parameter to `triggerCheckpoint` and let the triggerCheckpoint to complete the future instead of hide it from `triggerCheckpoint`. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -901,54 +893,12 @@ public StreamStatusMaintainer getStreamStatusMaintainer() { return result; } - private boolean triggerCheckpoint( + protected boolean triggerCheckpoint( Review comment: I also commit a diff to show the changes if we make `triggerCheckpoint` to be abstract: https://github.com/apache/flink/pull/14820/commits/83de6dda8fab862126c1e91010141828b93fe466 ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -901,54 +893,12 @@ public StreamStatusMaintainer getStreamStatusMaintainer() { return result; } - private boolean triggerCheckpoint( + protected boolean triggerCheckpoint( Review comment: I also commit a diff to show the changes if we make `triggerCheckpoint` to be abstract: https://github.com/apache/flink/pull/14820/commits/42cdc662ef4b6416b30017a5c3cb090ee66f7d07 ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
