diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/DefaultTableMaintainerContext.java b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/DefaultTableMaintainerContext.java
index ca47e22522..77ba7903ad 100644
--- a/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/DefaultTableMaintainerContext.java
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/optimizing/maintainer/DefaultTableMaintainerContext.java
@@ -23,7 +23,6 @@
import org.apache.amoro.maintainer.OptimizingInfo;
import org.apache.amoro.maintainer.TableMaintainerContext;
import org.apache.amoro.server.table.DefaultTableRuntime;
-import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics;
import org.apache.amoro.server.utils.HiveLocationUtil;
import org.apache.amoro.table.MixedTable;
@@ -56,18 +55,11 @@ public TableConfiguration getTableConfiguration() {
@Override
public MaintainerMetrics getMetrics() {
- TableOrphanFilesCleaningMetrics metrics = tableRuntime.getOrphanFilesCleaningMetrics();
- return new MaintainerMetrics() {
- @Override
- public void recordOrphanDataFilesCleaned(int expected, int cleaned) {
- metrics.completeOrphanDataFiles(expected, cleaned);
- }
-
- @Override
- public void recordOrphanMetadataFilesCleaned(int expected, int cleaned) {
- metrics.completeOrphanMetadataFiles(expected, cleaned);
- }
- };
+ // Return the full TableMaintainerMetricsImpl directly
+ // This provides access to all maintainer metrics including orphan files cleaning,
+ // dangling delete files cleaning, snapshot expiration, data expiration, tag creation,
+ // and partition expiration.
+ return tableRuntime.getMaintainerMetrics();
}
@Override
diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMaintainerMetrics.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMaintainerMetrics.java
new file mode 100644
index 0000000000..5f5efe94b1
--- /dev/null
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMaintainerMetrics.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.server.table;
+
+import static org.apache.amoro.metrics.MetricDefine.defineCounter;
+import static org.apache.amoro.metrics.MetricDefine.defineGauge;
+
+import org.apache.amoro.ServerTableIdentifier;
+import org.apache.amoro.maintainer.MaintainerMetrics;
+import org.apache.amoro.metrics.MetricDefine;
+
+/**
+ * Abstract base class for table maintenance operation metrics
+ *
+ *
Responsibilities:
+ *
+ *
+ * - Define all MetricDefine constants
+ *
- Provide template methods for metrics registration
+ *
- Handle tags (catalog, database, table, table_format, operation_type)
+ *
+ *
+ * Note: All metrics include the table_format tag to distinguish Iceberg and Paimon tables
+ */
+public abstract class AbstractTableMaintainerMetrics extends AbstractTableMetrics
+ implements MaintainerMetrics {
+
+ /** Table format constant: Iceberg native table */
+ protected static final String TABLE_FORMAT_ICEBERG = "iceberg";
+
+ /** Table format constant: Paimon native table */
+ protected static final String TABLE_FORMAT_PAIMON = "paimon";
+
+ /** Table format constant: Mixed table (based on Iceberg) */
+ protected static final String TABLE_FORMAT_MIXED_ICEBERG = "mixed_iceberg";
+
+ /** Table format constant: Hive table */
+ protected static final String TABLE_FORMAT_HIVE = "hive";
+
+ // ========== Orphan Files Related MetricDefine ==========
+
+ /** Count of orphan data files cleaned */
+ public static final MetricDefine TABLE_ORPHAN_DATA_FILES_CLEANED_COUNT =
+ defineCounter("table_orphan_data_files_cleaned_count")
+ .withDescription("Count of orphan data files cleaned")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Expected count of orphan data files to clean */
+ public static final MetricDefine TABLE_ORPHAN_DATA_FILES_CLEANED_EXPECTED_COUNT =
+ defineCounter("table_orphan_data_files_cleaned_expected_count")
+ .withDescription("Expected count of orphan data files to clean")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Count of orphan metadata files cleaned */
+ public static final MetricDefine TABLE_ORPHAN_METADATA_FILES_CLEANED_COUNT =
+ defineCounter("table_orphan_metadata_files_cleaned_count")
+ .withDescription("Count of orphan metadata files cleaned")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Expected count of orphan metadata files to clean */
+ public static final MetricDefine TABLE_ORPHAN_METADATA_FILES_CLEANED_EXPECTED_COUNT =
+ defineCounter("table_orphan_metadata_files_cleaned_expected_count")
+ .withDescription("Expected count of orphan metadata files to clean")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of orphan files cleaning operation (milliseconds) */
+ public static final MetricDefine TABLE_ORPHAN_FILES_CLEANING_DURATION =
+ defineGauge("table_orphan_files_cleaning_duration_millis")
+ .withDescription("Duration of orphan files cleaning operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== Dangling Delete Files Related MetricDefine (Iceberg) ==========
+
+ /** Count of dangling delete files cleaned */
+ public static final MetricDefine TABLE_DANGLING_DELETE_FILES_CLEANED_COUNT =
+ defineCounter("table_dangling_delete_files_cleaned_count")
+ .withDescription("Count of dangling delete files cleaned")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of dangling delete files cleaning operation (milliseconds) */
+ public static final MetricDefine TABLE_DANGLING_DELETE_FILES_CLEANING_DURATION =
+ defineGauge("table_dangling_delete_files_cleaning_duration_millis")
+ .withDescription("Duration of dangling delete files cleaning operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== Snapshot Expiration Related MetricDefine ==========
+
+ /** Count of snapshots expired */
+ public static final MetricDefine TABLE_SNAPSHOTS_EXPIRED_COUNT =
+ defineCounter("table_snapshots_expired_count")
+ .withDescription("Count of snapshots expired")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Count of data files deleted during snapshot expiration */
+ public static final MetricDefine TABLE_SNAPSHOTS_EXPIRED_DATA_FILES_DELETED =
+ defineCounter("table_snapshots_expired_data_files_deleted")
+ .withDescription("Count of data files deleted during snapshot expiration")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of snapshot expiration operation (milliseconds) */
+ public static final MetricDefine TABLE_SNAPSHOTS_EXPIRATION_DURATION =
+ defineGauge("table_snapshots_expiration_duration_millis")
+ .withDescription("Duration of snapshot expiration operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== Data Expiration Related MetricDefine (Iceberg) ==========
+
+ /** Count of data files expired */
+ public static final MetricDefine TABLE_DATA_EXPIRED_DATA_FILES_COUNT =
+ defineCounter("table_data_expired_data_files_count")
+ .withDescription("Count of data files expired")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Count of delete files expired */
+ public static final MetricDefine TABLE_DATA_EXPIRED_DELETE_FILES_COUNT =
+ defineCounter("table_data_expired_delete_files_count")
+ .withDescription("Count of delete files expired")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of data expiration operation (milliseconds) */
+ public static final MetricDefine TABLE_DATA_EXPIRATION_DURATION =
+ defineGauge("table_data_expiration_duration_millis")
+ .withDescription("Duration of data expiration operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== Tag Creation Related MetricDefine (Iceberg) ==========
+
+ /** Count of tags created */
+ public static final MetricDefine TABLE_TAGS_CREATED_COUNT =
+ defineCounter("table_tags_created_count")
+ .withDescription("Count of tags created")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of tag creation operation (milliseconds) */
+ public static final MetricDefine TABLE_TAG_CREATION_DURATION =
+ defineGauge("table_tag_creation_duration_millis")
+ .withDescription("Duration of tag creation operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== Partition Expiration Related MetricDefine (Paimon) ==========
+
+ /** Count of partitions expired */
+ public static final MetricDefine TABLE_PARTITIONS_EXPIRED_COUNT =
+ defineCounter("table_partitions_expired_count")
+ .withDescription("Count of partitions expired")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Count of files expired during partition expiration */
+ public static final MetricDefine TABLE_PARTITIONS_EXPIRED_FILES_COUNT =
+ defineCounter("table_partitions_expired_files_count")
+ .withDescription("Count of files expired during partition expiration")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ /** Duration of partition expiration operation (milliseconds) */
+ public static final MetricDefine TABLE_PARTITION_EXPIRATION_DURATION =
+ defineGauge("table_partition_expiration_duration_millis")
+ .withDescription("Duration of partition expiration operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format")
+ .build();
+
+ // ========== General Operation Status Related MetricDefine ==========
+
+ /** Count of successful maintainer operations */
+ public static final MetricDefine TABLE_MAINTAINER_OPERATION_SUCCESS_COUNT =
+ defineCounter("table_maintainer_operation_success_count")
+ .withDescription("Count of successful maintainer operations")
+ .withTags("catalog", "database", "table", "table_format", "operation_type")
+ .build();
+
+ /** Count of failed maintainer operations */
+ public static final MetricDefine TABLE_MAINTAINER_OPERATION_FAILURE_COUNT =
+ defineCounter("table_maintainer_operation_failure_count")
+ .withDescription("Count of failed maintainer operations")
+ .withTags("catalog", "database", "table", "table_format", "operation_type")
+ .build();
+
+ /** Duration of maintainer operation (milliseconds) */
+ public static final MetricDefine TABLE_MAINTAINER_OPERATION_DURATION =
+ defineGauge("table_maintainer_operation_duration_millis")
+ .withDescription("Duration of maintainer operation in milliseconds")
+ .withTags("catalog", "database", "table", "table_format", "operation_type")
+ .build();
+
+ /** Table format type */
+ protected final String tableFormat;
+
+ /**
+ * Constructor
+ *
+ * @param identifier Table identifier
+ * @param tableFormat Table format type (iceberg/paimon/mixed_iceberg/hive)
+ */
+ protected AbstractTableMaintainerMetrics(ServerTableIdentifier identifier, String tableFormat) {
+ super(identifier);
+ this.tableFormat = tableFormat;
+ }
+}
diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMetrics.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMetrics.java
index 74e52f0b7e..c1a14adb08 100644
--- a/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMetrics.java
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/AbstractTableMetrics.java
@@ -37,6 +37,15 @@ protected AbstractTableMetrics(ServerTableIdentifier identifier) {
this.identifier = identifier;
}
+ /**
+ * Get the table identifier.
+ *
+ * @return ServerTableIdentifier
+ */
+ public ServerTableIdentifier getIdentifier() {
+ return identifier;
+ }
+
protected void registerMetric(MetricRegistry registry, MetricDefine define, Metric metric) {
MetricKey key =
registry.register(
diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java
index c1506218d2..534d6192d6 100644
--- a/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/DefaultTableRuntime.java
@@ -21,6 +21,7 @@
import org.apache.amoro.Action;
import org.apache.amoro.AmoroTable;
import org.apache.amoro.SupportsProcessPlugins;
+import org.apache.amoro.TableFormat;
import org.apache.amoro.TableRuntime;
import org.apache.amoro.api.BlockableOperation;
import org.apache.amoro.config.OptimizingConfig;
@@ -97,7 +98,7 @@ public class DefaultTableRuntime extends AbstractTableRuntime
private final Map processContainerMap = Maps.newConcurrentMap();
private final TableOptimizingMetrics optimizingMetrics;
- private final TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics;
+ private final TableMaintainerMetricsImpl maintainerMetrics;
private final TableSummaryMetrics tableSummaryMetrics;
private volatile long lastPlanTime;
private volatile OptimizingProcess optimizingProcess;
@@ -107,11 +108,22 @@ public DefaultTableRuntime(TableRuntimeStore store) {
super(store);
this.optimizingMetrics =
new TableOptimizingMetrics(store.getTableIdentifier(), store.getGroupName());
- this.orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(store.getTableIdentifier());
+ this.maintainerMetrics =
+ new TableMaintainerMetricsImpl(store.getTableIdentifier(), getTableFormatString(store));
this.tableSummaryMetrics = new TableSummaryMetrics(store.getTableIdentifier());
}
+ /**
+ * Get the table format string for metrics.
+ *
+ * @param store TableRuntimeStore
+ * @return lowercase format string (iceberg/paimon/mixed_iceberg/hive)
+ */
+ private String getTableFormatString(TableRuntimeStore store) {
+ TableFormat format = store.getTableIdentifier().getFormat();
+ return format.name().toLowerCase();
+ }
+
public void recover(OptimizingProcess optimizingProcess) {
if (!getOptimizingStatus().isProcessing()
|| !Objects.equals(optimizingProcess.getProcessId(), getProcessId())) {
@@ -124,7 +136,7 @@ public void recover(OptimizingProcess optimizingProcess) {
public void registerMetric(MetricRegistry metricRegistry) {
// TODO: extract method to interface.
this.optimizingMetrics.register(metricRegistry);
- this.orphanFilesCleaningMetrics.register(metricRegistry);
+ this.maintainerMetrics.registerMetrics(metricRegistry);
this.tableSummaryMetrics.register(metricRegistry);
}
@@ -161,8 +173,13 @@ public List getProcessStates(Action action) {
return processContainerMap.get(action).getProcessStates();
}
- public TableOrphanFilesCleaningMetrics getOrphanFilesCleaningMetrics() {
- return orphanFilesCleaningMetrics;
+ /**
+ * Get the maintainer metrics implementation.
+ *
+ * @return TableMaintainerMetricsImpl instance
+ */
+ public TableMaintainerMetricsImpl getMaintainerMetrics() {
+ return maintainerMetrics;
}
public long getCurrentSnapshotId() {
@@ -472,7 +489,7 @@ public void beginCommitting() {
@Override
public void unregisterMetric() {
tableSummaryMetrics.unregister();
- orphanFilesCleaningMetrics.unregister();
+ maintainerMetrics.unregister();
optimizingMetrics.unregister();
}
diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableMaintainerMetricsImpl.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableMaintainerMetricsImpl.java
new file mode 100644
index 0000000000..7c9757f139
--- /dev/null
+++ b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableMaintainerMetricsImpl.java
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.server.table;
+
+import org.apache.amoro.ServerTableIdentifier;
+import org.apache.amoro.maintainer.MaintainerOperationType;
+import org.apache.amoro.metrics.Counter;
+import org.apache.amoro.metrics.Gauge;
+import org.apache.amoro.metrics.Metric;
+import org.apache.amoro.metrics.MetricDefine;
+import org.apache.amoro.metrics.MetricKey;
+import org.apache.amoro.metrics.MetricRegistry;
+import org.apache.amoro.shade.guava32.com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Concrete implementation of table maintenance operation metrics
+ *
+ * Design notes:
+ *
+ *
+ * - Each maintenance operation corresponds to a set of metrics (Counter + Gauge)
+ *
- Counter is used for cumulative counting (e.g., file count)
+ *
- Gauge is used to record current state (e.g., execution duration)
+ *
- Supports thread safety in concurrent scenarios
+ *
- Includes table_format tag to distinguish Iceberg and Paimon tables
+ *
+ */
+public class TableMaintainerMetricsImpl extends AbstractTableMaintainerMetrics {
+
+ // ========== Orphan Files Metrics ==========
+ private final Counter orphanDataFilesCount = new Counter();
+ private final Counter orphanDataFilesExpectedCount = new Counter();
+ private final Counter orphanMetadataFilesCount = new Counter();
+ private final Counter orphanMetadataFilesExpectedCount = new Counter();
+ private final LastOperationDurationGauge orphanFilesCleaningDuration =
+ new LastOperationDurationGauge();
+
+ // ========== Dangling Delete Files Metrics ==========
+ private final Counter danglingDeleteFilesCount = new Counter();
+ private final LastOperationDurationGauge danglingDeleteFilesCleaningDuration =
+ new LastOperationDurationGauge();
+
+ // ========== Snapshot Expiration Metrics ==========
+ private final Counter snapshotsExpiredCount = new Counter();
+ private final Counter snapshotsExpiredDataFilesDeleted = new Counter();
+ private final LastOperationDurationGauge snapshotsExpirationDuration =
+ new LastOperationDurationGauge();
+
+ // ========== Data Expiration Metrics ==========
+ private final Counter dataExpiredDataFilesCount = new Counter();
+ private final Counter dataExpiredDeleteFilesCount = new Counter();
+ private final LastOperationDurationGauge dataExpirationDuration =
+ new LastOperationDurationGauge();
+
+ // ========== Tag Creation Metrics ==========
+ private final Counter tagsCreatedCount = new Counter();
+ private final LastOperationDurationGauge tagCreationDuration = new LastOperationDurationGauge();
+
+ // ========== Partition Expiration Metrics ==========
+ private final Counter partitionsExpiredCount = new Counter();
+ private final Counter partitionsExpiredFilesCount = new Counter();
+ private final LastOperationDurationGauge partitionExpirationDuration =
+ new LastOperationDurationGauge();
+
+ // ========== Operation Status Metrics ==========
+ private final ConcurrentHashMap successCounters =
+ new ConcurrentHashMap<>();
+ private final ConcurrentHashMap failureCounters =
+ new ConcurrentHashMap<>();
+ private final ConcurrentHashMap durationGauges =
+ new ConcurrentHashMap<>();
+
+ /**
+ * Constructor
+ *
+ * @param identifier Table identifier
+ * @param tableFormat Table format type (iceberg/paimon/mixed_iceberg/hive)
+ */
+ public TableMaintainerMetricsImpl(ServerTableIdentifier identifier, String tableFormat) {
+ super(identifier, tableFormat);
+ // Initialize operation type counters and gauges
+ for (MaintainerOperationType type : MaintainerOperationType.values()) {
+ successCounters.put(type, new Counter());
+ failureCounters.put(type, new Counter());
+ durationGauges.put(type, new OperationDurationGauge());
+ }
+ }
+
+ @Override
+ public void registerMetrics(MetricRegistry registry) {
+ if (globalRegistry != null) {
+ return;
+ }
+
+ // Build base tags (including table_format)
+ Map baseTags =
+ ImmutableMap.of(
+ "catalog",
+ identifier.getCatalog(),
+ "database",
+ identifier.getDatabase(),
+ "table",
+ identifier.getTableName(),
+ "table_format",
+ tableFormat);
+
+ // Orphan files
+ registerMetricWithTags(
+ registry, TABLE_ORPHAN_DATA_FILES_CLEANED_COUNT, orphanDataFilesCount, baseTags);
+ registerMetricWithTags(
+ registry,
+ TABLE_ORPHAN_DATA_FILES_CLEANED_EXPECTED_COUNT,
+ orphanDataFilesExpectedCount,
+ baseTags);
+ registerMetricWithTags(
+ registry, TABLE_ORPHAN_METADATA_FILES_CLEANED_COUNT, orphanMetadataFilesCount, baseTags);
+ registerMetricWithTags(
+ registry,
+ TABLE_ORPHAN_METADATA_FILES_CLEANED_EXPECTED_COUNT,
+ orphanMetadataFilesExpectedCount,
+ baseTags);
+ registerMetricWithTags(
+ registry, TABLE_ORPHAN_FILES_CLEANING_DURATION, orphanFilesCleaningDuration, baseTags);
+
+ // Dangling delete files
+ registerMetricWithTags(
+ registry, TABLE_DANGLING_DELETE_FILES_CLEANED_COUNT, danglingDeleteFilesCount, baseTags);
+ registerMetricWithTags(
+ registry,
+ TABLE_DANGLING_DELETE_FILES_CLEANING_DURATION,
+ danglingDeleteFilesCleaningDuration,
+ baseTags);
+
+ // Snapshot expiration
+ registerMetricWithTags(
+ registry, TABLE_SNAPSHOTS_EXPIRED_COUNT, snapshotsExpiredCount, baseTags);
+ registerMetricWithTags(
+ registry,
+ TABLE_SNAPSHOTS_EXPIRED_DATA_FILES_DELETED,
+ snapshotsExpiredDataFilesDeleted,
+ baseTags);
+ registerMetricWithTags(
+ registry, TABLE_SNAPSHOTS_EXPIRATION_DURATION, snapshotsExpirationDuration, baseTags);
+
+ // Data expiration
+ registerMetricWithTags(
+ registry, TABLE_DATA_EXPIRED_DATA_FILES_COUNT, dataExpiredDataFilesCount, baseTags);
+ registerMetricWithTags(
+ registry, TABLE_DATA_EXPIRED_DELETE_FILES_COUNT, dataExpiredDeleteFilesCount, baseTags);
+ registerMetricWithTags(
+ registry, TABLE_DATA_EXPIRATION_DURATION, dataExpirationDuration, baseTags);
+
+ // Tag creation
+ registerMetricWithTags(registry, TABLE_TAGS_CREATED_COUNT, tagsCreatedCount, baseTags);
+ registerMetricWithTags(registry, TABLE_TAG_CREATION_DURATION, tagCreationDuration, baseTags);
+
+ // Partition expiration
+ registerMetricWithTags(
+ registry, TABLE_PARTITIONS_EXPIRED_COUNT, partitionsExpiredCount, baseTags);
+ registerMetricWithTags(
+ registry, TABLE_PARTITIONS_EXPIRED_FILES_COUNT, partitionsExpiredFilesCount, baseTags);
+ registerMetricWithTags(
+ registry, TABLE_PARTITION_EXPIRATION_DURATION, partitionExpirationDuration, baseTags);
+
+ // Operation status (needs to include operation_type tag)
+ for (MaintainerOperationType type : MaintainerOperationType.values()) {
+ Map operationTags =
+ ImmutableMap.builder()
+ .putAll(baseTags)
+ .put("operation_type", type.getMetricName())
+ .build();
+ registerMetricWithTags(
+ registry,
+ TABLE_MAINTAINER_OPERATION_SUCCESS_COUNT,
+ successCounters.get(type),
+ operationTags);
+ registerMetricWithTags(
+ registry,
+ TABLE_MAINTAINER_OPERATION_FAILURE_COUNT,
+ failureCounters.get(type),
+ operationTags);
+ registerMetricWithTags(
+ registry, TABLE_MAINTAINER_OPERATION_DURATION, durationGauges.get(type), operationTags);
+ }
+
+ globalRegistry = registry;
+ }
+
+ /**
+ * Register metric with specified tags
+ *
+ * @param registry MetricRegistry
+ * @param define MetricDefine
+ * @param metric Metric instance
+ * @param tags Tags
+ */
+ private void registerMetricWithTags(
+ MetricRegistry registry, MetricDefine define, Metric metric, Map tags) {
+ MetricKey key = registry.register(define, tags, metric);
+ registeredMetricKeys.add(key);
+ }
+
+ // ========== MaintainerMetrics Interface Implementation ==========
+
+ @Override
+ public void recordOrphanDataFilesCleaned(int expected, int cleaned) {
+ orphanDataFilesExpectedCount.inc(expected);
+ orphanDataFilesCount.inc(cleaned);
+ }
+
+ @Override
+ public void recordOrphanMetadataFilesCleaned(int expected, int cleaned) {
+ orphanMetadataFilesExpectedCount.inc(expected);
+ orphanMetadataFilesCount.inc(cleaned);
+ }
+
+ @Override
+ public void recordDanglingDeleteFilesCleaned(int cleaned) {
+ danglingDeleteFilesCount.inc(cleaned);
+ }
+
+ @Override
+ public void recordSnapshotsExpired(int snapshotCount, int dataFilesDeleted, long durationMillis) {
+ snapshotsExpiredCount.inc(snapshotCount);
+ snapshotsExpiredDataFilesDeleted.inc(dataFilesDeleted);
+ snapshotsExpirationDuration.setValue(durationMillis);
+ }
+
+ @Override
+ public void recordDataExpired(int dataFilesExpired, int deleteFilesExpired, long durationMillis) {
+ dataExpiredDataFilesCount.inc(dataFilesExpired);
+ dataExpiredDeleteFilesCount.inc(deleteFilesExpired);
+ dataExpirationDuration.setValue(durationMillis);
+ }
+
+ @Override
+ public void recordTagsCreated(int tagsCreated, long durationMillis) {
+ tagsCreatedCount.inc(tagsCreated);
+ tagCreationDuration.setValue(durationMillis);
+ }
+
+ @Override
+ public void recordPartitionsExpired(
+ int partitionsExpired, int filesExpired, long durationMillis) {
+ partitionsExpiredCount.inc(partitionsExpired);
+ partitionsExpiredFilesCount.inc(filesExpired);
+ partitionExpirationDuration.setValue(durationMillis);
+ }
+
+ @Override
+ public void recordOperationStart(MaintainerOperationType operationType) {
+ durationGauges.get(operationType).recordStart();
+ }
+
+ @Override
+ public void recordOperationSuccess(MaintainerOperationType operationType, long durationMillis) {
+ successCounters.get(operationType).inc();
+ durationGauges.get(operationType).setValue(durationMillis);
+ }
+
+ @Override
+ public void recordOperationFailure(
+ MaintainerOperationType operationType, long durationMillis, Throwable throwable) {
+ failureCounters.get(operationType).inc();
+ durationGauges.get(operationType).setValue(durationMillis);
+ }
+
+ // ========== Internal Helper Classes ==========
+
+ /** Gauge implementation for recording last operation duration */
+ private static class LastOperationDurationGauge implements Gauge {
+ private volatile long value = 0L;
+
+ public void setValue(long value) {
+ this.value = value;
+ }
+
+ @Override
+ public Long getValue() {
+ return value;
+ }
+ }
+
+ /** Gauge implementation for recording operation duration (supports recording start time) */
+ private static class OperationDurationGauge implements Gauge {
+ private volatile long value = 0L;
+ private volatile long startTime = 0L;
+
+ public void recordStart() {
+ this.startTime = System.currentTimeMillis();
+ }
+
+ public void setValue(long value) {
+ this.value = value;
+ }
+
+ @Override
+ public Long getValue() {
+ return value;
+ }
+ }
+}
diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableOrphanFilesCleaningMetrics.java b/amoro-ams/src/main/java/org/apache/amoro/server/table/TableOrphanFilesCleaningMetrics.java
deleted file mode 100644
index 481eb175b3..0000000000
--- a/amoro-ams/src/main/java/org/apache/amoro/server/table/TableOrphanFilesCleaningMetrics.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.amoro.server.table;
-
-import static org.apache.amoro.metrics.MetricDefine.defineCounter;
-
-import org.apache.amoro.ServerTableIdentifier;
-import org.apache.amoro.maintainer.MaintainerMetrics;
-import org.apache.amoro.metrics.Counter;
-import org.apache.amoro.metrics.MetricDefine;
-import org.apache.amoro.metrics.MetricRegistry;
-
-/** Table Orphan Files Cleaning metrics. */
-public class TableOrphanFilesCleaningMetrics extends AbstractTableMetrics
- implements MaintainerMetrics {
- private final Counter orphanDataFilesCount = new Counter();
- private final Counter expectedOrphanDataFilesCount = new Counter();
-
- private final Counter orphanMetadataFilesCount = new Counter();
- private final Counter expectedOrphanMetadataFilesCount = new Counter();
-
- public TableOrphanFilesCleaningMetrics(ServerTableIdentifier identifier) {
- super(identifier);
- }
-
- public static final MetricDefine TABLE_ORPHAN_CONTENT_FILE_CLEANING_COUNT =
- defineCounter("table_orphan_content_file_cleaning_count")
- .withDescription("Count of orphan content files cleaned in the table since ams started")
- .withTags("catalog", "database", "table")
- .build();
-
- public static final MetricDefine TABLE_ORPHAN_METADATA_FILE_CLEANING_COUNT =
- defineCounter("table_orphan_metadata_file_cleaning_count")
- .withDescription("Count of orphan metadata files cleaned in the table since ams started")
- .withTags("catalog", "database", "table")
- .build();
-
- public static final MetricDefine TABLE_EXPECTED_ORPHAN_CONTENT_FILE_CLEANING_COUNT =
- defineCounter("table_expected_orphan_content_file_cleaning_count")
- .withDescription(
- "Expected count of orphan content files cleaned in the table since ams started")
- .withTags("catalog", "database", "table")
- .build();
-
- public static final MetricDefine TABLE_EXPECTED_ORPHAN_METADATA_FILE_CLEANING_COUNT =
- defineCounter("table_expected_orphan_metadata_file_cleaning_count")
- .withDescription(
- "Expected count of orphan metadata files cleaned in the table since ams started")
- .withTags("catalog", "database", "table")
- .build();
-
- @Override
- public void registerMetrics(MetricRegistry registry) {
- if (globalRegistry == null) {
- registerMetric(registry, TABLE_ORPHAN_CONTENT_FILE_CLEANING_COUNT, orphanDataFilesCount);
- registerMetric(registry, TABLE_ORPHAN_METADATA_FILE_CLEANING_COUNT, orphanMetadataFilesCount);
- registerMetric(
- registry,
- TABLE_EXPECTED_ORPHAN_CONTENT_FILE_CLEANING_COUNT,
- expectedOrphanDataFilesCount);
- registerMetric(
- registry,
- TABLE_EXPECTED_ORPHAN_METADATA_FILE_CLEANING_COUNT,
- expectedOrphanMetadataFilesCount);
- globalRegistry = registry;
- }
- }
-
- public void completeOrphanDataFiles(int expected, int cleaned) {
- expectedOrphanDataFilesCount.inc(expected);
- orphanDataFilesCount.inc(cleaned);
- }
-
- public void completeOrphanMetadataFiles(int expected, int cleaned) {
- expectedOrphanMetadataFilesCount.inc(expected);
- orphanMetadataFilesCount.inc(cleaned);
- }
-
- @Override
- public void recordOrphanDataFilesCleaned(int expected, int cleaned) {
- completeOrphanDataFiles(expected, cleaned);
- }
-
- @Override
- public void recordOrphanMetadataFilesCleaned(int expected, int cleaned) {
- completeOrphanMetadataFiles(expected, cleaned);
- }
-}
diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java
index 1326bacdd7..f25d955f67 100644
--- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java
+++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileClean.java
@@ -22,14 +22,13 @@
import static org.apache.amoro.formats.iceberg.maintainer.IcebergTableMaintainer.FLINK_JOB_ID;
import org.apache.amoro.BasicTableTestHelper;
-import org.apache.amoro.ServerTableIdentifier;
import org.apache.amoro.TableFormat;
import org.apache.amoro.TableTestHelper;
import org.apache.amoro.catalog.BasicCatalogTestHelper;
import org.apache.amoro.catalog.CatalogTestHelper;
import org.apache.amoro.formats.iceberg.maintainer.MixedTableMaintainer;
+import org.apache.amoro.maintainer.MaintainerMetrics;
import org.apache.amoro.server.scheduler.inline.ExecutorTestBase;
-import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics;
import org.apache.amoro.table.TableIdentifier;
import org.apache.amoro.table.TableProperties;
import org.apache.amoro.table.UnkeyedTable;
@@ -110,23 +109,17 @@ public void orphanDataFileClean() throws IOException {
Assert.assertTrue(getMixedTable().io().exists(changeOrphanFilePath));
}
TableIdentifier tableIdentifier = getMixedTable().id();
- TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(
- ServerTableIdentifier.of(
- tableIdentifier.getCatalog(),
- tableIdentifier.getDatabase(),
- tableIdentifier.getTableName(),
- getTestFormat()));
+ MaintainerMetrics metrics = MaintainerMetrics.NOOP;
MixedTableMaintainer maintainer =
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()));
maintainer.cleanContentFiles(
System.currentTimeMillis()
- TableProperties.MIN_ORPHAN_FILE_EXISTING_TIME_DEFAULT * 60 * 1000,
- orphanFilesCleaningMetrics);
+ metrics);
maintainer.cleanMetadata(
System.currentTimeMillis()
- TableProperties.MIN_ORPHAN_FILE_EXISTING_TIME_DEFAULT * 60 * 1000,
- orphanFilesCleaningMetrics);
+ metrics);
Assert.assertTrue(getMixedTable().io().exists(baseOrphanFileDir));
Assert.assertTrue(getMixedTable().io().exists(baseOrphanFilePath));
@@ -135,8 +128,8 @@ public void orphanDataFileClean() throws IOException {
Assert.assertTrue(getMixedTable().io().exists(changeOrphanFilePath));
}
- maintainer.cleanContentFiles(System.currentTimeMillis(), orphanFilesCleaningMetrics);
- maintainer.cleanMetadata(System.currentTimeMillis(), orphanFilesCleaningMetrics);
+ maintainer.cleanContentFiles(System.currentTimeMillis(), metrics);
+ maintainer.cleanMetadata(System.currentTimeMillis(), metrics);
Assert.assertFalse(getMixedTable().io().exists(baseOrphanFileDir));
Assert.assertFalse(getMixedTable().io().exists(baseOrphanFilePath));
@@ -204,15 +197,8 @@ public void orphanMetadataFileClean() throws IOException {
MixedTableMaintainer maintainer =
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()));
- TableIdentifier tableIdentifier = getMixedTable().id();
- TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(
- ServerTableIdentifier.of(
- tableIdentifier.getCatalog(),
- tableIdentifier.getDatabase(),
- tableIdentifier.getTableName(),
- getTestFormat()));
- maintainer.cleanMetadata(System.currentTimeMillis(), orphanFilesCleaningMetrics);
+ MaintainerMetrics metrics = MaintainerMetrics.NOOP;
+ maintainer.cleanMetadata(System.currentTimeMillis(), metrics);
Assert.assertFalse(getMixedTable().io().exists(baseOrphanFilePath));
if (isKeyedTable()) {
@@ -298,16 +284,8 @@ public void notDeleteFlinkTemporaryFile() throws IOException {
MixedTableMaintainer tableMaintainer =
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()));
- TableIdentifier tableIdentifier = getMixedTable().id();
- TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(
- ServerTableIdentifier.of(
- tableIdentifier.getCatalog(),
- tableIdentifier.getDatabase(),
- tableIdentifier.getTableName(),
- getTestFormat()));
-
- tableMaintainer.cleanMetadata(System.currentTimeMillis(), orphanFilesCleaningMetrics);
+ MaintainerMetrics metrics = MaintainerMetrics.NOOP;
+ tableMaintainer.cleanMetadata(System.currentTimeMillis(), metrics);
Assert.assertFalse(getMixedTable().io().exists(baseOrphanFilePath));
if (isKeyedTable()) {
// files whose file name starts with flink.job-id should not be deleted
@@ -334,22 +312,15 @@ public void notDeleteStatisticsFile() {
StatisticsFile file3 =
commitStatisticsFile(unkeyedTable, unkeyedTable.location() + "/data/puffin/test3.puffin");
- TableIdentifier tableIdentifier = getMixedTable().id();
- TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(
- ServerTableIdentifier.of(
- tableIdentifier.getCatalog(),
- tableIdentifier.getDatabase(),
- tableIdentifier.getTableName(),
- getTestFormat()));
+ MaintainerMetrics metrics = MaintainerMetrics.NOOP;
Assert.assertTrue(unkeyedTable.io().exists(file1.path()));
Assert.assertTrue(unkeyedTable.io().exists(file2.path()));
Assert.assertTrue(unkeyedTable.io().exists(file3.path()));
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()))
- .cleanContentFiles(System.currentTimeMillis() + 1, orphanFilesCleaningMetrics);
+ .cleanContentFiles(System.currentTimeMillis() + 1, metrics);
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()))
- .cleanMetadata(System.currentTimeMillis() + 1, orphanFilesCleaningMetrics);
+ .cleanMetadata(System.currentTimeMillis() + 1, metrics);
Assert.assertTrue(unkeyedTable.io().exists(file1.path()));
Assert.assertTrue(unkeyedTable.io().exists(file2.path()));
Assert.assertTrue(unkeyedTable.io().exists(file3.path()));
diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileCleanHive.java b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileCleanHive.java
index 729999525c..50c238318f 100644
--- a/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileCleanHive.java
+++ b/amoro-ams/src/test/java/org/apache/amoro/server/optimizing/maintainer/TestOrphanFileCleanHive.java
@@ -20,7 +20,6 @@
import static org.apache.amoro.formats.iceberg.maintainer.IcebergTableMaintainer.DATA_FOLDER_NAME;
-import org.apache.amoro.ServerTableIdentifier;
import org.apache.amoro.TableFormat;
import org.apache.amoro.TableTestHelper;
import org.apache.amoro.catalog.CatalogTestHelper;
@@ -29,8 +28,7 @@
import org.apache.amoro.hive.catalog.HiveCatalogTestHelper;
import org.apache.amoro.hive.catalog.HiveTableTestHelper;
import org.apache.amoro.hive.table.SupportHive;
-import org.apache.amoro.server.table.TableOrphanFilesCleaningMetrics;
-import org.apache.amoro.table.TableIdentifier;
+import org.apache.amoro.maintainer.MaintainerMetrics;
import org.apache.iceberg.io.OutputFile;
import org.junit.Assert;
import org.junit.ClassRule;
@@ -87,15 +85,8 @@ public void hiveLocationOrphanDataFileClean() throws IOException {
MixedTableMaintainer maintainer =
new MixedTableMaintainer(getMixedTable(), TestTableMaintainerContext.of(getMixedTable()));
- TableIdentifier tableIdentifier = getMixedTable().id();
- TableOrphanFilesCleaningMetrics orphanFilesCleaningMetrics =
- new TableOrphanFilesCleaningMetrics(
- ServerTableIdentifier.of(
- tableIdentifier.getCatalog(),
- tableIdentifier.getDatabase(),
- tableIdentifier.getTableName(),
- getTestFormat()));
- maintainer.cleanContentFiles(System.currentTimeMillis(), orphanFilesCleaningMetrics);
+ MaintainerMetrics metrics = MaintainerMetrics.NOOP;
+ maintainer.cleanContentFiles(System.currentTimeMillis(), metrics);
Assert.assertTrue(getMixedTable().io().exists(hiveOrphanFilePath));
}
}
diff --git a/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerMetrics.java b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerMetrics.java
index 420c61e3b0..0ad0fab728 100644
--- a/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerMetrics.java
+++ b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerMetrics.java
@@ -40,6 +40,73 @@ public interface MaintainerMetrics {
*/
void recordOrphanMetadataFilesCleaned(int expected, int cleaned);
+ /**
+ * Record dangling delete files cleaning result.
+ *
+ * @param cleaned number of files cleaned
+ */
+ void recordDanglingDeleteFilesCleaned(int cleaned);
+
+ /**
+ * Record snapshot expiration operation result.
+ *
+ * @param snapshotCount number of snapshots expired
+ * @param dataFilesDeleted number of data files deleted
+ * @param durationMillis operation duration in milliseconds
+ */
+ void recordSnapshotsExpired(int snapshotCount, int dataFilesDeleted, long durationMillis);
+
+ /**
+ * Record data expiration operation result.
+ *
+ * @param dataFilesExpired number of data files expired
+ * @param deleteFilesExpired number of delete files expired
+ * @param durationMillis operation duration in milliseconds
+ */
+ void recordDataExpired(int dataFilesExpired, int deleteFilesExpired, long durationMillis);
+
+ /**
+ * Record tag creation operation result.
+ *
+ * @param tagsCreated number of tags created
+ * @param durationMillis operation duration in milliseconds
+ */
+ void recordTagsCreated(int tagsCreated, long durationMillis);
+
+ /**
+ * Record partition expiration operation result.
+ *
+ * @param partitionsExpired number of partitions expired
+ * @param filesExpired number of files expired
+ * @param durationMillis operation duration in milliseconds
+ */
+ void recordPartitionsExpired(int partitionsExpired, int filesExpired, long durationMillis);
+
+ /**
+ * Record operation start.
+ *
+ * @param operationType operation type
+ */
+ void recordOperationStart(MaintainerOperationType operationType);
+
+ /**
+ * Record operation success completion.
+ *
+ * @param operationType operation type
+ * @param durationMillis operation duration in milliseconds
+ */
+ void recordOperationSuccess(MaintainerOperationType operationType, long durationMillis);
+
+ /**
+ * Record operation failure.
+ *
+ * @param operationType operation type
+ * @param durationMillis operation duration in milliseconds
+ * @param throwable exception information
+ */
+ void recordOperationFailure(
+ MaintainerOperationType operationType, long durationMillis, Throwable throwable);
+
/** No-op implementation that does nothing. */
MaintainerMetrics NOOP =
new MaintainerMetrics() {
@@ -48,5 +115,34 @@ public void recordOrphanDataFilesCleaned(int expected, int cleaned) {}
@Override
public void recordOrphanMetadataFilesCleaned(int expected, int cleaned) {}
+
+ @Override
+ public void recordDanglingDeleteFilesCleaned(int cleaned) {}
+
+ @Override
+ public void recordSnapshotsExpired(
+ int snapshotCount, int dataFilesDeleted, long durationMillis) {}
+
+ @Override
+ public void recordDataExpired(
+ int dataFilesExpired, int deleteFilesExpired, long durationMillis) {}
+
+ @Override
+ public void recordTagsCreated(int tagsCreated, long durationMillis) {}
+
+ @Override
+ public void recordPartitionsExpired(
+ int partitionsExpired, int filesExpired, long durationMillis) {}
+
+ @Override
+ public void recordOperationStart(MaintainerOperationType operationType) {}
+
+ @Override
+ public void recordOperationSuccess(
+ MaintainerOperationType operationType, long durationMillis) {}
+
+ @Override
+ public void recordOperationFailure(
+ MaintainerOperationType operationType, long durationMillis, Throwable throwable) {}
};
}
diff --git a/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationTemplate.java b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationTemplate.java
new file mode 100644
index 0000000000..b141d9833f
--- /dev/null
+++ b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationTemplate.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.maintainer;
+
+/**
+ * Template for executing maintainer operations with consistent metrics recording.
+ *
+ * This template ensures that all maintainer operations record metrics in a consistent way:
+ *
+ *
+ * - Record operation start
+ *
- Execute the operation
+ *
- Record operation success/failure with duration
+ *
+ *
+ * Usage example:
+ *
+ *
{@code
+ * MaintainerOperationTemplate template = new MaintainerOperationTemplate(metrics);
+ * template.execute(
+ * MaintainerOperationType.ORPHAN_FILES_CLEANING,
+ * () -> {
+ * // Operation logic here
+ * cleanOrphanFiles();
+ * }
+ * );
+ * }
+ */
+public class MaintainerOperationTemplate {
+
+ private final MaintainerMetrics metrics;
+
+ /**
+ * Creates a new operation template with the given metrics collector.
+ *
+ * @param metrics the metrics collector (can be null, will use NOOP in that case)
+ */
+ public MaintainerOperationTemplate(MaintainerMetrics metrics) {
+ this.metrics = metrics != null ? metrics : MaintainerMetrics.NOOP;
+ }
+
+ /**
+ * Executes a maintainer operation with metrics recording.
+ *
+ * This method will:
+ *
+ *
+ * - Record the operation start via {@link MaintainerMetrics#recordOperationStart}
+ *
- Execute the provided operation
+ *
- On success: record operation success via {@link MaintainerMetrics#recordOperationSuccess}
+ *
- On failure: record operation failure via {@link MaintainerMetrics#recordOperationFailure}
+ * and rethrow the exception
+ *
+ *
+ * @param operationType the type of operation being executed
+ * @param operation the operation to execute
+ * @return true if operation succeeded, false otherwise
+ * @throws Throwable if the operation throws an exception
+ */
+ public boolean execute(MaintainerOperationType operationType, MaintainerOperation operation)
+ throws Throwable {
+ long startTime = System.currentTimeMillis();
+ metrics.recordOperationStart(operationType);
+
+ try {
+ operation.execute();
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationSuccess(operationType, duration);
+ return true;
+ } catch (Throwable t) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationFailure(operationType, duration, t);
+ throw t;
+ }
+ }
+
+ /**
+ * Executes a maintainer operation with metrics recording and return result.
+ *
+ * This method will:
+ *
+ *
+ * - Record the operation start via {@link MaintainerMetrics#recordOperationStart}
+ *
- Execute the provided operation
+ *
- On success: record operation success via {@link MaintainerMetrics#recordOperationSuccess}
+ *
- On failure: record operation failure via {@link MaintainerMetrics#recordOperationFailure}
+ * and rethrow the exception
+ *
+ *
+ * @param operationType the type of operation being executed
+ * @param operation the operation to execute
+ * @param the result type
+ * @return the operation result
+ * @throws Throwable if the operation throws an exception
+ */
+ public T executeAndReturn(
+ MaintainerOperationType operationType, MaintainerOperationWithResult operation)
+ throws Throwable {
+ long startTime = System.currentTimeMillis();
+ metrics.recordOperationStart(operationType);
+
+ try {
+ T result = operation.execute();
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationSuccess(operationType, duration);
+ return result;
+ } catch (Throwable t) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationFailure(operationType, duration, t);
+ throw t;
+ }
+ }
+
+ /** Functional interface for maintainer operation without return value. */
+ @FunctionalInterface
+ public interface MaintainerOperation {
+ /** Executes the operation. */
+ void execute() throws Throwable;
+ }
+
+ /**
+ * Functional interface for maintainer operation with return value.
+ *
+ * @param the result type
+ */
+ @FunctionalInterface
+ public interface MaintainerOperationWithResult {
+ /**
+ * Executes the operation.
+ *
+ * @return the operation result
+ */
+ T execute() throws Throwable;
+ }
+}
diff --git a/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationType.java b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationType.java
new file mode 100644
index 0000000000..41baafe3a0
--- /dev/null
+++ b/amoro-common/src/main/java/org/apache/amoro/maintainer/MaintainerOperationType.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.maintainer;
+
+/**
+ * Enumeration of maintenance operation types
+ *
+ * Corresponds to various maintenance operations in TableMaintainer, used for metrics recording
+ * and classification
+ */
+public enum MaintainerOperationType {
+ /** Orphan files cleaning (including data files and metadata files) */
+ ORPHAN_FILES_CLEANING("orphan_files_cleaning"),
+
+ /** Dangling delete files cleaning (Iceberg specific) */
+ DANGLING_DELETE_FILES_CLEANING("dangling_delete_files_cleaning"),
+
+ /** Snapshot expiration */
+ SNAPSHOT_EXPIRATION("snapshot_expiration"),
+
+ /** Data expiration */
+ DATA_EXPIRATION("data_expiration"),
+
+ /** Tag creation */
+ TAG_CREATION("tag_creation"),
+
+ /** Partition expiration (Paimon specific) */
+ PARTITION_EXPIRATION("partition_expiration");
+
+ private final String metricName;
+
+ MaintainerOperationType(String metricName) {
+ this.metricName = metricName;
+ }
+
+ /**
+ * Get metric name
+ *
+ * @return Metric name
+ */
+ public String getMetricName() {
+ return metricName;
+ }
+}
diff --git a/amoro-common/src/test/java/org/apache/amoro/maintainer/TestMaintainerOperationTemplate.java b/amoro-common/src/test/java/org/apache/amoro/maintainer/TestMaintainerOperationTemplate.java
new file mode 100644
index 0000000000..67d5eddf99
--- /dev/null
+++ b/amoro-common/src/test/java/org/apache/amoro/maintainer/TestMaintainerOperationTemplate.java
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.maintainer;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InOrder;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class TestMaintainerOperationTemplate {
+
+ @Mock private MaintainerMetrics mockMetrics;
+
+ private MaintainerOperationTemplate template;
+
+ @BeforeEach
+ public void setUp() {
+ template = new MaintainerOperationTemplate(mockMetrics);
+ }
+
+ @Test
+ public void testSuccessfulOperation() throws Throwable {
+ // Execute operation
+ boolean result =
+ template.execute(
+ MaintainerOperationType.ORPHAN_FILES_CLEANING,
+ () -> {
+ // Operation logic here
+ });
+
+ // Verify result
+ assertTrue(result);
+
+ // Verify metrics recorded
+ InOrder inOrder = inOrder(mockMetrics);
+ inOrder.verify(mockMetrics).recordOperationStart(MaintainerOperationType.ORPHAN_FILES_CLEANING);
+ inOrder
+ .verify(mockMetrics)
+ .recordOperationSuccess(eq(MaintainerOperationType.ORPHAN_FILES_CLEANING), anyLong());
+ verify(mockMetrics, never())
+ .recordOperationFailure(
+ eq(MaintainerOperationType.ORPHAN_FILES_CLEANING), anyLong(), any());
+ }
+
+ @Test
+ public void testFailedOperation() throws Throwable {
+ RuntimeException exception = new RuntimeException("Test error");
+
+ // Execute operation and expect exception
+ assertThrows(
+ RuntimeException.class,
+ () ->
+ template.execute(
+ MaintainerOperationType.SNAPSHOT_EXPIRATION,
+ () -> {
+ throw exception;
+ }));
+
+ // Verify metrics recorded
+ InOrder inOrder = inOrder(mockMetrics);
+ inOrder.verify(mockMetrics).recordOperationStart(MaintainerOperationType.SNAPSHOT_EXPIRATION);
+ inOrder
+ .verify(mockMetrics)
+ .recordOperationFailure(
+ eq(MaintainerOperationType.SNAPSHOT_EXPIRATION), anyLong(), eq(exception));
+ verify(mockMetrics, never())
+ .recordOperationSuccess(eq(MaintainerOperationType.SNAPSHOT_EXPIRATION), anyLong());
+ }
+
+ @Test
+ public void testOperationWithResult() throws Throwable {
+ Integer expected = 42;
+
+ // Execute operation with result
+ Integer result =
+ template.executeAndReturn(
+ MaintainerOperationType.DATA_EXPIRATION,
+ () -> {
+ return expected;
+ });
+
+ // Verify result
+ assertEquals(expected, result);
+
+ // Verify metrics recorded
+ InOrder inOrder = inOrder(mockMetrics);
+ inOrder.verify(mockMetrics).recordOperationStart(MaintainerOperationType.DATA_EXPIRATION);
+ inOrder
+ .verify(mockMetrics)
+ .recordOperationSuccess(eq(MaintainerOperationType.DATA_EXPIRATION), anyLong());
+ }
+
+ @Test
+ public void testOperationWithResultFailure() throws Throwable {
+ IllegalStateException exception = new IllegalStateException("Test state error");
+
+ // Execute operation and expect exception
+ assertThrows(
+ IllegalStateException.class,
+ () ->
+ template.executeAndReturn(
+ MaintainerOperationType.TAG_CREATION,
+ () -> {
+ throw exception;
+ }));
+
+ // Verify metrics recorded
+ verify(mockMetrics).recordOperationStart(MaintainerOperationType.TAG_CREATION);
+ verify(mockMetrics)
+ .recordOperationFailure(eq(MaintainerOperationType.TAG_CREATION), anyLong(), eq(exception));
+ }
+
+ @Test
+ public void testNullMetricsUsesNoop() throws Throwable {
+ // Create template with null metrics
+ MaintainerOperationTemplate noopTemplate = new MaintainerOperationTemplate(null);
+
+ // Should not throw exception
+ assertDoesNotThrow(
+ () ->
+ noopTemplate.execute(
+ MaintainerOperationType.DANGLING_DELETE_FILES_CLEANING,
+ () -> {
+ // Operation logic here
+ }));
+
+ // Execute with result
+ Integer result =
+ assertDoesNotThrow(
+ () ->
+ noopTemplate.executeAndReturn(
+ MaintainerOperationType.PARTITION_EXPIRATION, () -> 123));
+ assertEquals(123, result);
+ }
+
+ @Test
+ public void testMultipleOperations() throws Throwable {
+ // Execute multiple operations
+ template.execute(
+ MaintainerOperationType.ORPHAN_FILES_CLEANING,
+ () -> {
+ // First operation
+ });
+
+ template.execute(
+ MaintainerOperationType.SNAPSHOT_EXPIRATION,
+ () -> {
+ // Second operation
+ });
+
+ // Verify both operations were recorded
+ verify(mockMetrics, times(1))
+ .recordOperationStart(MaintainerOperationType.ORPHAN_FILES_CLEANING);
+ verify(mockMetrics, times(1))
+ .recordOperationSuccess(eq(MaintainerOperationType.ORPHAN_FILES_CLEANING), anyLong());
+ verify(mockMetrics, times(1)).recordOperationStart(MaintainerOperationType.SNAPSHOT_EXPIRATION);
+ verify(mockMetrics, times(1))
+ .recordOperationSuccess(eq(MaintainerOperationType.SNAPSHOT_EXPIRATION), anyLong());
+ }
+
+ @Test
+ public void testDurationIsRecorded() throws Throwable {
+ // Add a small delay to ensure duration > 0
+ long startTime = System.currentTimeMillis();
+ template.execute(
+ MaintainerOperationType.DATA_EXPIRATION,
+ () -> {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ });
+ long endTime = System.currentTimeMillis();
+
+ // Verify duration is recorded and is reasonable
+ InOrder inOrder = inOrder(mockMetrics);
+ inOrder.verify(mockMetrics).recordOperationStart(MaintainerOperationType.DATA_EXPIRATION);
+ inOrder
+ .verify(mockMetrics)
+ .recordOperationSuccess(eq(MaintainerOperationType.DATA_EXPIRATION), anyLong());
+ }
+
+ @Test
+ public void testAllOperationTypes() throws Throwable {
+ // Test all operation types
+ MaintainerOperationType[] operationTypes = MaintainerOperationType.values();
+
+ for (MaintainerOperationType operationType : operationTypes) {
+ template.execute(
+ operationType,
+ () -> {
+ // Operation logic
+ });
+ }
+
+ // Verify all operation types were recorded
+ for (MaintainerOperationType operationType : operationTypes) {
+ verify(mockMetrics, times(1)).recordOperationStart(operationType);
+ verify(mockMetrics, times(1)).recordOperationSuccess(eq(operationType), anyLong());
+ }
+ }
+}
diff --git a/amoro-format-iceberg/src/main/java/org/apache/amoro/formats/iceberg/maintainer/IcebergTableMaintainer.java b/amoro-format-iceberg/src/main/java/org/apache/amoro/formats/iceberg/maintainer/IcebergTableMaintainer.java
index 3d1d4bfb1c..a8407578dc 100644
--- a/amoro-format-iceberg/src/main/java/org/apache/amoro/formats/iceberg/maintainer/IcebergTableMaintainer.java
+++ b/amoro-format-iceberg/src/main/java/org/apache/amoro/formats/iceberg/maintainer/IcebergTableMaintainer.java
@@ -31,6 +31,7 @@
import org.apache.amoro.io.PathInfo;
import org.apache.amoro.io.SupportsFileSystemOperations;
import org.apache.amoro.maintainer.MaintainerMetrics;
+import org.apache.amoro.maintainer.MaintainerOperationType;
import org.apache.amoro.maintainer.OptimizingInfo;
import org.apache.amoro.maintainer.TableMaintainer;
import org.apache.amoro.maintainer.TableMaintainerContext;
@@ -153,6 +154,8 @@ public void cleanOrphanFiles() {
@Override
public void cleanDanglingDeleteFiles() {
TableConfiguration tableConfiguration = context.getTableConfiguration();
+ MaintainerMetrics metrics = context.getMetrics();
+
if (!tableConfiguration.isDeleteDanglingDeleteFilesEnabled()) {
return;
}
@@ -165,7 +168,17 @@ public void cleanDanglingDeleteFiles() {
Optional.ofNullable(currentSnapshot.summary().get(SnapshotSummary.TOTAL_DELETE_FILES_PROP));
if (totalDeleteFiles.isPresent() && Long.parseLong(totalDeleteFiles.get()) > 0) {
// clear dangling delete files
- doCleanDanglingDeleteFiles();
+ LOG.info("Starting cleaning dangling delete files for table {}", table.name());
+ int danglingDeleteFilesCnt = clearInternalTableDanglingDeleteFiles();
+ runWithCondition(
+ danglingDeleteFilesCnt > 0,
+ () -> {
+ LOG.info(
+ "Deleted {} dangling delete files for table {}",
+ danglingDeleteFilesCnt,
+ table.name());
+ metrics.recordDanglingDeleteFilesCleaned(danglingDeleteFilesCnt);
+ });
} else {
LOG.debug(
"There are no delete files here, so there is no need to clean dangling delete file for table {}",
@@ -178,66 +191,108 @@ public void expireSnapshots() {
if (!expireSnapshotEnabled()) {
return;
}
- expireSnapshots(
- mustOlderThan(System.currentTimeMillis()),
- context.getTableConfiguration().getSnapshotMinCount());
- }
+ MaintainerMetrics metrics = context.getMetrics();
+ long mustOlderThan = mustOlderThan(System.currentTimeMillis());
+ int minCount = context.getTableConfiguration().getSnapshotMinCount();
- public boolean expireSnapshotEnabled() {
- TableConfiguration tableConfiguration = context.getTableConfiguration();
- return tableConfiguration.isExpireSnapshotEnabled();
- }
+ long startTime = System.currentTimeMillis();
+ metrics.recordOperationStart(MaintainerOperationType.SNAPSHOT_EXPIRATION);
- @VisibleForTesting
- public void expireSnapshots(long mustOlderThan, int minCount) {
- expireSnapshots(mustOlderThan, minCount, expireSnapshotNeedToExcludeFiles());
+ try {
+ expireSnapshotsWithMetrics(mustOlderThan, minCount, metrics, startTime);
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationSuccess(MaintainerOperationType.SNAPSHOT_EXPIRATION, duration);
+ } catch (Throwable t) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationFailure(MaintainerOperationType.SNAPSHOT_EXPIRATION, duration, t);
+ throw t;
+ }
}
- private void expireSnapshots(long olderThan, int minCount, Set exclude) {
+ private void expireSnapshotsWithMetrics(
+ long mustOlderThan, int minCount, MaintainerMetrics metrics, long startTime) {
+ Set exclude = expireSnapshotNeedToExcludeFiles();
LOG.debug(
"Starting snapshots expiration for table {}, expiring snapshots older than {} and retain last {} snapshots, excluding {}",
table.name(),
- olderThan,
+ mustOlderThan,
minCount,
exclude);
RollingFileCleaner expiredFileCleaner = new RollingFileCleaner(fileIO(), exclude);
table
.expireSnapshots()
.retainLast(Math.max(minCount, 1))
- .expireOlderThan(olderThan)
+ .expireOlderThan(mustOlderThan)
.deleteWith(expiredFileCleaner::addFile)
.cleanExpiredFiles(
true) /* enable clean only for collecting the expired files, will delete them later */
.commit();
- int collectedFiles = expiredFileCleaner.fileCount();
+ int snapshotCount = expiredFileCleaner.fileCount();
+ int dataFilesDeleted = expiredFileCleaner.cleanedFileCount();
expiredFileCleaner.clear();
- if (collectedFiles > 0) {
+
+ long duration = System.currentTimeMillis() - startTime;
+
+ if (snapshotCount > 0) {
LOG.info(
- "Expired {}/{} files for table {} order than {}",
- collectedFiles,
- expiredFileCleaner.cleanedFileCount(),
+ "Expired {}/{} files for table {} older than {}",
+ snapshotCount,
+ dataFilesDeleted,
table.name(),
- DateTimeUtil.formatTimestampMillis(olderThan));
+ DateTimeUtil.formatTimestampMillis(mustOlderThan));
+ metrics.recordSnapshotsExpired(snapshotCount, dataFilesDeleted, duration);
} else {
LOG.debug(
- "No expired files found for table {} order than {}",
+ "No expired files found for table {} older than {}",
table.name(),
- DateTimeUtil.formatTimestampMillis(olderThan));
+ DateTimeUtil.formatTimestampMillis(mustOlderThan));
}
}
+ public boolean expireSnapshotEnabled() {
+ TableConfiguration tableConfiguration = context.getTableConfiguration();
+ return tableConfiguration.isExpireSnapshotEnabled();
+ }
+
+ @VisibleForTesting
+ public void expireSnapshots(long mustOlderThan, int minCount) {
+ MaintainerMetrics metrics = context.getMetrics();
+ long startTime = System.currentTimeMillis();
+ expireSnapshotsWithMetrics(mustOlderThan, minCount, metrics, startTime);
+ }
+
+ private void expireSnapshots(long olderThan, int minCount, Set exclude) {
+ // This method is kept for backward compatibility with potential subclasses
+ // Redirect to the new method with metrics
+ MaintainerMetrics metrics = context.getMetrics();
+ long startTime = System.currentTimeMillis();
+ expireSnapshotsWithMetrics(olderThan, minCount, metrics, startTime);
+ }
+
@Override
public void expireData() {
DataExpirationConfig expirationConfig = context.getTableConfiguration().getExpiringDataConfig();
+ MaintainerMetrics metrics = context.getMetrics();
+
+ long startTime = System.currentTimeMillis();
+ metrics.recordOperationStart(MaintainerOperationType.DATA_EXPIRATION);
+
try {
Types.NestedField field = table.schema().findField(expirationConfig.getExpirationField());
if (!isValidDataExpirationField(expirationConfig, field, table.name())) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationSuccess(MaintainerOperationType.DATA_EXPIRATION, duration);
return;
}
- expireDataFrom(expirationConfig, expireBaseOnRule(expirationConfig, field));
+ expireDataFromWithMetrics(
+ expirationConfig, expireBaseOnRule(expirationConfig, field), metrics, startTime);
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationSuccess(MaintainerOperationType.DATA_EXPIRATION, duration);
} catch (Throwable t) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationFailure(MaintainerOperationType.DATA_EXPIRATION, duration, t);
LOG.error("Unexpected purge error for table {} ", tableIdentifier, t);
}
}
@@ -273,26 +328,191 @@ public void expireDataFrom(DataExpirationConfig expirationConfig, Instant instan
if (instant.equals(Instant.MIN)) {
return;
}
+ MaintainerMetrics metrics = context.getMetrics();
+ long startTime = System.currentTimeMillis();
+ expireDataFromWithMetrics(expirationConfig, instant, metrics, startTime);
+ }
+ private void expireDataFromWithMetrics(
+ DataExpirationConfig expirationConfig,
+ Instant instant,
+ MaintainerMetrics metrics,
+ long startTime) {
long expireTimestamp = instant.minusMillis(expirationConfig.getRetentionTime()).toEpochMilli();
+ Types.NestedField field = table.schema().findField(expirationConfig.getExpirationField());
LOG.info(
"Expiring data older than {} in table {} ",
- Instant.ofEpochMilli(expireTimestamp)
- .atZone(
- getDefaultZoneId(table.schema().findField(expirationConfig.getExpirationField())))
- .toLocalDateTime(),
+ Instant.ofEpochMilli(expireTimestamp).atZone(getDefaultZoneId(field)).toLocalDateTime(),
table.name());
Expression dataFilter = getDataExpression(table.schema(), expirationConfig, expireTimestamp);
ExpireFiles expiredFiles = expiredFileScan(expirationConfig, dataFilter, expireTimestamp);
- expireFiles(expiredFiles, expireTimestamp);
+
+ int dataFilesCount = expiredFiles.dataFiles.size();
+ int deleteFilesCount = expiredFiles.deleteFiles.size();
+
+ if (dataFilesCount > 0 || deleteFilesCount > 0) {
+ expireFiles(expiredFiles, expireTimestamp);
+ }
+
+ long duration = System.currentTimeMillis() - startTime;
+
+ if (dataFilesCount > 0 || deleteFilesCount > 0) {
+ LOG.info(
+ "Data expiration completed for table {}, {} data files and {} delete files expired, duration: {}ms",
+ table.name(),
+ dataFilesCount,
+ deleteFilesCount,
+ duration);
+ metrics.recordDataExpired(dataFilesCount, deleteFilesCount, duration);
+ }
}
@Override
public void autoCreateTags() {
TagConfiguration tagConfiguration = context.getTableConfiguration().getTagConfiguration();
- new AutoCreateIcebergTagAction(table, tagConfiguration, LocalDateTime.now()).execute();
+ MaintainerMetrics metrics = context.getMetrics();
+
+ long startTime = System.currentTimeMillis();
+ metrics.recordOperationStart(MaintainerOperationType.TAG_CREATION);
+
+ try {
+ int tagsCreated = autoCreateTagsWithMetrics(tagConfiguration);
+ long duration = System.currentTimeMillis() - startTime;
+ if (tagsCreated > 0) {
+ metrics.recordTagsCreated(tagsCreated, duration);
+ }
+ metrics.recordOperationSuccess(MaintainerOperationType.TAG_CREATION, duration);
+ } catch (Throwable t) {
+ long duration = System.currentTimeMillis() - startTime;
+ metrics.recordOperationFailure(MaintainerOperationType.TAG_CREATION, duration, t);
+ throw t;
+ }
+ }
+
+ private int autoCreateTagsWithMetrics(TagConfiguration tagConfiguration) {
+ LocalDateTime checkTime = LocalDateTime.now();
+
+ if (!tagConfiguration.isAutoCreateTag()) {
+ return 0;
+ }
+ LOG.debug("Start checking the automatic creation of tags for {}", table.name());
+ if (tagExists(tagConfiguration, checkTime)) {
+ LOG.debug("Found the expected tag on {}, skip", table.name());
+ return 0;
+ }
+ boolean success = createTagWithConfig(tagConfiguration, checkTime);
+ if (success) {
+ LOG.info("Created a tag successfully on {}", table.name());
+ return 1;
+ } else {
+ LOG.info("Skipped tag creation on {}", table.name());
+ return 0;
+ }
+ }
+
+ /**
+ * Check if tag exists for the given tag configuration.
+ *
+ * @param tagConfiguration tag configuration
+ * @param checkTime check time
+ * @return true if tag exists, false otherwise
+ */
+ private boolean tagExists(TagConfiguration tagConfiguration, LocalDateTime checkTime) {
+ LocalDateTime tagTime =
+ tagConfiguration
+ .getTriggerPeriod()
+ .getTagTime(checkTime, tagConfiguration.getTriggerOffsetMinutes());
+ LocalDateTime triggerTime = tagTime.plusMinutes(tagConfiguration.getTriggerOffsetMinutes());
+ String tagName =
+ tagConfiguration
+ .getTriggerPeriod()
+ .generateTagName(tagTime, tagConfiguration.getTagFormat());
+
+ String tag =
+ table.refs().entrySet().stream()
+ .filter(entry -> entry.getValue().isTag())
+ .map(Map.Entry::getKey)
+ .filter(tagName::equals)
+ .findFirst()
+ .orElse(null);
+ return tag != null;
+ }
+
+ /**
+ * Create tag using the provided configuration.
+ *
+ * @param tagConfiguration the tag configuration
+ * @param checkTime the check time
+ * @return true if tag was created, false otherwise
+ */
+ private boolean createTagWithConfig(TagConfiguration tagConfiguration, LocalDateTime checkTime) {
+ LocalDateTime tagTime =
+ tagConfiguration
+ .getTriggerPeriod()
+ .getTagTime(checkTime, tagConfiguration.getTriggerOffsetMinutes());
+ String tagName =
+ tagConfiguration
+ .getTriggerPeriod()
+ .generateTagName(tagTime, tagConfiguration.getTagFormat());
+
+ // Create the tag
+ long tagTriggerTimestampMillis =
+ tagTime.atZone(java.time.ZoneId.systemDefault()).toInstant().toEpochMilli();
+ org.apache.iceberg.Snapshot snapshot = findSnapshotForTag(table, tagTriggerTimestampMillis);
+ if (snapshot == null) {
+ LOG.info("Found no snapshot at {} for {}", tagTriggerTimestampMillis, table.name());
+ return false;
+ }
+
+ if (exceedMaxDelayForTag(snapshot, tagConfiguration, tagTriggerTimestampMillis)) {
+ LOG.info(
+ "{}'s snapshot {} at {} exceeds max delay {}, and the expected trigger time is {}",
+ table.name(),
+ snapshot.snapshotId(),
+ snapshot.timestampMillis(),
+ tagConfiguration.getMaxDelayMinutes(),
+ tagTriggerTimestampMillis);
+ return false;
+ }
+
+ org.apache.iceberg.ManageSnapshots tag =
+ table.manageSnapshots().createTag(tagName, snapshot.snapshotId());
+ if (tagConfiguration.getTagMaxAgeMs() > 0) {
+ tag.setMaxRefAgeMs(tagName, tagConfiguration.getTagMaxAgeMs());
+ }
+ tag.commit();
+ LOG.info(
+ "Created a tag {} for {} on snapshot {} at {}",
+ tagName,
+ table.name(),
+ snapshot.snapshotId(),
+ snapshot.timestampMillis());
+ return true;
+ }
+
+ private static org.apache.iceberg.Snapshot findSnapshotForTag(
+ org.apache.iceberg.Table table, long tagTriggerTime) {
+ Iterable snapshots = table.snapshots();
+ for (org.apache.iceberg.Snapshot snapshot : snapshots) {
+ long waterMark = snapshot.timestampMillis();
+ if (waterMark >= tagTriggerTime) {
+ return snapshot;
+ }
+ }
+ return null;
+ }
+
+ private boolean exceedMaxDelayForTag(
+ org.apache.iceberg.Snapshot snapshot,
+ TagConfiguration tagConfig,
+ long tagTriggerTimestampMillis) {
+ if (tagConfig.getMaxDelayMinutes() <= 0) {
+ return false;
+ }
+ long delay = snapshot.timestampMillis() - tagTriggerTimestampMillis;
+ return delay > tagConfig.getMaxDelayMinutes() * 60_000L;
}
public void cleanContentFiles(long lastTime, MaintainerMetrics metrics) {
diff --git a/amoro-format-iceberg/src/test/java/org/apache/amoro/formats/iceberg/maintainer/TestIcebergTableMaintainerMetrics.java b/amoro-format-iceberg/src/test/java/org/apache/amoro/formats/iceberg/maintainer/TestIcebergTableMaintainerMetrics.java
new file mode 100644
index 0000000000..03a7e1ac51
--- /dev/null
+++ b/amoro-format-iceberg/src/test/java/org/apache/amoro/formats/iceberg/maintainer/TestIcebergTableMaintainerMetrics.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.amoro.formats.iceberg.maintainer;
+
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.amoro.config.TableConfiguration;
+import org.apache.amoro.maintainer.MaintainerMetrics;
+import org.apache.amoro.maintainer.MaintainerOperationType;
+import org.apache.amoro.maintainer.OptimizingInfo;
+import org.apache.amoro.maintainer.TableMaintainerContext;
+import org.apache.amoro.table.TableIdentifier;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.InOrder;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+/**
+ * Unit tests for IcebergTableMaintainer metrics recording.
+ *
+ * These tests verify that IcebergTableMaintainer correctly records metrics through the
+ * MaintainerMetrics interface for all maintainer operations.
+ */
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.LENIENT)
+public class TestIcebergTableMaintainerMetrics {
+
+ @Mock private TableMaintainerContext mockContext;
+
+ @Mock private MaintainerMetrics mockMetrics;
+
+ @Mock private TableConfiguration mockTableConfiguration;
+
+ @Mock private OptimizingInfo mockOptimizingInfo;
+
+ @Mock private org.apache.amoro.config.TagConfiguration mockTagConfiguration;
+
+ @Mock private org.apache.amoro.config.DataExpirationConfig mockDataExpirationConfig;
+
+ private IcebergTableMaintainer maintainer;
+
+ @BeforeEach
+ public void setUp() {
+ TableIdentifier tableIdentifier = TableIdentifier.of("test_catalog", "test_db", "test_table");
+
+ when(mockContext.getMetrics()).thenReturn(mockMetrics);
+ when(mockContext.getTableConfiguration()).thenReturn(mockTableConfiguration);
+ when(mockContext.getOptimizingInfo()).thenReturn(mockOptimizingInfo);
+ when(mockOptimizingInfo.isProcessing()).thenReturn(false);
+
+ // Feature disabled by default to avoid complex setup
+ when(mockTableConfiguration.isCleanOrphanEnabled()).thenReturn(false);
+ when(mockTableConfiguration.isDeleteDanglingDeleteFilesEnabled()).thenReturn(false);
+ when(mockTableConfiguration.isExpireSnapshotEnabled()).thenReturn(false);
+
+ // Setup Tag and DataExpiration configs to avoid NPE
+ when(mockTableConfiguration.getTagConfiguration()).thenReturn(mockTagConfiguration);
+ when(mockTagConfiguration.isAutoCreateTag()).thenReturn(false);
+ when(mockTableConfiguration.getExpiringDataConfig()).thenReturn(mockDataExpirationConfig);
+ when(mockDataExpirationConfig.getExpirationField()).thenReturn("");
+
+ // Create a minimal Table mock to avoid complex setup
+ org.apache.iceberg.Table table = createMinimalTableMock();
+
+ maintainer = new IcebergTableMaintainer(table, tableIdentifier, mockContext);
+ }
+
+ @Test
+ public void testExpireSnapshotsDisabledDoesNotRecordMetrics() {
+ // Feature disabled by default in setUp()
+ when(mockTableConfiguration.isExpireSnapshotEnabled()).thenReturn(false);
+
+ // Execute
+ maintainer.expireSnapshots();
+
+ // Verify no metrics are recorded when feature is disabled
+ verify(mockMetrics, never()).recordOperationStart(any(MaintainerOperationType.class));
+ }
+
+ @Test
+ public void testAutoCreateTagsDisabledDoesNotRecordDetailedMetrics() {
+ // Setup - feature disabled
+ when(mockTagConfiguration.isAutoCreateTag()).thenReturn(false);
+
+ // Execute
+ maintainer.autoCreateTags();
+
+ // Verify operation-level metrics are still recorded even when no tags are created
+ InOrder inOrder = inOrder(mockMetrics);
+ inOrder.verify(mockMetrics).recordOperationStart(MaintainerOperationType.TAG_CREATION);
+ inOrder
+ .verify(mockMetrics)
+ .recordOperationSuccess(eq(MaintainerOperationType.TAG_CREATION), anyLong());
+ }
+
+ @Test
+ public void testExpireDataInvalidFieldDoesNotRecordDetailedMetrics() {
+ // Setup - empty expiration field causes IllegalArgumentException
+ when(mockDataExpirationConfig.getExpirationField()).thenReturn("");
+ when(mockDataExpirationConfig.getBaseOnRule())
+ .thenReturn(org.apache.amoro.config.DataExpirationConfig.BaseOnRule.CURRENT_TIME);
+
+ // Execute - this will throw exception for empty field name
+ maintainer.expireData();
+
+ // Verify operation-level metrics are still recorded even when field is invalid
+ // Exception is caught and logged, metrics still record failure
+ verify(mockMetrics).recordOperationStart(MaintainerOperationType.DATA_EXPIRATION);
+ // When exception occurs, recordOperationFailure is called
+ verify(mockMetrics)
+ .recordOperationFailure(eq(MaintainerOperationType.DATA_EXPIRATION), anyLong(), any());
+ }
+
+ @Test
+ public void testCleanOrphanFilesDisabledDoesNotRecordMetrics() {
+ // Feature disabled by default in setUp()
+ when(mockTableConfiguration.isCleanOrphanEnabled()).thenReturn(false);
+
+ // Execute
+ maintainer.cleanOrphanFiles();
+
+ // Verify no metrics are recorded when feature is disabled
+ verify(mockMetrics, never()).recordOperationStart(any(MaintainerOperationType.class));
+ }
+
+ @Test
+ public void testCleanDanglingDeleteFilesDisabledDoesNotRecordMetrics() {
+ // Feature disabled by default in setUp()
+ when(mockTableConfiguration.isDeleteDanglingDeleteFilesEnabled()).thenReturn(false);
+
+ // Execute
+ maintainer.cleanDanglingDeleteFiles();
+
+ // Verify no dangling delete metrics are recorded when feature is disabled
+ verify(mockMetrics, never()).recordDanglingDeleteFilesCleaned(anyInt());
+ }
+
+ @Test
+ public void testAllMaintainerOperationsExist() {
+ // Verify that IcebergTableMaintainer implements methods for all relevant operation types
+
+ // IcebergTableMaintainer should cover 5 out of 6 operation types:
+ // ORPHAN_FILES_CLEANING - cleanOrphanFiles()
+ // DANGLING_DELETE_FILES_CLEANING - cleanDanglingDeleteFiles()
+ // SNAPSHOT_EXPIRATION - expireSnapshots()
+ // DATA_EXPIRATION - expireData()
+ // TAG_CREATION - autoCreateTags()
+ // PARTITION_EXPIRATION - not in IcebergTableMaintainer (Paimon-specific)
+
+ // Verify the methods exist and can be called
+ assertDoesNotThrow(() -> maintainer.cleanOrphanFiles());
+ assertDoesNotThrow(() -> maintainer.cleanDanglingDeleteFiles());
+ assertDoesNotThrow(() -> maintainer.expireSnapshots());
+ assertDoesNotThrow(() -> maintainer.expireData());
+ assertDoesNotThrow(() -> maintainer.autoCreateTags());
+ }
+
+ @Test
+ public void testMaintainerMetricsInterfaceHasAllOperationTypes() {
+ // Verify MaintainerOperationType enum has all expected operation types
+ MaintainerOperationType[] operationTypes = MaintainerOperationType.values();
+
+ // Should have 6 operation types
+ assertEquals(6, operationTypes.length);
+
+ // Verify specific operation types exist
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("ORPHAN_FILES_CLEANING"));
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("DANGLING_DELETE_FILES_CLEANING"));
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("SNAPSHOT_EXPIRATION"));
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("DATA_EXPIRATION"));
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("TAG_CREATION"));
+ assertDoesNotThrow(() -> MaintainerOperationType.valueOf("PARTITION_EXPIRATION"));
+ }
+
+ /**
+ * Creates a minimal Table mock that avoids complex setup. The mock is configured to handle basic
+ * method calls without throwing exceptions.
+ */
+ private org.apache.iceberg.Table createMinimalTableMock() {
+ org.apache.iceberg.Table table = org.mockito.Mockito.mock(org.apache.iceberg.Table.class);
+
+ // Configure minimal behavior to avoid NullPointerExceptions
+ when(table.name()).thenReturn("test_catalog.test_db.test_table");
+ when(table.schema())
+ .thenReturn(
+ new org.apache.iceberg.Schema(
+ org.apache.iceberg.types.Types.NestedField.optional(
+ 1, "ts", org.apache.iceberg.types.Types.TimestampType.withoutZone())));
+ when(table.currentSnapshot()).thenReturn(null);
+
+ return table;
+ }
+}