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 the configured minimum pause between checkpoints 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 the configured minimum pause between checkpoints 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 the configured minimum pause between checkpoints 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,24 @@ 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 the configured minimum pause between checkpoints and "
+ "will not fire if a checkpoint is already in progress or being triggered.")
.build());

/**
* @deprecated Use {@link
* JobManagerOptions#SCHEDULER_SUBMISSION_RESOURCE_STABILIZATION_TIMEOUT}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import javax.annotation.concurrent.GuardedBy;

import java.io.IOException;
import java.time.Duration;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
Expand Down Expand Up @@ -1999,6 +2000,27 @@ public long getCheckpointTimeout() {
return checkpointTimeout;
}

/**
* Returns the remaining {@link Duration} until {@code minPauseBetweenCheckpoints} is satisfied
* for a new active-trigger checkpoint, computed from the time elapsed since the last completed
* checkpoint (or from the coordinator clock's epoch when no checkpoint has completed yet —
* which is normally far in the past in production). {@link Duration#ZERO} means the trigger can
* fire immediately.
*
* <p>Returns {@link Optional#empty()} as a fallback if a checkpoint is already in flight
* (triggering or pending), in which case no active trigger should be scheduled.
*/
public Optional<Duration> getActiveCheckpointTriggerDelay() {
synchronized (lock) {
if (isTriggering || !pendingCheckpoints.isEmpty()) {
return Optional.empty();
}
final long elapsed = clock.relativeTimeMillis() - lastCheckpointCompletionRelativeTime;
Comment thread
XComp marked this conversation as resolved.
final long remaining = minPauseBetweenCheckpoints - elapsed;
return Optional.of(remaining > 0 ? Duration.ofMillis(remaining) : Duration.ZERO);
}
}

/**
* @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,21 @@ private void progressToStabilized(Temporal firstChangeEventTimestamp) {
progressToPhase(new Stabilized(clock, this, firstChangeEventTimestamp, maxTriggerDelay));
}

/**
* Requests the active checkpoint trigger from the context. Called from within phase lifecycle
* methods:
*
* <ul>
* <li>On entering {@link Stabilizing} (to overlap checkpoint with the stabilization wait)
* <li>On each {@link Stabilizing#onChange} event (retry if a previous trigger was skipped)
* <li>On entering {@link Stabilized} (fallback if no checkpoint completed during
* stabilization)
* </ul>
*/
private void requestActiveCheckpointTrigger() {
transitionContext.requestActiveCheckpointTrigger();
}

private void triggerTransitionToSubsequentState() {
progressToPhase(new Transitioning(clock, this));
transitionContext.transitionToSubsequentState();
Expand Down Expand Up @@ -362,6 +377,7 @@ private Stabilizing(
resourceStabilizationTimeout);

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

@Override
Expand All @@ -370,6 +386,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 +444,7 @@ private Stabilized(
},
firstChangeEventTimestamp,
maxTriggerDelay);
context().requestActiveCheckpointTrigger();
}

@Override
Expand Down
Loading