Skip to content
Open
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
6 changes: 6 additions & 0 deletions docs/layouts/shortcodes/generated/all_jobmanager_section.html
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,12 @@
<td>Boolean</td>
<td>This parameter defines whether the adaptive scheduler prioritizes using the minimum number of <code class="highlighter-rouge">TaskManagers</code> when scheduling tasks.<br />Note, this parameter is suitable if <code class="highlighter-rouge">execution.state-recovery.from-local</code> is not enabled. More details about this configuration are available at <a href="https://issues.apache.org/jira/browse/FLINK-33977">FLINK-33977</a>.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>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 <code class="highlighter-rouge">execution.checkpointing.min-pause</code> and will not fire if a checkpoint is already in progress or being triggered.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures</h5></td>
<td style="word-wrap: break-word;">2</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,12 @@
<td>Boolean</td>
<td>This parameter defines whether the adaptive scheduler prioritizes using the minimum number of <code class="highlighter-rouge">TaskManagers</code> when scheduling tasks.<br />Note, this parameter is suitable if <code class="highlighter-rouge">execution.state-recovery.from-local</code> is not enabled. More details about this configuration are available at <a href="https://issues.apache.org/jira/browse/FLINK-33977">FLINK-33977</a>.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>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 <code class="highlighter-rouge">execution.checkpointing.min-pause</code> and will not fire if a checkpoint is already in progress or being triggered.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures</h5></td>
<td style="word-wrap: break-word;">2</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,12 @@
<td>Boolean</td>
<td>This parameter defines whether the adaptive scheduler prioritizes using the minimum number of <code class="highlighter-rouge">TaskManagers</code> when scheduling tasks.<br />Note, this parameter is suitable if <code class="highlighter-rouge">execution.state-recovery.from-local</code> is not enabled. More details about this configuration are available at <a href="https://issues.apache.org/jira/browse/FLINK-33977">FLINK-33977</a>.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.active-checkpoint.enabled</h5></td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>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 <code class="highlighter-rouge">execution.checkpointing.min-pause</code> and will not fire if a checkpoint is already in progress or being triggered.</td>
</tr>
<tr>
<td><h5>jobmanager.adaptive-scheduler.rescale-trigger.max-checkpoint-failures</h5></td>
<td style="word-wrap: break-word;">2</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -745,6 +745,25 @@ public InlineElement getDescription() {
.key()))
.build());

@Documentation.Section({
Documentation.Sections.EXPERT_SCHEDULING,
Documentation.Sections.ALL_JOB_MANAGER
})
public static final ConfigOption<Boolean> 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 %s and "
+ "will not fire if a checkpoint is already in progress or being triggered.",
code("execution.checkpointing.min-pause"))
.build());

/**
* @deprecated Use {@link
* JobManagerOptions#SCHEDULER_SUBMISSION_RESOURCE_STABILIZATION_TIMEOUT}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1999,6 +1999,21 @@ public long getCheckpointTimeout() {
return checkpointTimeout;
}

/**
* Returns {@code -1} if a checkpoint is already in flight, otherwise the remaining time (in ms)
* until {@code minPauseBetweenCheckpoints} is satisfied ({@code 0} = trigger now). All checks
* are made under the coordinator lock.
*/
public long getActiveCheckpointTriggerDelay() {
synchronized (lock) {
if (isTriggering || !pendingCheckpoints.isEmpty()) {
return -1L;
}
final long elapsed = clock.relativeTimeMillis() - lastCheckpointCompletionRelativeTime;
return Math.max(0L, minPauseBetweenCheckpoints - elapsed);
}
}

