115115import org .apache .flink .runtime .scheduler .adaptive .allocator .ReservedSlots ;
116116import org .apache .flink .runtime .scheduler .adaptive .allocator .SlotAllocator ;
117117import org .apache .flink .runtime .scheduler .adaptive .allocator .VertexParallelism ;
118+ import org .apache .flink .runtime .scheduler .adaptive .timeline .DefaultRescaleTimeline ;
119+ import org .apache .flink .runtime .scheduler .adaptive .timeline .Rescale ;
120+ import org .apache .flink .runtime .scheduler .adaptive .timeline .RescaleTimeline ;
121+ import org .apache .flink .runtime .scheduler .adaptive .timeline .TerminalState ;
122+ import org .apache .flink .runtime .scheduler .adaptive .timeline .TerminatedReason ;
123+ import org .apache .flink .runtime .scheduler .adaptive .timeline .TriggerCause ;
118124import org .apache .flink .runtime .scheduler .adaptivebatch .NonAdaptiveExecutionPlanSchedulingContext ;
119125import org .apache .flink .runtime .scheduler .exceptionhistory .ExceptionHistoryEntry ;
120126import org .apache .flink .runtime .scheduler .exceptionhistory .RootExceptionHistoryEntry ;
149155import java .util .Iterator ;
150156import java .util .List ;
151157import java .util .Map ;
158+ import java .util .Objects ;
152159import java .util .Optional ;
153160import java .util .concurrent .CompletableFuture ;
154161import java .util .concurrent .Executor ;
162169import static org .apache .flink .configuration .TraceOptions .CHECKPOINT_SPAN_DETAIL_LEVEL ;
163170import static org .apache .flink .runtime .executiongraph .ExecutionGraphUtils .isAnyOutputBlocking ;
164171import static org .apache .flink .runtime .scheduler .adaptive .ForwardEdgesAdapter .copyJobGraphWithAdaptedForwardEdges ;
172+ import static org .apache .flink .runtime .scheduler .adaptive .timeline .RescaleTimeline .NoOpRescaleTimeline ;
165173
166174/**
167175 * A {@link SchedulerNG} implementation that uses the declarative resource management and
@@ -453,6 +461,8 @@ public JobRescaleConfigInfo toJobRescaleConfigInfo() {
453461
454462 private final Supplier <Temporal > clock = Instant ::now ;
455463
464+ private final RescaleTimeline rescaleTimeline ;
465+
456466 public AdaptiveScheduler (
457467 Settings settings ,
458468 JobGraph jobGraph ,
@@ -546,6 +556,12 @@ public AdaptiveScheduler(
546556 this .initialParallelismStore = vertexParallelismStore ;
547557 this .jobInformation = new JobGraphJobInformation (jobGraph , vertexParallelismStore );
548558
559+ final int rescaleHistoryMax = settings .getRescaleHistoryMax ();
560+ this .rescaleTimeline =
561+ rescaleHistoryMax <= 0
562+ ? NoOpRescaleTimeline .INSTANCE
563+ : new DefaultRescaleTimeline (() -> jobInformation , rescaleHistoryMax );
564+
549565 this .declarativeSlotPool = declarativeSlotPool ;
550566 this .initializationTimestamp = initializationTimestamp ;
551567 this .ioExecutor = ioExecutor ;
@@ -730,6 +746,11 @@ static VertexParallelismStore computeVertexParallelismStoreForExecution(
730746 jobGraph .getVertices (), defaultMaxParallelismFunc );
731747 }
732748
749+ @ Override
750+ public RescaleTimeline getRescaleTimeline () {
751+ return rescaleTimeline ;
752+ }
753+
733754 private void newResourcesAvailable (Collection <? extends PhysicalSlot > physicalSlots ) {
734755 state .tryRun (
735756 ResourceListener .class ,
@@ -1125,6 +1146,7 @@ public void updateJobResourceRequirements(JobResourceRequirements jobResourceReq
11251146 if (maybeUpdateVertexParallelismStore .isPresent ()) {
11261147 this .jobInformation =
11271148 new JobGraphJobInformation (jobGraph , maybeUpdateVertexParallelismStore .get ());
1149+ recordRescaleForNewResourceRequirements ();
11281150 declareDesiredResources ();
11291151 state .tryRun (
11301152 ResourceListener .class ,
@@ -1133,6 +1155,27 @@ public void updateJobResourceRequirements(JobResourceRequirements jobResourceReq
11331155 }
11341156 }
11351157
1158+ private void recordRescaleForNewResourceRequirements () {
1159+ rescaleTimeline .updateRescale (
1160+ rescale ->
1161+ rescale .addSchedulerState (state )
1162+ .setTerminatedReason (TerminatedReason .RESOURCE_REQUIREMENTS_UPDATED )
1163+ .setEndToNow ()
1164+ .log ());
1165+ rescaleTimeline .newRescale (true );
1166+ rescaleTimeline .updateRescale (
1167+ rescale ->
1168+ rescale .setStartToNow ()
1169+ .setDesiredVertexParallelism (jobInformation )
1170+ .setTriggerCause (TriggerCause .UPDATE_REQUIREMENT )
1171+ .setDesiredSlots (jobInformation )
1172+ .setMinimalRequiredSlots (jobInformation )
1173+ .setPreRescaleSlotsAndParallelisms (
1174+ jobInformation ,
1175+ rescaleTimeline .getLatestRescale (TerminalState .COMPLETED ))
1176+ .log ());
1177+ }
1178+
11361179 // ----------------------------------------------------------------
11371180
11381181 @ Override
@@ -1288,6 +1331,15 @@ public void goToCanceling(
12881331 OperatorCoordinatorHandler operatorCoordinatorHandler ,
12891332 List <ExceptionHistoryEntry > failureCollection ) {
12901333
1334+ if (!rescaleTimeline .isIdling ()) {
1335+ rescaleTimeline .updateRescale (
1336+ rescale ->
1337+ rescale .addSchedulerState (state )
1338+ .setEndToNow ()
1339+ .setTerminatedReason (TerminatedReason .JOB_CANCELED )
1340+ .log ());
1341+ }
1342+
12911343 transitionToState (
12921344 new Canceling .Factory (
12931345 this ,
@@ -1308,6 +1360,8 @@ public void goToRestarting(
13081360 @ Nullable VertexParallelism restartWithParallelism ,
13091361 List <ExceptionHistoryEntry > failureCollection ) {
13101362
1363+ recordRescaleForJobRestarting (restartWithParallelism );
1364+
13111365 for (ExecutionVertex executionVertex : executionGraph .getAllExecutionVertices ()) {
13121366 final int attemptNumber =
13131367 executionVertex .getCurrentExecutionAttempt ().getAttemptNumber ();
@@ -1336,13 +1390,54 @@ public void goToRestarting(
13361390 }
13371391 }
13381392
1393+ private void recordRescaleForJobRestarting (VertexParallelism restartWithParallelism ) {
1394+ // For the normal rescaling restarting.
1395+ if (Objects .nonNull (restartWithParallelism )) {
1396+ rescaleTimeline .updateRescale (
1397+ rescale ->
1398+ rescale .setMinimalRequiredSlots (jobInformation )
1399+ .setDesiredVertexParallelism (jobInformation )
1400+ .setDesiredSlots (jobInformation )
1401+ .log ());
1402+ return ;
1403+ }
1404+ // For the failover restarting.
1405+ if (!rescaleTimeline .isIdling ()) {
1406+ // Process by https://lists.apache.org/thread/hh7w2p6lnmbo1q6d9ngkttdyrw4lp74h.
1407+ LOG .info (
1408+ "Merge the current non-terminated rescale and the new rescale triggered by recoverable failover into the current rescale." );
1409+ rescaleTimeline .updateRescale (Rescale ::clearSchedulerStates );
1410+ } else if (rescaleTimeline .isIdling ()) {
1411+ rescaleTimeline .newRescale (false );
1412+ }
1413+ rescaleTimeline .updateRescale (
1414+ rescale ->
1415+ rescale .setStartToNow ()
1416+ .setTriggerCause (TriggerCause .RECOVERABLE_FAILOVER )
1417+ .setMinimalRequiredSlots (jobInformation )
1418+ .setPreRescaleSlotsAndParallelisms (
1419+ jobInformation ,
1420+ rescaleTimeline .getLatestRescale (TerminalState .COMPLETED ))
1421+ .setDesiredVertexParallelism (jobInformation )
1422+ .setDesiredSlots (jobInformation )
1423+ .log ());
1424+ }
1425+
13391426 @ Override
13401427 public void goToFailing (
13411428 ExecutionGraph executionGraph ,
13421429 ExecutionGraphHandler executionGraphHandler ,
13431430 OperatorCoordinatorHandler operatorCoordinatorHandler ,
13441431 Throwable failureCause ,
13451432 List <ExceptionHistoryEntry > failureCollection ) {
1433+ rescaleTimeline .updateRescale (
1434+ rescale ->
1435+ rescale .setEndToNow ()
1436+ .addSchedulerState (state , failureCause )
1437+ .setTerminatedReason (TerminatedReason .JOB_FAILED )
1438+ .setStringifiedException (
1439+ ExceptionUtils .stringifyException (failureCause ))
1440+ .log ());
13461441 transitionToState (
13471442 new Failing .Factory (
13481443 this ,
@@ -1381,6 +1476,12 @@ public CompletableFuture<String> goToStopWithSavepoint(
13811476
13821477 @ Override
13831478 public void goToFinished (ArchivedExecutionGraph archivedExecutionGraph ) {
1479+ rescaleTimeline .updateRescale (
1480+ rescale ->
1481+ rescale .addSchedulerState (state )
1482+ .setEndToNow ()
1483+ .setTerminatedReason (TerminatedReason .JOB_FINISHED )
1484+ .log ());
13841485 transitionToState (new Finished .Factory (this , archivedExecutionGraph , LOG ));
13851486 }
13861487
@@ -1650,6 +1751,9 @@ <T extends State> T transitionToState(StateFactory<T> targetState) {
16501751 final JobStatus previousJobStatus = state .getJobStatus ();
16511752
16521753 state .onLeave (targetState .getStateClass ());
1754+
1755+ rescaleTimeline .updateRescale (rescale -> rescale .addSchedulerState (state ));
1756+
16531757 T targetStateInstance = targetState .getState ();
16541758 state = targetStateInstance ;
16551759
@@ -1669,8 +1773,7 @@ <T extends State> T transitionToState(StateFactory<T> targetState) {
16691773 }
16701774 }
16711775
1672- @ VisibleForTesting
1673- State getState () {
1776+ public State getState () {
16741777 return state ;
16751778 }
16761779
0 commit comments