2626import org .apache .amoro .api .OptimizingTask ;
2727import org .apache .amoro .api .OptimizingTaskId ;
2828import org .apache .amoro .api .OptimizingTaskResult ;
29+ import org .apache .amoro .client .AmsServerInfo ;
2930import org .apache .amoro .config .Configurations ;
3031import org .apache .amoro .config .TableConfiguration ;
3132import org .apache .amoro .exception .ForbiddenException ;
3839import org .apache .amoro .resource .ResourceType ;
3940import org .apache .amoro .server .catalog .CatalogManager ;
4041import org .apache .amoro .server .dashboard .model .OptimizerResourceInfo ;
42+ import org .apache .amoro .server .ha .HighAvailabilityContainer ;
4143import org .apache .amoro .server .manager .AbstractOptimizerContainer ;
4244import org .apache .amoro .server .optimizing .OptimizingProcess ;
4345import org .apache .amoro .server .optimizing .OptimizingQueue ;
6668
6769import java .time .Duration ;
6870import java .util .ArrayList ;
71+ import java .util .Collections ;
72+ import java .util .HashMap ;
73+ import java .util .HashSet ;
6974import java .util .List ;
7075import java .util .Map ;
7176import java .util .Objects ;
@@ -116,12 +121,17 @@ public class DefaultOptimizingService extends StatedPersistentBase
116121 private final TableService tableService ;
117122 private final RuntimeHandlerChain tableHandlerChain ;
118123 private final ExecutorService planExecutor ;
124+ private final BucketAssignStore bucketAssignStore ;
125+ private final HighAvailabilityContainer haContainer ;
126+ private final boolean isMasterSlaveMode ;
119127
120128 public DefaultOptimizingService (
121129 Configurations serviceConfig ,
122130 CatalogManager catalogManager ,
123131 OptimizerManager optimizerManager ,
124- TableService tableService ) {
132+ TableService tableService ,
133+ BucketAssignStore bucketAssignStore ,
134+ HighAvailabilityContainer haContainer ) {
125135 this .optimizerTouchTimeout =
126136 serviceConfig .getDurationInMillis (AmoroManagementConf .OPTIMIZER_HB_TIMEOUT );
127137 this .taskAckTimeout =
@@ -144,6 +154,10 @@ public DefaultOptimizingService(
144154 this .tableService = tableService ;
145155 this .catalogManager = catalogManager ;
146156 this .optimizerManager = optimizerManager ;
157+ this .bucketAssignStore = bucketAssignStore ;
158+ this .haContainer = haContainer ;
159+ this .isMasterSlaveMode =
160+ haContainer != null && serviceConfig .getBoolean (AmoroManagementConf .USE_MASTER_SLAVE_MODE );
147161 this .tableHandlerChain = new TableRuntimeHandlerImpl ();
148162 this .planExecutor =
149163 Executors .newCachedThreadPool (
@@ -322,6 +336,28 @@ public boolean cancelProcess(long processId) {
322336 return true ;
323337 }
324338
339+ @ Override
340+ public List <String > getOptimizingNodeUrls () {
341+ if (bucketAssignStore == null ) {
342+ return Collections .emptyList ();
343+ }
344+ try {
345+ List <AmsServerInfo > nodes = bucketAssignStore .getAliveNodes ();
346+ List <String > urls = new ArrayList <>(nodes .size ());
347+ for (AmsServerInfo node : nodes ) {
348+ if (node .getHost () != null
349+ && node .getThriftBindPort () != null
350+ && node .getThriftBindPort () > 0 ) {
351+ urls .add (String .format ("thrift://%s:%d" , node .getHost (), node .getThriftBindPort ()));
352+ }
353+ }
354+ return urls ;
355+ } catch (Exception e ) {
356+ LOG .warn ("Failed to get optimizing node URLs from bucket assign store" , e );
357+ return Collections .emptyList ();
358+ }
359+ }
360+
325361 /**
326362 * Get optimizing queue.
327363 *
@@ -530,10 +566,27 @@ public void dispose() {
530566
531567 @ Override
532568 public void run () {
569+ // Use 1/4 of optimizerTouchTimeout as sync interval (default ~30 seconds), used for
570+ // master-slave follower sync.
571+ long syncInterval = Math .max (5000 , optimizerTouchTimeout / 4 );
572+ // In non-master-slave mode, this node is always the leader.
573+ boolean wasLeader = !isMasterSlaveMode ;
533574 while (!stopped ) {
534575 try {
535- T keepingTask = suspendingQueue .take ();
536- this .processTask (keepingTask );
576+ boolean isLeader = !isMasterSlaveMode || haContainer .hasLeadership ();
577+ if (!wasLeader && isLeader ) {
578+ // Follower → Leader transition: subclass takes over monitoring of inherited optimizers.
579+ onBecomeLeader ();
580+ }
581+ wasLeader = isLeader ;
582+
583+ if (isLeader ) {
584+ T keepingTask = suspendingQueue .take ();
585+ this .processTask (keepingTask );
586+ } else {
587+ // Not leader: let subclass handle follower state (e.g. sync optimizer list from DB)
588+ onFollowerTick (syncInterval );
589+ }
537590 } catch (InterruptedException ignored ) {
538591 } catch (Throwable t ) {
539592 LOG .error ("{} has encountered a problem." , this .getClass ().getSimpleName (), t );
@@ -542,6 +595,12 @@ public void run() {
542595 }
543596
544597 protected abstract void processTask (T task ) throws Exception ;
598+
599+ protected void onFollowerTick (long syncInterval ) throws InterruptedException {
600+ Thread .sleep (syncInterval );
601+ }
602+
603+ protected void onBecomeLeader () {}
545604 }
546605
547606 private class OptimizerKeeper extends AbstractKeeper <OptimizerKeepingTask > {
@@ -575,6 +634,96 @@ protected void processTask(OptimizerKeepingTask keepingTask) {
575634 }
576635 }
577636
637+ @ Override
638+ protected void onFollowerTick (long syncInterval ) throws InterruptedException {
639+ loadOptimizersFromDatabase ();
640+ Thread .sleep (syncInterval );
641+ }
642+
643+ @ Override
644+ protected void onBecomeLeader () {
645+ LOG .info (
646+ "Became leader, starting heartbeat monitoring for {} inherited optimizers" ,
647+ authOptimizers .size ());
648+ // All optimizers in authOptimizers were loaded from DB by the follower sync loop.
649+ // Their touchTime reflects the latest DB-persisted heartbeat, which is the correct
650+ // baseline for the new leader's expiry detection.
651+ authOptimizers .values ().forEach (this ::keepInTouch );
652+ }
653+
654+ /**
655+ * Load optimizer information from database. This is used in master-slave mode for follower
656+ * nodes to sync optimizer state from database. This method performs incremental updates by
657+ * comparing database state with local authOptimizers, only adding new optimizers and removing
658+ * missing ones.
659+ */
660+ private void loadOptimizersFromDatabase () {
661+ try {
662+ List <OptimizerInstance > dbOptimizers =
663+ getAs (OptimizerMapper .class , OptimizerMapper ::selectAll );
664+
665+ Map <String , OptimizerInstance > dbOptimizersByToken = new HashMap <>();
666+ for (OptimizerInstance optimizer : dbOptimizers ) {
667+ String token = optimizer .getToken ();
668+ if (token != null ) {
669+ dbOptimizersByToken .put (token , optimizer );
670+ }
671+ }
672+
673+ Set <String > localTokens = new HashSet <>(authOptimizers .keySet ());
674+ Set <String > dbTokens = new HashSet <>(dbOptimizersByToken .keySet ());
675+ Set <String > tokensToAdd = new HashSet <>(dbTokens );
676+ tokensToAdd .removeAll (localTokens );
677+
678+ Set <String > tokensToRemove = new HashSet <>(localTokens );
679+ tokensToRemove .removeAll (dbTokens );
680+
681+ for (String token : tokensToAdd ) {
682+ OptimizerInstance optimizer = dbOptimizersByToken .get (token );
683+ if (optimizer != null ) {
684+ registerOptimizerWithoutPersist (optimizer );
685+ LOG .debug ("Added optimizer {} from database" , token );
686+ }
687+ }
688+
689+ for (String token : tokensToRemove ) {
690+ removeOptimizerFromLocal (token );
691+ LOG .debug ("Removed optimizer {} (not in database)" , token );
692+ }
693+
694+ LOG .debug (
695+ "Synced optimizers from database: total={}, added={}, removed={}, current={}" ,
696+ dbOptimizersByToken .size (),
697+ tokensToAdd .size (),
698+ tokensToRemove .size (),
699+ authOptimizers .size ());
700+ } catch (Exception e ) {
701+ LOG .error ("Failed to load optimizers from database" , e );
702+ }
703+ }
704+
705+ private void registerOptimizerWithoutPersist (OptimizerInstance optimizer ) {
706+ OptimizingQueue optimizingQueue = optimizingQueueByGroup .get (optimizer .getGroupName ());
707+ if (optimizingQueue == null ) {
708+ LOG .warn (
709+ "Cannot register optimizer {}: optimizing queue for group {} not found" ,
710+ optimizer .getToken (),
711+ optimizer .getGroupName ());
712+ return ;
713+ }
714+ optimizingQueue .addOptimizer (optimizer );
715+ authOptimizers .put (optimizer .getToken (), optimizer );
716+ optimizingQueueByToken .put (optimizer .getToken (), optimizingQueue );
717+ }
718+
719+ private void removeOptimizerFromLocal (String token ) {
720+ OptimizingQueue optimizingQueue = optimizingQueueByToken .remove (token );
721+ OptimizerInstance optimizer = authOptimizers .remove (token );
722+ if (optimizingQueue != null && optimizer != null ) {
723+ optimizingQueue .removeOptimizer (optimizer );
724+ }
725+ }
726+
578727 private void retryTask (TaskRuntime <?> task , OptimizingQueue queue ) {
579728 if (isTaskExecTimeout (task )) {
580729 LOG .warn (
0 commit comments