/**
* @deprecated use {@link #getNumQueuedRequests()}
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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,
Expand All @@ -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;
Expand All @@ -346,6 +350,7 @@ private Settings(
this.maximumDelayForTriggeringRescale = maximumDelayForTriggeringRescale;
this.rescaleOnFailedCheckpointCount = rescaleOnFailedCheckpointCount;
this.rescaleHistoryMax = rescaleHistoryMax;
this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled;
}

public SchedulerExecutionMode getExecutionMode() {
Expand Down Expand Up @@ -384,6 +389,10 @@ public int getRescaleHistoryMax() {
return rescaleHistoryMax;
}

public boolean isActiveCheckpointTriggerEnabled() {
return activeCheckpointTriggerEnabled;
}

public JobRescaleConfigInfo toJobRescaleConfigInfo() {
return new JobRescaleConfigInfo(
rescaleHistoryMax,
Expand Down Expand Up @@ -1311,7 +1320,8 @@ public void goToExecuting(
userCodeClassLoader,
failureCollection,
this::createExecutingStateTransitionManager,
settings.getRescaleOnFailedCheckpointCount()));
settings.getRescaleOnFailedCheckpointCount(),
settings.isActiveCheckpointTriggerEnabled()));
}

private StateTransitionManager createExecutingStateTransitionManager(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down Expand Up @@ -362,6 +366,7 @@ private Stabilizing(
resourceStabilizationTimeout);

scheduleTransitionEvaluation();
context().requestActiveCheckpointTrigger();
}

@Override
Expand All @@ -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
Expand Down Expand Up @@ -427,6 +433,7 @@ private Stabilized(
},
firstChangeEventTimestamp,
maxTriggerDelay);
context().requestActiveCheckpointTrigger();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@

package org.apache.flink.runtime.scheduler.adaptive;

import org.apache.flink.api.common.JobID;
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;
Expand All @@ -40,6 +42,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;

Expand All @@ -64,6 +67,8 @@ class Executing extends StateWithExecutionGraph

private final StateTransitionManager stateTransitionManager;
private final int rescaleOnFailedCheckpointCount;
private final boolean activeCheckpointTriggerEnabled;
private boolean activeCheckpointTriggerScheduled;
// null indicates that there was no change event observed, yet
@Nullable private AtomicInteger failedCheckpointCountdown;

Expand All @@ -77,7 +82,8 @@ class Executing extends StateWithExecutionGraph
List<ExceptionHistoryEntry> failureCollection,
Function<StateTransitionManager.Context, StateTransitionManager>
stateTransitionManagerFactory,
int rescaleOnFailedCheckpointCount) {
int rescaleOnFailedCheckpointCount,
boolean activeCheckpointTriggerEnabled) {
super(
context,
executionGraph,
Expand All @@ -96,6 +102,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);
Expand Down Expand Up @@ -182,6 +189,103 @@ public ScheduledFuture<?> scheduleOperation(Runnable callback, Duration delay) {
return context.runIfState(this, callback, delay);
}

@Override
public void requestActiveCheckpointTrigger() {
if (!activeCheckpointTriggerEnabled) {
return;
}

final CheckpointCoordinator checkpointCoordinator =
getExecutionGraph().getCheckpointCoordinator();
if (!shouldTriggerActiveCheckpoint(checkpointCoordinator)) {
return;
}

final long triggerDelayMillis = checkpointCoordinator.getActiveCheckpointTriggerDelay();
if (triggerDelayMillis < 0L) {
getLogger()
.debug(
"Skipping active checkpoint trigger for rescale: checkpoint already in progress.");
return;
}
if (triggerDelayMillis > 0L) {
scheduleActiveCheckpointTriggerRetry(triggerDelayMillis);
return;
}
fireActiveCheckpointTrigger(checkpointCoordinator);
}

private boolean shouldTriggerActiveCheckpoint(
@Nullable CheckpointCoordinator checkpointCoordinator) {
if (checkpointCoordinator == null
|| !checkpointCoordinator.isPeriodicCheckpointingConfigured()) {
getLogger()
.debug(
"Skipping active checkpoint trigger for rescale: checkpointing not configured.");
return false;
}
if (!parallelismChanged()) {
getLogger()
.debug(
"Skipping active checkpoint trigger for rescale: parallelism unchanged.");
return false;
}
return true;
}

private void scheduleActiveCheckpointTriggerRetry(long delayMillis) {
if (activeCheckpointTriggerScheduled) {
return;
}
activeCheckpointTriggerScheduled = true;
getLogger()
.debug(
"Min pause not satisfied, scheduling active checkpoint trigger retry in {} ms.",
delayMillis);
context.runIfState(
this, this::tryFireActiveCheckpointAfterRetry, Duration.ofMillis(delayMillis));
}

private void tryFireActiveCheckpointAfterRetry() {
activeCheckpointTriggerScheduled = false;
final CheckpointCoordinator checkpointCoordinator =
getExecutionGraph().getCheckpointCoordinator();
if (!shouldTriggerActiveCheckpoint(checkpointCoordinator)) {
return;
}
if (checkpointCoordinator.getActiveCheckpointTriggerDelay() == 0L) {
fireActiveCheckpointTrigger(checkpointCoordinator);
}
}

private void fireActiveCheckpointTrigger(CheckpointCoordinator checkpointCoordinator) {
Preconditions.checkState(
activeCheckpointTriggerEnabled,
"Active checkpoint trigger fired while the feature is disabled.");
activeCheckpointTriggerScheduled = false;
final JobID jobId = getExecutionGraph().getJobID();
getLogger().info("Actively triggering checkpoint to expedite rescaling, job {}.", jobId);
// isPeriodic=false: min-pause is enforced above via getActiveCheckpointTriggerDelay.
FutureUtils.assertNoException(
checkpointCoordinator
.triggerCheckpoint(false)
.handle(
(completedCheckpoint, throwable) -> {
if (throwable != null) {
getLogger()
.warn(
"Active checkpoint trigger for rescale failed.",
throwable);
} else {
getLogger()
.info(
"Active checkpoint for rescale completed successfully: {}.",
completedCheckpoint.getCheckpointID());
}
return null;
}));
}

@Override
public void transitionToSubsequentState() {
Optional<VertexParallelism> availableVertexParallelism =
Expand Down Expand Up @@ -399,6 +503,7 @@ static class Factory implements StateFactory<Executing> {
private final Function<StateTransitionManager.Context, StateTransitionManager>
stateTransitionManagerFactory;
private final int rescaleOnFailedCheckpointCount;
private final boolean activeCheckpointTriggerEnabled;

Factory(
ExecutionGraph executionGraph,
Expand All @@ -410,7 +515,8 @@ static class Factory implements StateFactory<Executing> {
List<ExceptionHistoryEntry> failureCollection,
Function<StateTransitionManager.Context, StateTransitionManager>
stateTransitionManagerFactory,
int rescaleOnFailedCheckpointCount) {
int rescaleOnFailedCheckpointCount,
boolean activeCheckpointTriggerEnabled) {
this.context = context;
this.log = log;
this.executionGraph = executionGraph;
Expand All @@ -420,6 +526,7 @@ static class Factory implements StateFactory<Executing> {
this.failureCollection = failureCollection;
this.stateTransitionManagerFactory = stateTransitionManagerFactory;
this.rescaleOnFailedCheckpointCount = rescaleOnFailedCheckpointCount;
this.activeCheckpointTriggerEnabled = activeCheckpointTriggerEnabled;
}

public Class<Executing> getStateClass() {
Expand All @@ -436,7 +543,8 @@ public Executing getState() {
userCodeClassLoader,
failureCollection,
stateTransitionManagerFactory,
rescaleOnFailedCheckpointCount);
rescaleOnFailedCheckpointCount,
activeCheckpointTriggerEnabled);
}
}
}
Loading