4040import org .apache .fluss .rpc .messages .PbLakeTieringTableInfo ;
4141import org .apache .fluss .rpc .metrics .ClientMetricGroup ;
4242import org .apache .fluss .utils .MapUtils ;
43- import org .apache .fluss .utils .clock .Clock ;
44- import org .apache .fluss .utils .clock .SystemClock ;
4543
4644import org .apache .flink .api .connector .source .ReaderInfo ;
4745import org .apache .flink .api .connector .source .SourceEvent ;
6563import java .util .Set ;
6664import java .util .TreeSet ;
6765import java .util .concurrent .CompletableFuture ;
66+ import java .util .concurrent .Executors ;
67+ import java .util .concurrent .ScheduledExecutorService ;
6868import java .util .concurrent .TimeUnit ;
6969import java .util .stream .Collectors ;
7070
@@ -95,20 +95,17 @@ public class TieringSourceEnumerator
9595
9696 private final Configuration flussConf ;
9797 private final SplitEnumeratorContext <TieringSplit > context ;
98+ private final ScheduledExecutorService timerService ;
9899 private final SplitEnumeratorMetricGroup enumeratorMetricGroup ;
99100 private final long pollTieringTableIntervalMs ;
100- private final long tieringTableDurationMaxMs ;
101- private final long tieringTableDurationDetectIntervalMs ;
101+ private final long tieringTableMaxDurationMs ;
102102 private final List <TieringSplit > pendingSplits ;
103103 private final Set <Integer > readersAwaitingSplit ;
104104
105- private final Map <Long , Long > tieringTablesDeadline ;
106105 private final Map <Long , Long > tieringTableEpochs ;
107106 private final Map <Long , Long > failedTableEpochs ;
108107 private final Map <Long , Long > finishedTableEpochs ;
109108
110- private final Clock clock ;
111-
112109 // lazily instantiated
113110 private RpcClient rpcClient ;
114111 private CoordinatorGateway coordinatorGateway ;
@@ -125,37 +122,20 @@ public TieringSourceEnumerator(
125122 Configuration flussConf ,
126123 SplitEnumeratorContext <TieringSplit > context ,
127124 long pollTieringTableIntervalMs ,
128- long tieringTableDurationMaxMs ,
129- long tieringTableDurationDetectIntervalMs ) {
130- this (
131- flussConf ,
132- context ,
133- pollTieringTableIntervalMs ,
134- tieringTableDurationMaxMs ,
135- tieringTableDurationDetectIntervalMs ,
136- SystemClock .getInstance ());
137- }
138-
139- public TieringSourceEnumerator (
140- Configuration flussConf ,
141- SplitEnumeratorContext <TieringSplit > context ,
142- long pollTieringTableIntervalMs ,
143- long tieringTableDurationMaxMs ,
144- long tieringTableDurationDetectIntervalMs ,
145- Clock clock ) {
125+ long tieringTableMaxDurationMs ) {
146126 this .flussConf = flussConf ;
147127 this .context = context ;
128+ this .timerService =
129+ Executors .newSingleThreadScheduledExecutor (
130+ r -> new Thread (r , "Tiering-Timer-Thread" ));
148131 this .enumeratorMetricGroup = context .metricGroup ();
149132 this .pollTieringTableIntervalMs = pollTieringTableIntervalMs ;
150- this .tieringTableDurationMaxMs = tieringTableDurationMaxMs ;
151- this .tieringTableDurationDetectIntervalMs = tieringTableDurationDetectIntervalMs ;
133+ this .tieringTableMaxDurationMs = tieringTableMaxDurationMs ;
152134 this .pendingSplits = Collections .synchronizedList (new ArrayList <>());
153135 this .readersAwaitingSplit = Collections .synchronizedSet (new TreeSet <>());
154136 this .tieringTableEpochs = MapUtils .newConcurrentHashMap ();
155137 this .finishedTableEpochs = MapUtils .newConcurrentHashMap ();
156138 this .failedTableEpochs = MapUtils .newConcurrentHashMap ();
157- this .tieringTablesDeadline = MapUtils .newConcurrentHashMap ();
158- this .clock = clock ;
159139 }
160140
161141 @ Override
@@ -191,12 +171,6 @@ public void start() {
191171 this ::generateAndAssignSplits ,
192172 0 ,
193173 pollTieringTableIntervalMs );
194-
195- this .context .callAsync (
196- this ::checkTableReachMaxTieringDuration ,
197- this ::handleReachMaxTieringDurationTables ,
198- 0 ,
199- tieringTableDurationDetectIntervalMs );
200174 }
201175
202176 @ Override
@@ -282,7 +256,6 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
282256 } else {
283257 finishedTableEpochs .put (finishedTableId , tieringEpoch );
284258 }
285- tieringTablesDeadline .remove (finishedTableId );
286259 }
287260
288261 if (sourceEvent instanceof FailedTieringEvent ) {
@@ -301,7 +274,6 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) {
301274 } else {
302275 failedTableEpochs .put (failedTableId , tieringEpoch );
303276 }
304- tieringTablesDeadline .remove (failedTableId );
305277 }
306278
307279 if (!finishedTableEpochs .isEmpty () || !failedTableEpochs .isEmpty ()) {
@@ -327,29 +299,17 @@ private void handleSourceReaderFailOver() {
327299 }
328300 }
329301
330- private Set <Long > checkTableReachMaxTieringDuration () {
331- Set <Long > tieringReachMaxDurationTables = new HashSet <>();
332- long currentTime = clock .milliseconds ();
333- for (Map .Entry <Long , Long > tieringTableDeadline : tieringTablesDeadline .entrySet ()) {
334- long tableId = tieringTableDeadline .getKey ();
335- long deadline = tieringTableDeadline .getValue ();
336- if (deadline < currentTime ) {
337- tieringReachMaxDurationTables .add (tableId );
338- }
339- }
340- return tieringReachMaxDurationTables ;
341- }
342-
343- private void handleReachMaxTieringDurationTables (
344- Set <Long > tieringReachMaxDurationTables , Throwable throwable ) {
345- if (throwable != null ) {
346- LOG .error ("Fail to check tiering timeout tables." , throwable );
347- return ;
348- }
302+ @ VisibleForTesting
303+ protected void handleTableTieringReachMaxDuration (long tableId , long tieringEpoch ) {
304+ Long currentEpoch = tieringTableEpochs .get (tableId );
305+ if (currentEpoch != null && currentEpoch .equals (tieringEpoch )) {
306+ LOG .info (
307+ "Table {} reached max duration ({}ms). Force completing." ,
308+ tableId ,
309+ tieringTableMaxDurationMs );
349310
350- for (Long reachMaxDurationTable : tieringReachMaxDurationTables ) {
351311 for (TieringSplit tieringSplit : pendingSplits ) {
352- if (tieringSplit .getTableBucket ().getTableId () == reachMaxDurationTable ) {
312+ if (tieringSplit .getTableBucket ().getTableId () == tableId ) {
353313 // mark this tiering split to skip the current round since the tiering for
354314 // this table has timed out, so the tiering source reader can skip them directly
355315 tieringSplit .skipCurrentRound ();
@@ -361,14 +321,11 @@ private void handleReachMaxTieringDurationTables(
361321 }
362322 }
363323
364- LOG .info ("Found the table {} reach max tiering duration." , reachMaxDurationTable );
365-
366324 // broadcast the tiering reach max duration event to all readers,
367325 // we broadcast all for simplicity
368326 Set <Integer > readers = new HashSet <>(context .registeredReaders ().keySet ());
369327 for (int reader : readers ) {
370- context .sendEventToSourceReader (
371- reader , new TieringReachMaxDurationEvent (reachMaxDurationTable ));
328+ context .sendEventToSourceReader (reader , new TieringReachMaxDurationEvent (tableId ));
372329 }
373330 }
374331 }
@@ -478,8 +435,15 @@ private void generateTieringSplits(Tuple3<Long, Long, TablePath> tieringTable)
478435 } else {
479436 tieringTableEpochs .put (tieringTable .f0 , tieringTable .f1 );
480437 pendingSplits .addAll (tieringSplits );
481- tieringTablesDeadline .put (
482- tieringTable .f0 , clock .milliseconds () + tieringTableDurationMaxMs );
438+
439+ timerService .schedule (
440+ () ->
441+ context .runInCoordinatorThread (
442+ () ->
443+ handleTableTieringReachMaxDuration (
444+ tieringTable .f0 , tieringTable .f1 )),
445+ tieringTableMaxDurationMs ,
446+ TimeUnit .MILLISECONDS );
483447 }
484448 } catch (Exception e ) {
485449 LOG .warn ("Fail to generate Tiering splits for table {}." , tieringTable .f2 , e );
@@ -503,6 +467,7 @@ public TieringSourceEnumeratorState snapshotState(long checkpointId) throws Exce
503467 @ Override
504468 public void close () throws IOException {
505469 closed = true ;
470+ timerService .shutdownNow ();
506471 if (rpcClient != null ) {
507472 failedTableEpochs .putAll (tieringTableEpochs );
508473 tieringTableEpochs .clear ();
0 commit comments