stevenzwu commented on code in PR #11144:
URL: https://github.com/apache/iceberg/pull/11144#discussion_r1776132332


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.iceberg.flink.maintenance.operator;
+
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Delete the files using the {@link FileIO} which implements {@link 
SupportsBulkOperations}. */
+@Internal
+public class DeleteFilesProcessor extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<String, Void> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DeleteFilesProcessor.class);
+
+  private final String name;
+  private final SupportsBulkOperations io;
+  private final String tableName;
+  private final Set<String> filesToDelete = Sets.newHashSet();
+  private final int batchSize;
+
+  private transient Counter failedCounter;
+  private transient Counter succeededCounter;
+
+  public DeleteFilesProcessor(String name, TableLoader tableLoader, int 
batchSize) {
+    Preconditions.checkNotNull(name, "Name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    tableLoader.open();

Review Comment:
   table loader seems not closed. I actually think we don't need a 
`TableLoader` here. A read-only `SerializableTable` should work too.



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.iceberg.flink.maintenance.api;
+
+import static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.ExpireSnapshots.DELETE_FILES_OPERATOR_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.Duration;
+import java.util.Set;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.awaitility.Awaitility;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class TestExpireSnapshots extends MaintenanceTaskTestBase {
+  private Table table;
+
+  @BeforeEach
+  void before() {
+    MetricsReporterFactoryForTests.reset();
+    this.table = createTable();
+  }
+
+  @Test
+  void testExpireSnapshots() throws Exception {
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");
+
+    Set<Snapshot> snapshots = Sets.newHashSet(table.snapshots());
+    assertThat(snapshots).hasSize(4);
+
+    ExpireSnapshots.builder()
+        .parallelism(1)
+        .planningWorkerPoolSize(2)
+        .deleteBatchSize(3)
+        .deleteParallelism(1)
+        .maxSnapshotAge(Duration.ZERO)
+        .retainLast(1)
+        .uidSuffix(UID_SUFFIX)
+        .append(
+            infra.triggerStream(),
+            0,
+            DUMMY_NAME,
+            tableLoader(),
+            "OTHER",
+            StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP,
+            1)
+        .sinkTo(infra.sink());
+
+    runAndWaitForSuccess(
+        infra.env(), infra.source(), infra.sink(), () -> 
checkDeleteFinished(3L), table);
+
+    table.refresh();
+    assertThat(Sets.newHashSet(table.snapshots())).hasSize(1);
+    // Check that the table data not changed
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "a"),
+            createRecord(2, "b"),
+            createRecord(3, "c"),
+            createRecord(4, "d")));
+  }
+
+  @Test
+  void testFailure() throws Exception {
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    SerializableTable serializableTable = (SerializableTable) 
SerializableTable.copyOf(table);
+
+    ExpireSnapshots.builder()
+        .append(
+            infra.triggerStream(),
+            0,
+            DUMMY_NAME,
+            tableLoader(),
+            UID_SUFFIX,
+            StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP,
+            1)
+        .sinkTo(infra.sink());
+
+    JobClient jobClient = null;
+    try {
+      jobClient = infra.env().executeAsync();
+
+      // Do a single task run
+      long time = System.currentTimeMillis();
+      infra
+          .source()
+          .sendRecord(Trigger.create(time, serializableTable, 1), 
System.currentTimeMillis());
+
+      // First successful run (ensure that the operators are loaded/opened 
etc.)
+      assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isTrue();
+
+      // Drop the table, so it will cause an exception
+      dropTable();
+
+      // Failed run
+      infra.source().sendRecord(Trigger.create(time + 1, serializableTable, 
1), time + 1);
+
+      assertThat(infra.sink().poll(Duration.ofSeconds(5)).success()).isFalse();
+    } finally {
+      closeJobClient(jobClient);
+    }
+
+    // Check the metrics
+    MetricsReporterFactoryForTests.assertCounters(
+        new ImmutableMap.Builder<String, Long>()
+            .put(
+                DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + 
DELETE_FILE_FAILED_COUNTER,
+                0L)
+            .put(
+                DELETE_FILES_OPERATOR_NAME + "." + DUMMY_NAME + "." + 
DELETE_FILE_SUCCEEDED_COUNTER,
+                0L)

Review Comment:
   there was a successful task run in line 117-124. why is the success counter 
still zero?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestExpireSnapshots.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.iceberg.flink.maintenance.api;
+
+import static org.apache.iceberg.flink.SimpleDataUtil.createRecord;
+import static 
org.apache.iceberg.flink.maintenance.api.ExpireSnapshots.DELETE_FILES_OPERATOR_NAME;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_FAILED_COUNTER;
+import static 
org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.DELETE_FILE_SUCCEEDED_COUNTER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.Duration;
+import java.util.Set;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.SimpleDataUtil;
+import 
org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.awaitility.Awaitility;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class TestExpireSnapshots extends MaintenanceTaskTestBase {
+  private Table table;
+
+  @BeforeEach
+  void before() {
+    MetricsReporterFactoryForTests.reset();
+    this.table = createTable();
+  }
+
+  @Test
+  void testExpireSnapshots() throws Exception {
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+    insert(table, 3, "c");
+    insert(table, 4, "d");
+
+    Set<Snapshot> snapshots = Sets.newHashSet(table.snapshots());
+    assertThat(snapshots).hasSize(4);
+
+    ExpireSnapshots.builder()
+        .parallelism(1)
+        .planningWorkerPoolSize(2)
+        .deleteBatchSize(3)
+        .deleteParallelism(1)
+        .maxSnapshotAge(Duration.ZERO)
+        .retainLast(1)
+        .uidSuffix(UID_SUFFIX)
+        .append(
+            infra.triggerStream(),
+            0,
+            DUMMY_NAME,
+            tableLoader(),
+            "OTHER",
+            StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP,
+            1)
+        .sinkTo(infra.sink());
+
+    runAndWaitForSuccess(
+        infra.env(), infra.source(), infra.sink(), () -> 
checkDeleteFinished(3L), table);
+
+    table.refresh();
+    assertThat(Sets.newHashSet(table.snapshots())).hasSize(1);
+    // Check that the table data not changed
+    SimpleDataUtil.assertTableRecords(
+        table,
+        ImmutableList.of(
+            createRecord(1, "a"),
+            createRecord(2, "b"),
+            createRecord(3, "c"),
+            createRecord(4, "d")));
+  }
+
+  @Test
+  void testFailure() throws Exception {
+    insert(table, 1, "a");
+    insert(table, 2, "b");
+
+    SerializableTable serializableTable = (SerializableTable) 
SerializableTable.copyOf(table);
+
+    ExpireSnapshots.builder()
+        .append(
+            infra.triggerStream(),
+            0,
+            DUMMY_NAME,
+            tableLoader(),
+            UID_SUFFIX,
+            StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP,
+            1)
+        .sinkTo(infra.sink());
+
+    JobClient jobClient = null;
+    try {
+      jobClient = infra.env().executeAsync();
+
+      // Do a single task run
+      long time = System.currentTimeMillis();
+      infra
+          .source()
+          .sendRecord(Trigger.create(time, serializableTable, 1), 
System.currentTimeMillis());

Review Comment:
   should `System.currentTimeMillis()` be replaced as `time`?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestMaintenanceE2E.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.iceberg.flink.maintenance.api;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.IOException;
+import java.time.Duration;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.maintenance.operator.OperatorTestBase;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class TestMaintenanceE2E extends OperatorTestBase {
+  private StreamExecutionEnvironment env;
+
+  @BeforeEach
+  public void beforeEach() throws IOException {
+    this.env = StreamExecutionEnvironment.getExecutionEnvironment();
+    Table table = createTable();
+    insert(table, 1, "a");
+  }
+
+  @Test
+  void testE2e() throws Exception {
+    TableMaintenance.forTable(env, tableLoader(), LOCK_FACTORY)
+        .uidSuffix("E2eTestUID")
+        .rateLimit(Duration.ofMinutes(10))
+        .lockCheckDelay(Duration.ofSeconds(10))
+        .add(
+            ExpireSnapshots.builder()
+                .scheduleOnCommitCount(10)
+                .maxSnapshotAge(Duration.ofMinutes(10))
+                .retainLast(5)
+                .deleteBatchSize(5)
+                .parallelism(8))
+        .append();
+
+    JobClient jobClient = null;
+    try {
+      jobClient = env.executeAsync();
+
+      // Just make sure that we are able to instantiate the flow

Review Comment:
   should exception be thrown if the instantiation failed?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.iceberg.flink.maintenance.operator;
+
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Delete the files using the {@link FileIO} which implements {@link 
SupportsBulkOperations}. */
+@Internal
+public class DeleteFilesProcessor extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<String, Void> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DeleteFilesProcessor.class);
+
+  private final String name;
+  private final SupportsBulkOperations io;
+  private final String tableName;
+  private final Set<String> filesToDelete = Sets.newHashSet();
+  private final int batchSize;
+
+  private transient Counter failedCounter;
+  private transient Counter succeededCounter;
+
+  public DeleteFilesProcessor(String name, TableLoader tableLoader, int 
batchSize) {
+    Preconditions.checkNotNull(name, "Name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    tableLoader.open();
+    Table table = tableLoader.loadTable();
+    FileIO fileIO = table.io();
+    Preconditions.checkArgument(
+        fileIO instanceof SupportsBulkOperations,
+        "Unsupported FileIO. %s should support bulk delete",
+        fileIO);

Review Comment:
   nit: should this be `fileIO.getClass().getSimpleName()`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DeleteFilesProcessor.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.iceberg.flink.maintenance.operator;
+
+import java.util.Set;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.BulkDeletionFailureException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.SupportsBulkOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Delete the files using the {@link FileIO} which implements {@link 
SupportsBulkOperations}. */
+@Internal
+public class DeleteFilesProcessor extends AbstractStreamOperator<Void>
+    implements OneInputStreamOperator<String, Void> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(DeleteFilesProcessor.class);
+
+  private final String name;
+  private final SupportsBulkOperations io;
+  private final String tableName;
+  private final Set<String> filesToDelete = Sets.newHashSet();
+  private final int batchSize;
+
+  private transient Counter failedCounter;
+  private transient Counter succeededCounter;
+
+  public DeleteFilesProcessor(String name, TableLoader tableLoader, int 
batchSize) {
+    Preconditions.checkNotNull(name, "Name should no be null");
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+
+    tableLoader.open();
+    Table table = tableLoader.loadTable();
+    FileIO fileIO = table.io();
+    Preconditions.checkArgument(
+        fileIO instanceof SupportsBulkOperations,
+        "Unsupported FileIO. %s should support bulk delete",

Review Comment:
   nit: Iceberg style don't use class name in error msg. maybe `%s doesn't 
support bulk delete`?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ExpireSnapshots.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.iceberg.flink.maintenance.api;
+
+import java.time.Duration;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SystemConfigs;
+import org.apache.iceberg.flink.maintenance.operator.DeleteFilesProcessor;
+import org.apache.iceberg.flink.maintenance.operator.ExpireSnapshotsProcessor;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/** Deletes expired snapshots and the corresponding files. */
+public class ExpireSnapshots {
+  private static final int DELETE_BATCH_SIZE_DEFAULT = 1000;
+  private static final String EXECUTOR_OPERATOR_NAME = "Expire Snapshot";
+  @VisibleForTesting static final String DELETE_FILES_OPERATOR_NAME = "Delete 
file";
+
+  private ExpireSnapshots() {
+    // Do not instantiate directly

Review Comment:
   nit: Iceberg style doesn't add this obvious comment.. e.g. 
   ```
   public class ArrayUtil {
     private ArrayUtil() {}
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to