Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -476,12 +476,17 @@ private CompletableFuture<CompletedCheckpoint> triggerSavepointInternal(

checkNotNull(checkpointProperties);

return triggerCheckpointFromCheckpointThread(checkpointProperties, targetLocation, false);
}

private CompletableFuture<CompletedCheckpoint> triggerCheckpointFromCheckpointThread(
CheckpointProperties checkpointProperties, String targetLocation, boolean isPeriodic) {
// TODO, call triggerCheckpoint directly after removing timer thread
// for now, execute the trigger in timer thread to avoid competition
final CompletableFuture<CompletedCheckpoint> resultFuture = new CompletableFuture<>();
timer.execute(
() ->
triggerCheckpoint(checkpointProperties, targetLocation, false)
triggerCheckpoint(checkpointProperties, targetLocation, isPeriodic)
.whenComplete(
(completedCheckpoint, throwable) -> {
if (throwable == null) {
Expand All @@ -498,16 +503,15 @@ private CompletableFuture<CompletedCheckpoint> triggerSavepointInternal(
* The return value is a future. It completes when the checkpoint triggered finishes or an error
* occurred.
*
* @param isPeriodic Flag indicating whether this triggered checkpoint is periodic. If this flag
* is true, but the periodic scheduler is disabled, the checkpoint will be declined.
* @param isPeriodic Flag indicating whether this triggered checkpoint is periodic.
* @return a future to the completed checkpoint.
*/
public CompletableFuture<CompletedCheckpoint> triggerCheckpoint(boolean isPeriodic) {
return triggerCheckpoint(checkpointProperties, null, isPeriodic);
return triggerCheckpointFromCheckpointThread(checkpointProperties, null, isPeriodic);
}

@VisibleForTesting
public CompletableFuture<CompletedCheckpoint> triggerCheckpoint(
CompletableFuture<CompletedCheckpoint> triggerCheckpoint(
CheckpointProperties props,
@Nullable String externalSavepointLocation,
boolean isPeriodic) {
Expand Down Expand Up @@ -976,8 +980,10 @@ private Optional<CheckpointTriggerRequest> chooseQueuedRequestToExecute() {
private Optional<CheckpointTriggerRequest> chooseRequestToExecute(
CheckpointTriggerRequest request) {
synchronized (lock) {
return requestDecider.chooseRequestToExecute(
request, isTriggering, lastCheckpointCompletionRelativeTime);
Optional<CheckpointTriggerRequest> checkpointTriggerRequest =
requestDecider.chooseRequestToExecute(
request, isTriggering, lastCheckpointCompletionRelativeTime);
return checkpointTriggerRequest;
}
}

Expand Down Expand Up @@ -2026,7 +2032,7 @@ private final class ScheduledTrigger implements Runnable {
@Override
public void run() {
try {
triggerCheckpoint(true);
triggerCheckpoint(checkpointProperties, null, true);
} catch (Exception e) {
LOG.error("Exception while triggering checkpoint for job {}.", job, e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3220,6 +3220,7 @@ public void testSavepointScheduledInUnalignedMode() throws Exception {
checkpointFutures.add(coordinator.triggerCheckpoint(true));
activeRequests++;
}
manuallyTriggeredScheduledExecutor.triggerAll();
assertEquals(
activeRequests - maxConcurrentCheckpoints, coordinator.getNumQueuedRequests());

Expand Down Expand Up @@ -3730,7 +3731,9 @@ public void testAbortingBeforeTriggeringCheckpointOperatorCoordinator() throws E
.build(graph);
try {
checkpointCoordinator.triggerCheckpoint(false);
// trigger twice to get checkpoint id and create pending checkpoint
// trigger three times to trigger checkpoint, to get checkpoint id and create pending
// checkpoint
manuallyTriggeredScheduledExecutor.trigger();
manuallyTriggeredScheduledExecutor.trigger();
manuallyTriggeredScheduledExecutor.trigger();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder;
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
import org.apache.flink.runtime.executiongraph.ExecutionVertex;
Expand All @@ -32,6 +33,7 @@
import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration;
import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration.CheckpointCoordinatorConfigurationBuilder;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.testutils.DirectScheduledExecutorService;
import org.apache.flink.testutils.TestingUtils;
import org.apache.flink.testutils.executor.TestExecutorResource;
import org.apache.flink.util.ExceptionUtils;
Expand Down Expand Up @@ -716,7 +718,18 @@ public void discardingTriggeringCheckpointWillExecuteNextCheckpointRequest() thr
.setTimer(new ScheduledExecutorServiceAdapter(scheduledExecutorService))
.setCheckpointCoordinatorConfiguration(
CheckpointCoordinatorConfiguration.builder().build())
.build(EXECUTOR_RESOURCE.getExecutor());
// Since timer thread != main thread we should override the default main
// thread executor because it initially requires triggering a checkpoint
// from the main test thread.
.build(
new CheckpointCoordinatorTestingUtils
.CheckpointExecutionGraphBuilder()
.addJobVertex(new JobVertexID())
.setMainThreadExecutor(
ComponentMainThreadExecutorServiceAdapter
.forSingleThreadExecutor(
new DirectScheduledExecutorService()))
.build(EXECUTOR_RESOURCE.getExecutor()));

final CompletableFuture<String> masterHookCheckpointFuture = new CompletableFuture<>();
final OneShotLatch triggerCheckpointLatch = new OneShotLatch();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -788,7 +788,11 @@ public void abortPendingCheckpointsWhenRestartingTasks() throws Exception {

final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch();

final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
final DefaultScheduler scheduler =
createSchedulerAndStartScheduling(
jobGraph,
ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(
new DirectScheduledExecutorService()));

final ArchivedExecutionVertex onlyExecutionVertex =
Iterables.getOnlyElement(
Expand Down Expand Up @@ -818,7 +822,11 @@ public void restoreStateWhenRestartingTasks() throws Exception {

final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch();

final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
final DefaultScheduler scheduler =
createSchedulerAndStartScheduling(
jobGraph,
ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(
new DirectScheduledExecutorService()));

final ArchivedExecutionVertex onlyExecutionVertex =
Iterables.getOnlyElement(
Expand Down Expand Up @@ -856,7 +864,11 @@ public void failGlobalWhenRestoringStateFails() throws Exception {

final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch();

final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph);
final DefaultScheduler scheduler =
createSchedulerAndStartScheduling(
jobGraph,
ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(
new DirectScheduledExecutorService()));

final ArchivedExecutionVertex onlyExecutionVertex =
Iterables.getOnlyElement(
Expand Down