diff --git a/docs/layouts/shortcodes/generated/all_jobmanager_section.html b/docs/layouts/shortcodes/generated/all_jobmanager_section.html index 87d8875a6df2f..327404af884ab 100644 --- a/docs/layouts/shortcodes/generated/all_jobmanager_section.html +++ b/docs/layouts/shortcodes/generated/all_jobmanager_section.html @@ -50,6 +50,12 @@ Boolean This parameter defines whether the adaptive scheduler prioritizes using the minimum number of TaskManagers when scheduling tasks.
Note, this parameter is suitable if execution.state-recovery.from-local is not enabled. More details about this configuration are available at FLINK-33977. + +
jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled
+ true + Boolean + When enabled, the Adaptive Scheduler actively triggers a checkpoint when resources change and rescaling is desired, rather than waiting for the next periodic checkpoint. This reduces rescaling latency, especially when checkpoint intervals are large. The active trigger respects execution.checkpointing.min-pause and will not fire if a checkpoint is already in progress or being triggered. +
jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures
2 diff --git a/docs/layouts/shortcodes/generated/expert_scheduling_section.html b/docs/layouts/shortcodes/generated/expert_scheduling_section.html index 524ef524d1597..de91f5d4bf7c4 100644 --- a/docs/layouts/shortcodes/generated/expert_scheduling_section.html +++ b/docs/layouts/shortcodes/generated/expert_scheduling_section.html @@ -104,6 +104,12 @@ Boolean This parameter defines whether the adaptive scheduler prioritizes using the minimum number of TaskManagers when scheduling tasks.
Note, this parameter is suitable if execution.state-recovery.from-local is not enabled. More details about this configuration are available at FLINK-33977. + +
jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled
+ true + Boolean + When enabled, the Adaptive Scheduler actively triggers a checkpoint when resources change and rescaling is desired, rather than waiting for the next periodic checkpoint. This reduces rescaling latency, especially when checkpoint intervals are large. The active trigger respects execution.checkpointing.min-pause and will not fire if a checkpoint is already in progress or being triggered. +
jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures
2 diff --git a/docs/layouts/shortcodes/generated/job_manager_configuration.html b/docs/layouts/shortcodes/generated/job_manager_configuration.html index d197b268780c9..d1296c607f24d 100644 --- a/docs/layouts/shortcodes/generated/job_manager_configuration.html +++ b/docs/layouts/shortcodes/generated/job_manager_configuration.html @@ -50,6 +50,12 @@ Boolean This parameter defines whether the adaptive scheduler prioritizes using the minimum number of TaskManagers when scheduling tasks.
Note, this parameter is suitable if execution.state-recovery.from-local is not enabled. More details about this configuration are available at FLINK-33977. + +
jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled
+ true + Boolean + When enabled, the Adaptive Scheduler actively triggers a checkpoint when resources change and rescaling is desired, rather than waiting for the next periodic checkpoint. This reduces rescaling latency, especially when checkpoint intervals are large. The active trigger respects execution.checkpointing.min-pause and will not fire if a checkpoint is already in progress or being triggered. +
jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures
2 diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 065de7e63d320..c3e35125dfac9 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -745,6 +745,24 @@ public InlineElement getDescription() { .key())) .build()); + @Documentation.Section({ + Documentation.Sections.EXPERT_SCHEDULING, + Documentation.Sections.ALL_JOB_MANAGER + }) + public static final ConfigOption SCHEDULER_RESCALE_TRIGGER_ACTIVE_CHECKPOINT_ENABLED = + key("jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled") + .booleanType() + .defaultValue(true) + .withDescription( + Description.builder() + .text( + "When enabled, the Adaptive Scheduler actively triggers a checkpoint when resources change and rescaling is desired, " + + "rather than waiting for the next periodic checkpoint. " + + "This reduces rescaling latency, especially when checkpoint intervals are large. " + + "The active trigger respects execution.checkpointing.min-pause and " + + "will not fire if a checkpoint is already in progress or being triggered.") + .build()); + /** * @deprecated Use {@link * JobManagerOptions#SCHEDULER_SUBMISSION_RESOURCE_STABILIZATION_TIMEOUT}. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index bff3baa54c0e1..6862c7cd70a6e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1996,6 +1996,16 @@ public long getCheckpointTimeout() { return checkpointTimeout; } + /** + * Returns {@code true} if enough time has elapsed since the last checkpoint completion to + * satisfy the configured {@code minPauseBetweenCheckpoints}. This can be used by callers that + * trigger non-periodic checkpoints but still wish to respect the min-pause constraint. + */ + public boolean isMinPauseBetweenCheckpointsSatisfied() { + return clock.relativeTimeMillis() - lastCheckpointCompletionRelativeTime + >= minPauseBetweenCheckpoints; + } + /** * @deprecated use {@link #getNumQueuedRequests()} */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index 70c5b62ce90ad..b75f0d09f29e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -314,7 +314,9 @@ public static Settings of( SCHEDULER_RESCALE_TRIGGER_MAX_DELAY, maximumDelayForRescaleTriggerDefault), rescaleOnFailedCheckpointsCount, - configuration.get(WebOptions.MAX_ADAPTIVE_SCHEDULER_RESCALE_HISTORY_SIZE)); + configuration.get(WebOptions.MAX_ADAPTIVE_SCHEDULER_RESCALE_HISTORY_SIZE), + configuration.get( + JobManagerOptions.SCHEDULER_RESCALE_TRIGGER_ACTIVE_CHECKPOINT_ENABLED)); } private final SchedulerExecutionMode executionMode; @@ -326,6 +328,7 @@ public static Settings of( private final Duration maximumDelayForTriggeringRescale; private final int rescaleOnFailedCheckpointCount; private final int rescaleHistoryMax; + private final boolean activeCheckpointTriggerEnabled; private Settings( SchedulerExecutionMode executionMode, @@ -336,7 +339,8 @@ private Settings( Duration executingResourceStabilizationTimeout, Duration maximumDelayForTriggeringRescale, int rescaleOnFailedCheckpointCount, - int rescaleHistoryMax) { + int rescaleHistoryMax, + boolean activeCheckpointTriggerEnabled) { this.executionMode = executionMode; this.submissionResourceWaitTimeout = submissionResourceWaitTimeout; this.submissionResourceStabilizationTimeout = submissionResourceStabilizationTimeout; @@ -346,6 +350,7 @@ private Settings( this.maximumDelayForTriggeringRescale = maximumDelayForTriggeringRescale; this.rescaleOnFailedCheckpointCount = rescaleOnFailedCheckpointCount; this.rescaleHistoryMax = rescaleHistoryMax; + this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled; } public SchedulerExecutionMode getExecutionMode() { @@ -384,6 +389,10 @@ public int getRescaleHistoryMax() { return rescaleHistoryMax; } + public boolean isActiveCheckpointTriggerEnabled() { + return activeCheckpointTriggerEnabled; + } + public JobRescaleConfigInfo toJobRescaleConfigInfo() { return new JobRescaleConfigInfo( rescaleHistoryMax, @@ -1311,7 +1320,8 @@ public void goToExecuting( userCodeClassLoader, failureCollection, this::createExecutingStateTransitionManager, - settings.getRescaleOnFailedCheckpointCount())); + settings.getRescaleOnFailedCheckpointCount(), + settings.isActiveCheckpointTriggerEnabled())); } private StateTransitionManager createExecutingStateTransitionManager( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManager.java index 87f810ae784d3..f44f5e14c0372 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManager.java @@ -164,6 +164,10 @@ private void progressToStabilized(Temporal firstChangeEventTimestamp) { progressToPhase(new Stabilized(clock, this, firstChangeEventTimestamp, maxTriggerDelay)); } + void requestActiveCheckpointTrigger() { + transitionContext.requestActiveCheckpointTrigger(); + } + private void triggerTransitionToSubsequentState() { progressToPhase(new Transitioning(clock, this)); transitionContext.transitionToSubsequentState(); @@ -362,6 +366,7 @@ private Stabilizing( resourceStabilizationTimeout); scheduleTransitionEvaluation(); + context().requestActiveCheckpointTrigger(); } @Override @@ -370,6 +375,7 @@ void onChange(boolean newResourceDriven) { // event was already handled by a onTrigger callback with a no-op onChangeEventTimestamp = now(); scheduleTransitionEvaluation(); + context().requestActiveCheckpointTrigger(); } @Override @@ -427,6 +433,7 @@ private Stabilized( }, firstChangeEventTimestamp, maxTriggerDelay); + context().requestActiveCheckpointTrigger(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java index deae52856dc41..b6466db38ac89 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/Executing.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.checkpoint.CheckpointScheduling; import org.apache.flink.runtime.checkpoint.CheckpointStatsListener; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; @@ -40,6 +41,7 @@ import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointTerminationManager; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; import org.slf4j.Logger; @@ -64,6 +66,7 @@ class Executing extends StateWithExecutionGraph private final StateTransitionManager stateTransitionManager; private final int rescaleOnFailedCheckpointCount; + private final boolean activeCheckpointTriggerEnabled; // null indicates that there was no change event observed, yet @Nullable private AtomicInteger failedCheckpointCountdown; @@ -77,7 +80,8 @@ class Executing extends StateWithExecutionGraph List failureCollection, Function stateTransitionManagerFactory, - int rescaleOnFailedCheckpointCount) { + int rescaleOnFailedCheckpointCount, + boolean activeCheckpointTriggerEnabled) { super( context, executionGraph, @@ -96,6 +100,7 @@ class Executing extends StateWithExecutionGraph rescaleOnFailedCheckpointCount > 0, "The rescaleOnFailedCheckpointCount should be larger than 0."); this.rescaleOnFailedCheckpointCount = rescaleOnFailedCheckpointCount; + this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled; this.failedCheckpointCountdown = null; recordRescaleForJobIntoExecuting(logger, context); @@ -182,6 +187,74 @@ public ScheduledFuture scheduleOperation(Runnable callback, Duration delay) { return context.runIfState(this, callback, delay); } + private static final Duration ACTIVE_CHECKPOINT_RETRY_DELAY = Duration.ofSeconds(10); + + @Override + public void requestActiveCheckpointTrigger() { + if (!activeCheckpointTriggerEnabled) { + return; + } + + final CheckpointCoordinator checkpointCoordinator = + getExecutionGraph().getCheckpointCoordinator(); + + if (checkpointCoordinator == null + || !checkpointCoordinator.isPeriodicCheckpointingConfigured()) { + getLogger() + .debug( + "Skipping active checkpoint trigger for rescale: checkpointing not configured."); + return; + } + + if (!parallelismChanged()) { + getLogger() + .debug( + "Skipping active checkpoint trigger for rescale: parallelism unchanged."); + return; + } + + if (checkpointCoordinator.getNumberOfPendingCheckpoints() > 0 + || checkpointCoordinator.isTriggering()) { + getLogger() + .debug( + "Skipping active checkpoint trigger for rescale: checkpoint already in progress."); + return; + } + + if (!checkpointCoordinator.isMinPauseBetweenCheckpointsSatisfied()) { + getLogger() + .debug( + "Skipping active checkpoint trigger for rescale: min pause between checkpoints not satisfied, scheduling retry."); + context.runIfState( + this, this::requestActiveCheckpointTrigger, ACTIVE_CHECKPOINT_RETRY_DELAY); + return; + } + + getLogger().info("Actively triggering checkpoint to expedite rescaling."); + FutureUtils.assertNoException( + checkpointCoordinator + .triggerCheckpoint(false) + .handle( + (completedCheckpoint, throwable) -> { + if (throwable != null) { + getLogger() + .warn( + "Active checkpoint trigger for rescale failed, scheduling retry.", + throwable); + context.runIfState( + this, + this::requestActiveCheckpointTrigger, + ACTIVE_CHECKPOINT_RETRY_DELAY); + } else { + getLogger() + .info( + "Active checkpoint for rescale completed successfully: {}.", + completedCheckpoint.getCheckpointID()); + } + return null; + })); + } + @Override public void transitionToSubsequentState() { Optional availableVertexParallelism = @@ -399,6 +472,7 @@ static class Factory implements StateFactory { private final Function stateTransitionManagerFactory; private final int rescaleOnFailedCheckpointCount; + private final boolean activeCheckpointTriggerEnabled; Factory( ExecutionGraph executionGraph, @@ -410,7 +484,8 @@ static class Factory implements StateFactory { List failureCollection, Function stateTransitionManagerFactory, - int rescaleOnFailedCheckpointCount) { + int rescaleOnFailedCheckpointCount, + boolean activeCheckpointTriggerEnabled) { this.context = context; this.log = log; this.executionGraph = executionGraph; @@ -420,6 +495,7 @@ static class Factory implements StateFactory { this.failureCollection = failureCollection; this.stateTransitionManagerFactory = stateTransitionManagerFactory; this.rescaleOnFailedCheckpointCount = rescaleOnFailedCheckpointCount; + this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled; } public Class getStateClass() { @@ -436,7 +512,8 @@ public Executing getState() { userCodeClassLoader, failureCollection, stateTransitionManagerFactory, - rescaleOnFailedCheckpointCount); + rescaleOnFailedCheckpointCount, + activeCheckpointTriggerEnabled); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateTransitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateTransitionManager.java index 98229a9afd3e3..e057addd40f4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateTransitionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateTransitionManager.java @@ -89,5 +89,24 @@ interface Context extends RescaleContext { * @return the {@link JobID} of the job */ JobID getJobId(); + + /** + * Requests the context to actively trigger a checkpoint to expedite rescaling. Called by + * the {@link DefaultStateTransitionManager} from within phase lifecycle methods: + * + *
    + *
  • On entering {@link DefaultStateTransitionManager.Stabilizing} (to overlap + * checkpoint with the stabilization wait) + *
  • On each {@link DefaultStateTransitionManager.Stabilizing#onChange} event (retry if + * a previous trigger was skipped) + *
  • On entering {@link DefaultStateTransitionManager.Stabilized} (fallback if no + * checkpoint completed during stabilization) + *
+ * + *

The implementation decides whether to actually trigger based on its own guard + * conditions (e.g., checkpointing enabled, no checkpoint in progress, config flag). + * Multiple calls are safe; guards prevent redundant triggers. + */ + default void requestActiveCheckpointTrigger() {} } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 0a505ab07f2c6..5493e05667c68 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -63,6 +63,7 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.clock.Clock; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; @@ -791,6 +792,8 @@ public static class CheckpointCoordinatorBuilder { VertexFinishedStateChecker> vertexFinishedStateCheckerFactory = VertexFinishedStateChecker::new; + private Clock clock = SystemClock.getInstance(); + public CheckpointCoordinatorBuilder setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration) { this.checkpointCoordinatorConfiguration = checkpointCoordinatorConfiguration; @@ -870,6 +873,11 @@ public CheckpointCoordinatorBuilder setVertexFinishedStateCheckerFactory( return this; } + public CheckpointCoordinatorBuilder setClock(Clock clock) { + this.clock = clock; + return this; + } + public CheckpointCoordinator build(ScheduledExecutorService executorService) throws Exception { return build( @@ -899,7 +907,7 @@ public CheckpointCoordinator build(ExecutionGraph executionGraph) throws Excepti timer, failureManager, checkpointPlanCalculator, - SystemClock.getInstance(), + clock, checkpointStatsTracker, vertexFinishedStateCheckerFactory); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManagerTest.java index cf6051a90b315..83c9002d9facf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/DefaultStateTransitionManagerTest.java @@ -400,6 +400,59 @@ private static void assertFinalStateTransitionHappened( assertThat(testInstance.getPhase()).isInstanceOf(Transitioning.class); } + @Test + void testActiveCheckpointTriggerCalledOnEnteringStabilizing() { + final TestingStateTransitionManagerContext ctx = + TestingStateTransitionManagerContext.stableContext(); + ctx.withSufficientResources(); + final DefaultStateTransitionManager testInstance = + ctx.createTestInstanceThatPassedCooldownPhase(); + + assertThat(testInstance.getPhase()).isInstanceOf(Idling.class); + ctx.clearActiveCheckpointTriggerCount(); + + testInstance.onChange(true); + + assertThat(testInstance.getPhase()).isInstanceOf(Stabilizing.class); + assertThat(ctx.getActiveCheckpointTriggerCount()).isGreaterThanOrEqualTo(1); + } + + @Test + void testActiveCheckpointTriggerCalledOnChangeInStabilizing() { + final TestingStateTransitionManagerContext ctx = + TestingStateTransitionManagerContext.stableContext(); + ctx.withSufficientResources(); + final DefaultStateTransitionManager testInstance = + ctx.createTestInstanceThatPassedCooldownPhase(); + + testInstance.onChange(true); + assertThat(testInstance.getPhase()).isInstanceOf(Stabilizing.class); + ctx.clearActiveCheckpointTriggerCount(); + + testInstance.onChange(true); + + assertThat(testInstance.getPhase()).isInstanceOf(Stabilizing.class); + assertThat(ctx.getActiveCheckpointTriggerCount()).isGreaterThanOrEqualTo(1); + } + + @Test + void testActiveCheckpointTriggerCalledOnEnteringStabilized() { + final TestingStateTransitionManagerContext ctx = + TestingStateTransitionManagerContext.stableContext(); + ctx.withSufficientResources(); + final DefaultStateTransitionManager testInstance = + ctx.createTestInstanceThatPassedCooldownPhase(); + + testInstance.onChange(true); + assertThat(testInstance.getPhase()).isInstanceOf(Stabilizing.class); + ctx.clearActiveCheckpointTriggerCount(); + + ctx.passResourceStabilizationTimeout(); + + assertThat(testInstance.getPhase()).isInstanceOf(Stabilized.class); + assertThat(ctx.getActiveCheckpointTriggerCount()).isGreaterThanOrEqualTo(1); + } + private static void changeWithoutPhaseMove( TestingStateTransitionManagerContext ctx, DefaultStateTransitionManager testInstance, @@ -460,6 +513,7 @@ private static class TestingStateTransitionManagerContext // internal state used for assertions private final AtomicBoolean transitionTriggered = new AtomicBoolean(); + private int activeCheckpointTriggerCount = 0; private final SortedMap>> scheduledTasks = new TreeMap<>(); @@ -537,6 +591,11 @@ public void transitionToSubsequentState() { transitionTriggered.set(true); } + @Override + public void requestActiveCheckpointTrigger() { + activeCheckpointTriggerCount++; + } + @Override public ScheduledFuture scheduleOperation(Runnable callback, Duration delay) { final Instant triggerTime = @@ -703,5 +762,13 @@ public boolean stateTransitionWasTriggered() { public void clearStateTransition() { transitionTriggered.set(false); } + + public int getActiveCheckpointTriggerCount() { + return activeCheckpointTriggerCount; + } + + public void clearActiveCheckpointTriggerCount() { + activeCheckpointTriggerCount = 0; + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java index f593082f6c75b..cf216f0296e57 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java @@ -57,6 +57,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.coordination.CoordinatorStoreImpl; import org.apache.flink.runtime.scheduler.DefaultVertexParallelismInfo; @@ -75,6 +76,8 @@ import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.clock.ManualClock; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -160,7 +163,8 @@ void testNoDeploymentCallOnEnterWhenVertexRunning() throws Exception { ClassLoader.getSystemClassLoader(), new ArrayList<>(), (context) -> TestingStateTransitionManager.withNoOp(), - 1); + 1, + false); assertThat(mockExecutionVertex.isDeployCalled()).isFalse(); } } @@ -186,7 +190,8 @@ void testIllegalStateExceptionOnNotRunningExecutionGraph() { ClassLoader.getSystemClassLoader(), new ArrayList<>(), context -> TestingStateTransitionManager.withNoOp(), - 1); + 1, + false); } }) .isInstanceOf(IllegalStateException.class); @@ -556,6 +561,313 @@ public CheckpointCoordinator getCheckpointCoordinator() { } } + @Test + void testActiveCheckpointTriggerSkipsWhenDisabled() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + final AtomicBoolean coordinatorAccessed = new AtomicBoolean(false); + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .build(EXECUTOR_EXTENSION.getExecutor()); + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + coordinatorAccessed.set(true); + return coordinator; + } + }; + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(false) + .build(ctx); + + exec.requestActiveCheckpointTrigger(); + assertThat(coordinatorAccessed.get()).isFalse(); + } + } + + @Test + void testActiveCheckpointTriggerSkipsWhenNoCoordinator() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + ExecutionGraph graph = new MockExecutionGraph(() -> Collections.singletonList(mejv)); + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + exec.requestActiveCheckpointTrigger(); + } + } + + @Test + void testActiveCheckpointTriggerSkipsWhenPeriodicCheckpointingNotConfigured() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .setCheckpointCoordinatorConfiguration( + new CheckpointCoordinatorConfiguration + .CheckpointCoordinatorConfigurationBuilder() + .setCheckpointInterval(Long.MAX_VALUE) + .build()) + .build(EXECUTOR_EXTENSION.getExecutor()); + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + }; + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + + assertThat(coordinator.isPeriodicCheckpointingConfigured()).isFalse(); + exec.requestActiveCheckpointTrigger(); + } + } + + @Test + void testActiveCheckpointTriggerSkipsWhenParallelismUnchanged() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .build(EXECUTOR_EXTENSION.getExecutor()); + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + + @Override + public Iterable getVerticesTopologically() { + return Collections.singletonList(mejv); + } + }; + ctx.setVertexParallelism( + new VertexParallelism( + graph.getAllVertices().values().stream() + .collect( + Collectors.toMap( + AccessExecutionJobVertex::getJobVertexId, + AccessExecutionJobVertex::getParallelism)))); + + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + exec.requestActiveCheckpointTrigger(); + assertThat(coordinator.getNumberOfPendingCheckpoints()).isEqualTo(0); + } + } + + @Test + void testActiveCheckpointTriggerSkipsWhenCheckpointInProgress() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + + ManuallyTriggeredScheduledExecutor checkpointTimer = + new ManuallyTriggeredScheduledExecutor(); + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .setTimer(checkpointTimer) + .build(EXECUTOR_EXTENSION.getExecutor()); + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + + @Override + public Iterable getVerticesTopologically() { + return Collections.singletonList(mejv); + } + }; + + ctx.setVertexParallelism( + new VertexParallelism( + graph.getAllVertices().values().stream() + .collect( + Collectors.toMap( + AccessExecutionJobVertex::getJobVertexId, + v -> v.getParallelism() + 1)))); + + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + coordinator.triggerCheckpoint(false); + checkpointTimer.triggerAll(); + + int pendingBefore = coordinator.getNumberOfPendingCheckpoints(); + assertThat(pendingBefore).isGreaterThan(0); + exec.requestActiveCheckpointTrigger(); + checkpointTimer.triggerAll(); + assertThat(coordinator.getNumberOfPendingCheckpoints()).isEqualTo(pendingBefore); + } + } + + @Test + void testActiveCheckpointTriggerFiresWhenAllGuardsPass() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + + ManuallyTriggeredScheduledExecutor checkpointTimer = + new ManuallyTriggeredScheduledExecutor(); + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .setTimer(checkpointTimer) + .build(EXECUTOR_EXTENSION.getExecutor()); + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + + @Override + public Iterable getVerticesTopologically() { + return Collections.singletonList(mejv); + } + }; + ctx.setVertexParallelism( + new VertexParallelism( + graph.getAllVertices().values().stream() + .collect( + Collectors.toMap( + AccessExecutionJobVertex::getJobVertexId, + v -> v.getParallelism() + 1)))); + + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + + assertThat(coordinator.getNumberOfPendingCheckpoints()).isEqualTo(0); + + exec.requestActiveCheckpointTrigger(); + checkpointTimer.triggerAll(); + assertThat(coordinator.getNumberOfPendingCheckpoints()).isGreaterThan(0); + } + } + + @Test + void testActiveCheckpointTriggerRespectsMinPauseBetweenCheckpoints() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + + ManuallyTriggeredScheduledExecutor checkpointTimer = + new ManuallyTriggeredScheduledExecutor(); + ManualClock clock = new ManualClock(); + CheckpointCoordinatorConfiguration coordConfig = + new CheckpointCoordinatorConfiguration + .CheckpointCoordinatorConfigurationBuilder() + .setCheckpointInterval(10_000L) + .setMinPauseBetweenCheckpoints(10_000L) + .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) + .build(); + + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .setCheckpointCoordinatorConfiguration(coordConfig) + .setTimer(checkpointTimer) + .setClock(clock) + .build(EXECUTOR_EXTENSION.getExecutor()); + + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + + @Override + public Iterable getVerticesTopologically() { + return Collections.singletonList(mejv); + } + }; + ctx.setVertexParallelism( + new VertexParallelism( + graph.getAllVertices().values().stream() + .collect( + Collectors.toMap( + AccessExecutionJobVertex::getJobVertexId, + v -> v.getParallelism() + 1)))); + + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + exec.requestActiveCheckpointTrigger(); + checkpointTimer.triggerAll(); + assertThat(coordinator.getNumberOfPendingCheckpoints()).isEqualTo(0); + clock.advanceTime(10_000L, java.util.concurrent.TimeUnit.MILLISECONDS); + exec.requestActiveCheckpointTrigger(); + checkpointTimer.triggerAll(); + assertThat(coordinator.getNumberOfPendingCheckpoints()).isEqualTo(1); + } + } + + @Test + void testActiveCheckpointTriggerHandlesFailureGracefully() throws Exception { + try (MockExecutingContext ctx = new MockExecutingContext()) { + MockExecutionJobVertex mejv = new MockExecutionJobVertex(MockExecutionVertex::new); + ManuallyTriggeredScheduledExecutor checkpointTimer = + new ManuallyTriggeredScheduledExecutor(); + CheckpointCoordinator coordinator = + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .setTimer(checkpointTimer) + .build(EXECUTOR_EXTENSION.getExecutor()); + coordinator.shutdown(); + + StateTrackingMockExecutionGraph graph = + new StateTrackingMockExecutionGraph() { + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return coordinator; + } + + @Override + public Iterable getVerticesTopologically() { + return Collections.singletonList(mejv); + } + }; + ctx.setVertexParallelism( + new VertexParallelism( + graph.getAllVertices().values().stream() + .collect( + Collectors.toMap( + AccessExecutionJobVertex::getJobVertexId, + v -> v.getParallelism() + 1)))); + + Executing exec = + new ExecutingStateBuilder() + .setExecutionGraph(graph) + .setActiveCheckpointTriggerEnabled(true) + .build(ctx); + exec.requestActiveCheckpointTrigger(); + checkpointTimer.triggerAll(); + } + } + @Test void testJobInformationMethods() throws Exception { try (MockExecutingContext ctx = new MockExecutingContext()) { @@ -691,6 +1003,7 @@ private final class ExecutingStateBuilder { private Function stateTransitionManagerFactory = context -> TestingStateTransitionManager.withNoOp(); private int rescaleOnFailedCheckpointCount = 1; + private boolean activeCheckpointTriggerEnabled = false; private ExecutingStateBuilder() throws JobException, JobExecutionException { operatorCoordinatorHandler = new TestingOperatorCoordinatorHandler(); @@ -720,6 +1033,12 @@ public ExecutingStateBuilder setRescaleOnFailedCheckpointCount( return this; } + public ExecutingStateBuilder setActiveCheckpointTriggerEnabled( + boolean activeCheckpointTriggerEnabled) { + this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled; + return this; + } + private Executing build(MockExecutingContext ctx) { executionGraph.transitionToRunning(); @@ -733,7 +1052,8 @@ private Executing build(MockExecutingContext ctx) { ClassLoader.getSystemClassLoader(), new ArrayList<>(), stateTransitionManagerFactory::apply, - rescaleOnFailedCheckpointCount); + rescaleOnFailedCheckpointCount, + activeCheckpointTriggerEnabled); } finally { Preconditions.checkState( !ctx.hadStateTransition, @@ -1029,6 +1349,12 @@ public boolean updateState(TaskExecutionStateTransition state) { public Iterable getVerticesTopologically() { return getVerticesTopologicallySupplier.get(); } + + @Nullable + @Override + public CheckpointCoordinator getCheckpointCoordinator() { + return null; + } } private static class FinishingMockExecutionGraph extends StateTrackingMockExecutionGraph { diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/RescaleOnCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/RescaleOnCheckpointITCase.java index 6a24600f1ace1..dcc9ceefba776 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/RescaleOnCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/RescaleOnCheckpointITCase.java @@ -175,4 +175,93 @@ void testRescaleOnCheckpoint( restClusterClient.cancel(jobGraph.getJobID()).join(); } } + + @Test + void testRescaleWithActiveCheckpointTrigger( + @InjectMiniCluster MiniCluster miniCluster, + @InjectClusterClient RestClusterClient restClusterClient) + throws Exception { + final Configuration config = new Configuration(); + config.set(JobManagerOptions.SCHEDULER_RESCALE_TRIGGER_ACTIVE_CHECKPOINT_ENABLED, true); + + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(BEFORE_RESCALE_PARALLELISM); + env.enableCheckpointing(Duration.ofHours(24).toMillis()); + env.fromSequence(0, Integer.MAX_VALUE).sinkTo(new DiscardingSink<>()); + + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + final Iterator jobVertexIterator = jobGraph.getVertices().iterator(); + assertThat(jobVertexIterator.hasNext()).isTrue(); + final JobVertexID jobVertexId = jobVertexIterator.next().getID(); + + final JobResourceRequirements jobResourceRequirements = + JobResourceRequirements.newBuilder() + .setParallelismForJobVertex(jobVertexId, 1, AFTER_RESCALE_PARALLELISM) + .build(); + + restClusterClient.submitJob(jobGraph).join(); + + final JobID jobId = jobGraph.getJobID(); + try { + LOG.info( + "Waiting for job {} to reach parallelism of {} for vertex {}.", + jobId, + BEFORE_RESCALE_PARALLELISM, + jobVertexId); + waitForRunningTasks(restClusterClient, jobId, BEFORE_RESCALE_PARALLELISM); + + LOG.info( + "Updating job {} resource requirements: parallelism {} -> {}.", + jobId, + BEFORE_RESCALE_PARALLELISM, + AFTER_RESCALE_PARALLELISM); + restClusterClient.updateJobResourceRequirements(jobId, jobResourceRequirements).join(); + LOG.info( + "Waiting for job {} to rescale to parallelism {} via active checkpoint trigger.", + jobId, + AFTER_RESCALE_PARALLELISM); + waitForRunningTasks(restClusterClient, jobId, AFTER_RESCALE_PARALLELISM); + final int expectedFreeSlotCount = NUMBER_OF_SLOTS - AFTER_RESCALE_PARALLELISM; + LOG.info( + "Waiting for {} slot(s) to become available after scale down.", + expectedFreeSlotCount); + waitForAvailableSlots(restClusterClient, expectedFreeSlotCount); + } finally { + restClusterClient.cancel(jobGraph.getJobID()).join(); + } + } + + @Test + void testNoRescaleWithoutCheckpointingConfigured( + @InjectMiniCluster MiniCluster miniCluster, + @InjectClusterClient RestClusterClient restClusterClient) + throws Exception { + final Configuration config = new Configuration(); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(config); + env.setParallelism(BEFORE_RESCALE_PARALLELISM); + env.fromSequence(0, Integer.MAX_VALUE).sinkTo(new DiscardingSink<>()); + + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + final Iterator jobVertexIterator = jobGraph.getVertices().iterator(); + assertThat(jobVertexIterator.hasNext()).isTrue(); + final JobVertexID jobVertexId = jobVertexIterator.next().getID(); + + final JobResourceRequirements jobResourceRequirements = + JobResourceRequirements.newBuilder() + .setParallelismForJobVertex(jobVertexId, 1, AFTER_RESCALE_PARALLELISM) + .build(); + restClusterClient.submitJob(jobGraph).join(); + final JobID jobId = jobGraph.getJobID(); + try { + waitForRunningTasks(restClusterClient, jobId, BEFORE_RESCALE_PARALLELISM); + restClusterClient.updateJobResourceRequirements(jobId, jobResourceRequirements).join(); + Thread.sleep(REQUIREMENT_UPDATE_TO_CHECKPOINT_GAP.toMillis()); + waitForRunningTasks(restClusterClient, jobId, BEFORE_RESCALE_PARALLELISM); + LOG.info("Verified: job {} did not rescale without checkpointing configured.", jobId); + } finally { + restClusterClient.cancel(jobGraph.getJobID()).join(); + } + } }