aljoscha commented on a change in pull request #14820: URL: https://github.com/apache/flink/pull/14820#discussion_r569539155
########## File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestCheckpointBarrierHandler.java ########## @@ -0,0 +1,84 @@ +/* + * 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.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.streaming.runtime.io.checkpointing.CheckpointBarrierHandler; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * A test {@link TestCheckpointBarrierHandler} that records the history of checkpoint triggering. Review comment: ```suggestion * A {@link CheckpointBarrierHandler} for testing that records the history of checkpoint triggering. ``` ########## 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: Why is this not turned into an `abstract` method? ########## 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: Why is this moved to the "inner" method? I found it a bit confusing that the "non-async" method not determines the start of the async part. ########## 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: Why don't the tests work anymore with just the regular `OneInputStreamTask`? They also have an implemented `triggerCheckpoint()`. ---------------------------------------------------------------- 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: us...@infra.apache.org