pnowojski commented on a change in pull request #11899: URL: https://github.com/apache/flink/pull/11899#discussion_r416653169
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRequestDecider.java ########## @@ -0,0 +1,156 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator.CheckpointTriggerRequest; +import org.apache.flink.runtime.util.clock.Clock; + +import java.util.Optional; +import java.util.PriorityQueue; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.apache.flink.runtime.checkpoint.CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS; +import static org.apache.flink.runtime.checkpoint.CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS; + +@SuppressWarnings("ConstantConditions") +class CheckpointRequestDecider { + private static final int MAX_QUEUED_REQUESTS = 1000; + + private final int maxConcurrentCheckpointAttempts; + private final Consumer<Long> rescheduleTrigger; + private final Clock clock; + private final long minPauseBetweenCheckpoints; + private final Supplier<Integer> pendingCheckpointsSizeSupplier; + private final Function<Supplier<Optional<CheckpointTriggerRequest>>, Optional<CheckpointTriggerRequest>> sync; Review comment: This should be defined as some concrete type, I'm struggling to understand what's happening here and it's so long, that even you mis-formatted it in the constructor :) ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRequestDecider.java ########## @@ -0,0 +1,156 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator.CheckpointTriggerRequest; +import org.apache.flink.runtime.util.clock.Clock; + +import java.util.Optional; +import java.util.PriorityQueue; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.apache.flink.runtime.checkpoint.CheckpointFailureReason.MINIMUM_TIME_BETWEEN_CHECKPOINTS; +import static org.apache.flink.runtime.checkpoint.CheckpointFailureReason.TOO_MANY_CONCURRENT_CHECKPOINTS; + +@SuppressWarnings("ConstantConditions") +class CheckpointRequestDecider { + private static final int MAX_QUEUED_REQUESTS = 1000; + + private final int maxConcurrentCheckpointAttempts; + private final Consumer<Long> rescheduleTrigger; + private final Clock clock; + private final long minPauseBetweenCheckpoints; + private final Supplier<Integer> pendingCheckpointsSizeSupplier; + private final Function<Supplier<Optional<CheckpointTriggerRequest>>, Optional<CheckpointTriggerRequest>> sync; + private final PriorityQueue<CheckpointTriggerRequest> triggerRequestQueue = new PriorityQueue<>((r1, r2) -> { + if (r1.props.isSavepoint() != r2.props.isSavepoint()) { + return r1.props.isSavepoint() ? -1 : 1; + } else if (r1.props.forceCheckpoint() != r2.props.forceCheckpoint()) { + return r1.props.forceCheckpoint() ? -1 : 1; + } else if (r1.isPeriodic != r2.isPeriodic) { + return r1.isPeriodic ? 1 : -1; + } else { + return Long.compare(r1.timestamp, r2.timestamp); + } + }); Review comment: Could extract this comparator to a named static method and also `r1/2` -> `request1/2` (I had to go into the `PriorityQueue` constructor to understand what's happening here and to remember that this is a comparator). Also a unit test for that would be a nice to have. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java ########## @@ -248,6 +248,15 @@ public String toString() { false, false); + private static final CheckpointProperties SYNC_SAVEPOINT_NOT_FORCED = new CheckpointProperties( + false, Review comment: Have you thought about implications of setting this flag to `false` in other places? like `PendingCheckpoint#canBeSubsumed()`? or `PendingCheckpoint#assertAbortSubsumedForced`? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java ########## @@ -969,15 +969,17 @@ private void configureCheckpointing() { triggerVertices, ackVertices, commitVertices, - new CheckpointCoordinatorConfiguration( - interval, - cfg.getCheckpointTimeout(), - cfg.getMinPauseBetweenCheckpoints(), - cfg.getMaxConcurrentCheckpoints(), - retentionAfterTermination, - getCheckpointingMode(cfg) == CheckpointingMode.EXACTLY_ONCE, - cfg.isPreferCheckpointForRecovery(), - cfg.getTolerableCheckpointFailureNumber()), + CheckpointCoordinatorConfiguration.builder() + .setCheckpointInterval(interval) + .setCheckpointTimeout(cfg.getCheckpointTimeout()) + .setMinPauseBetweenCheckpoints(cfg.getMinPauseBetweenCheckpoints()) + .setMaxConcurrentCheckpoints(cfg.getMaxConcurrentCheckpoints()) + .setCheckpointRetentionPolicy(retentionAfterTermination) + .setExactlyOnce(getCheckpointingMode(cfg) == CheckpointingMode.EXACTLY_ONCE) + .setPreferCheckpointForRecovery(cfg.isPreferCheckpointForRecovery()) + .setTolerableCheckpointFailureNumber(cfg.getTolerableCheckpointFailureNumber()) + .setUnalignedCheckpointsEnabled(cfg.isUnalignedCheckpointsEnabled()) Review comment: What if user took an unaligned checkpoint, shutdown the job, disabled unaligned checkpoints, start recovering from it and triggered savepoint? Maybe we don't need to handle this in this PR, but if it's really a problem we would need to follow up on this. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ########## @@ -497,44 +504,17 @@ public boolean isShutdown() { "Only synchronous savepoints are allowed to advance the watermark to MAX.")); } - final CompletableFuture<CompletedCheckpoint> onCompletionPromise = - new CompletableFuture<>(); - synchronized (lock) { - if (isTriggering || !triggerRequestQueue.isEmpty()) { - // we can't trigger checkpoint directly if there is a trigger request being processed - // or queued - triggerRequestQueue.add(new CheckpointTriggerRequest( - timestamp, - props, - externalSavepointLocation, - isPeriodic, - advanceToEndOfTime, - onCompletionPromise)); - return onCompletionPromise; - } - } - startTriggeringCheckpoint( - timestamp, - props, - externalSavepointLocation, - isPeriodic, - advanceToEndOfTime, - onCompletionPromise); - return onCompletionPromise; + final CheckpointTriggerRequest request = new CheckpointTriggerRequest(timestamp, props, externalSavepointLocation, isPeriodic, advanceToEndOfTime); + requestDecider + .chooseRequestToExecute(request, isTriggering, lastCheckpointCompletionRelativeTime) + .ifPresent(this::startTriggeringCheckpoint); + return request.onCompletionPromise; } - private void startTriggeringCheckpoint( - long timestamp, - CheckpointProperties props, - @Nullable String externalSavepointLocation, - boolean isPeriodic, - boolean advanceToEndOfTime, - CompletableFuture<CompletedCheckpoint> onCompletionPromise) { - + private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { Review comment: You could have extracted quite a bit of refactoring into an independent commit. Like using `CheckpointTriggerRequest` as a parameter here and moving `CompletableFuture<CompletedCheckpoint> onCompletionPromise` into the request (it would significantly simplify the review of this change) ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java ########## @@ -503,44 +507,17 @@ public boolean isShutdown() { "Only synchronous savepoints are allowed to advance the watermark to MAX.")); } - final CompletableFuture<CompletedCheckpoint> onCompletionPromise = - new CompletableFuture<>(); - synchronized (lock) { - if (isTriggering || !triggerRequestQueue.isEmpty()) { Review comment: How was it working before? If savepoint was triggered while a checkpoint was enqueued/triggering, it should have triggered this branch and savepoint should enqueue after the checkpoint? Or am I missing something? ---------------------------------------------------------------- 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