2323import org .apache .fluss .exception .TableNotExistException ;
2424import org .apache .fluss .metadata .TableInfo ;
2525import org .apache .fluss .metadata .TablePath ;
26+ import org .apache .fluss .metrics .MetricNames ;
27+ import org .apache .fluss .metrics .groups .MetricGroup ;
2628import org .apache .fluss .server .entity .LakeTieringTableInfo ;
29+ import org .apache .fluss .server .metrics .group .LakeTieringMetricGroup ;
2730import org .apache .fluss .server .utils .timer .DefaultTimer ;
2831import org .apache .fluss .server .utils .timer .Timer ;
2932import org .apache .fluss .server .utils .timer .TimerTask ;
@@ -126,25 +129,38 @@ public class LakeTableTieringManager implements AutoCloseable {
126129 // table_id -> the last timestamp of tiered lake snapshot
127130 private final Map <Long , Long > tableLastTieredTime ;
128131
132+ // table_id -> the last tiering duration in milliseconds
133+ private final Map <Long , Long > tableLastTieringDuration ;
134+
135+ // table_id -> the tiering failure count
136+ private final Map <Long , Long > tableTieringFailureCount ;
137+
129138 // the live tables that are tiering,
130139 // from table_id -> last heartbeat time by the tiering service
131140 private final Map <Long , Long > liveTieringTableIds ;
132141
142+ // global tiering failure count
143+ private long globalTieringFailureCount = 0 ;
144+
133145 private final Lock lock = new ReentrantLock ();
134146
135- public LakeTableTieringManager () {
147+ private final LakeTieringMetricGroup lakeTieringMetricGroup ;
148+
149+ public LakeTableTieringManager (LakeTieringMetricGroup lakeTieringMetricGroup ) {
136150 this (
137151 new DefaultTimer ("delay lake tiering" , 1_000 , 20 ),
138152 Executors .newSingleThreadScheduledExecutor (
139153 new ExecutorThreadFactory ("fluss-lake-tiering-timeout-checker" )),
140- SystemClock .getInstance ());
154+ SystemClock .getInstance (),
155+ lakeTieringMetricGroup );
141156 }
142157
143158 @ VisibleForTesting
144159 protected LakeTableTieringManager (
145160 Timer lakeTieringScheduleTimer ,
146161 ScheduledExecutorService lakeTieringServiceTimeoutChecker ,
147- Clock clock ) {
162+ Clock clock ,
163+ LakeTieringMetricGroup lakeTieringMetricGroup ) {
148164 this .lakeTieringScheduleTimer = lakeTieringScheduleTimer ;
149165 this .lakeTieringServiceTimeoutChecker = lakeTieringServiceTimeoutChecker ;
150166 this .clock = clock ;
@@ -159,6 +175,19 @@ protected LakeTableTieringManager(
159175 this ::checkTieringServiceTimeout , 0 , 15 , TimeUnit .SECONDS );
160176 this .tableTierEpoch = new HashMap <>();
161177 this .tableLastTieredTime = new HashMap <>();
178+ this .tableLastTieringDuration = new HashMap <>();
179+ this .tableTieringFailureCount = new HashMap <>();
180+ this .lakeTieringMetricGroup = lakeTieringMetricGroup ;
181+ registerMetrics ();
182+ }
183+
184+ private void registerMetrics () {
185+ lakeTieringMetricGroup .gauge (
186+ MetricNames .LAKE_TIERING_PENDING_TABLES_COUNT , pendingTieringTables ::size );
187+ lakeTieringMetricGroup .gauge (
188+ MetricNames .LAKE_TIERING_RUNNING_TABLES_COUNT , liveTieringTableIds ::size );
189+ lakeTieringMetricGroup .gauge (
190+ MetricNames .LAKE_TIERING_FAILURES_TOTAL , () -> globalTieringFailureCount );
162191 }
163192
164193 public void initWithLakeTables (List <Tuple2 <TableInfo , Long >> tableInfoWithTieredTime ) {
@@ -196,6 +225,11 @@ private void registerLakeTable(TableInfo tableInfo, long lastTieredTime) {
196225 tableId , tableInfo .getTableConfig ().getDataLakeFreshness ().toMillis ());
197226 tableLastTieredTime .put (tableId , lastTieredTime );
198227 tableTierEpoch .put (tableId , 0L );
228+ tableLastTieringDuration .put (tableId , 0L );
229+ tableTieringFailureCount .put (tableId , 0L );
230+
231+ // register table-level metrics
232+ registerTableMetrics (tableId , tableInfo .getTablePath ());
199233 }
200234
201235 private void scheduleTableTiering (long tableId ) {
@@ -211,13 +245,50 @@ private void scheduleTableTiering(long tableId) {
211245 lakeTieringScheduleTimer .add (new DelayedTiering (tableId , delayMs ));
212246 }
213247
248+ private void registerTableMetrics (long tableId , TablePath tablePath ) {
249+ // create table-level metric group
250+ MetricGroup tableMetricGroup =
251+ lakeTieringMetricGroup .addGroup (
252+ "table" , tablePath .getDatabaseName () + "." + tablePath .getTableName ());
253+
254+ // lastSuccessAgeMs: milliseconds since last successful tiering
255+ tableMetricGroup .gauge (
256+ MetricNames .LAKE_TIERING_LAST_SUCCESS_AGE_MS ,
257+ () -> {
258+ Long lastTiered = tableLastTieredTime .get (tableId );
259+ return lastTiered != null ? clock .milliseconds () - lastTiered : -1L ;
260+ });
261+
262+ // lastDurationMs: duration of last tiering job
263+ tableMetricGroup .gauge (
264+ MetricNames .LAKE_TIERING_LAST_DURATION_MS ,
265+ () -> tableLastTieringDuration .getOrDefault (tableId , 0L ));
266+
267+ // failuresTotal: total failure count for this table
268+ tableMetricGroup .gauge (
269+ MetricNames .LAKE_TIERING_TABLE_FAILURES_TOTAL ,
270+ () -> tableTieringFailureCount .getOrDefault (tableId , 0L ));
271+
272+ // state: current tiering state represented as ordinal value
273+ // (0=New, 1=Initialized, 2=Scheduled, 3=Pending, 4=Tiering, 5=Tiered, 6=Failed, -1=table
274+ // not exist)
275+ tableMetricGroup .gauge (
276+ MetricNames .LAKE_TIERING_TABLE_STATE ,
277+ () -> {
278+ TieringState state = tieringStates .get (tableId );
279+ return state != null ? state .ordinal () : -1 ;
280+ });
281+ }
282+
214283 public void removeLakeTable (long tableId ) {
215284 inLock (
216285 lock ,
217286 () -> {
218287 tablePaths .remove (tableId );
219288 tableLakeFreshness .remove (tableId );
220289 tableLastTieredTime .remove (tableId );
290+ tableLastTieringDuration .remove (tableId );
291+ tableTieringFailureCount .remove (tableId );
221292 tieringStates .remove (tableId );
222293 liveTieringTableIds .remove (tableId );
223294 tableTierEpoch .remove (tableId );
@@ -392,9 +463,17 @@ private void doHandleStateChange(long tableId, TieringState targetState) {
392463 break ;
393464 case Tiered :
394465 tableLastTieredTime .put (tableId , clock .milliseconds ());
395- liveTieringTableIds .remove (tableId );
466+ // calculate and record tiering duration
467+ Long startTime = liveTieringTableIds .remove (tableId );
468+ if (startTime != null ) {
469+ long duration = clock .milliseconds () - startTime ;
470+ tableLastTieringDuration .put (tableId , duration );
471+ }
396472 break ;
397473 case Failed :
474+ // increment failure counters
475+ tableTieringFailureCount .computeIfPresent (tableId , (t , v ) -> v + 1 );
476+ globalTieringFailureCount ++;
398477 liveTieringTableIds .remove (tableId );
399478 // do nothing
400479 break ;
@@ -480,7 +559,8 @@ private void advanceClock() throws InterruptedException {
480559 }
481560 }
482561
483- private enum TieringState {
562+ @ VisibleForTesting
563+ enum TieringState {
484564 // When a new lake table is created, the state will be New
485565 New {
486566 @ Override
@@ -537,4 +617,43 @@ public Set<TieringState> validPreviousStates() {
537617
538618 abstract Set <TieringState > validPreviousStates ();
539619 }
620+
621+ // ------------------------------------------------------------------------
622+ // Test-only methods
623+ // ------------------------------------------------------------------------
624+
625+ @ VisibleForTesting
626+ protected int getPendingTablesCount () {
627+ return pendingTieringTables .size ();
628+ }
629+
630+ @ VisibleForTesting
631+ protected int getRunningTablesCount () {
632+ return liveTieringTableIds .size ();
633+ }
634+
635+ @ VisibleForTesting
636+ protected long getGlobalFailureCount () {
637+ return globalTieringFailureCount ;
638+ }
639+
640+ @ VisibleForTesting
641+ protected Long getTableLastSuccessTime (long tableId ) {
642+ return tableLastTieredTime .get (tableId );
643+ }
644+
645+ @ VisibleForTesting
646+ protected Long getTableLastDuration (long tableId ) {
647+ return tableLastTieringDuration .get (tableId );
648+ }
649+
650+ @ VisibleForTesting
651+ protected Long getTableFailureCount (long tableId ) {
652+ return tableTieringFailureCount .get (tableId );
653+ }
654+
655+ @ VisibleForTesting
656+ protected TieringState getTableState (long tableId ) {
657+ return tieringStates .get (tableId );
658+ }
540659}
0 commit comments