You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/29 07:48:11 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #1264: Expire snapshots action

RussellSpitzer opened a new pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264


   WIP I'll update this and the commit messages when the underlying PR's are merged


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467122396



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+

Review comment:
       The only tests not directly ported were those doing file cleanup (except for the parallel delete one) since we already had new tests for those functions.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467120561



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466523990



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();

Review comment:
       I was thinking about that, but also thought that any exception here would probably kill the driver :/ I'll add in the try finally though




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466007033



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));
+  }
+
+  private Dataset<Row> getValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATAFILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFESTLIST))
+        .union(appendTypeString(buildOtherMetadataFileDF(spark, ops), OTHER));
+  }
+
+  private Set<String> getFilesOfType(List<Row> files, String type) {
+    return files.stream()
+        .filter(row -> row.getString(1).equals(type))
+        .map(row -> row.getString(0))
+        .collect(Collectors.toSet());
+  }
+
+  @Override
+  public ExpireSnapshotActionResult execute() {
+
+    Dataset<Row> originalFiles = getValidFileDF().persist();
+    originalFiles.count(); // Trigger Persist
+
+    localExpireSnapshots.commit();
+
+    Dataset<Row> validFiles = getValidFileDF();
+
+    List<Row> filesToDelete = originalFiles.except(validFiles).collectAsList();
+
+    LOG.warn("Deleting {} files", filesToDelete.size());
+    return new ExpireSnapshotActionResult(
+        deleteFiles(getFilesOfType(filesToDelete, DATAFILE), DATAFILE),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFEST), MANIFEST),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFESTLIST), MANIFESTLIST),
+        deleteFiles(getFilesOfType(filesToDelete, OTHER), OTHER));
+  }
+
+  private Long deleteFiles(Set<String> paths, String fileType) {
+    LOG.warn("{}s to delete: {}", fileType, Joiner.on(", ").join(paths));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(ThreadPools.getWorkerPool())

Review comment:
       going for executeDeleteWith
   




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467120228



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {

Review comment:
       Ported test - everything the same except for the manifest manipulation see comment below




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-669505845


   All of that sounds good to me, let me do a bit of a rework of the delete logic so it fits with an iterator


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468261497



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +50,66 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table
+   * @param table the table
+   * @return the paths of the Manifest Lists
+   */
+  protected List<String> getManifestListPaths(Table table) {

Review comment:
       I think this should be private. We don't want actions using it directly, do we?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468048096



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  /**
+   * A specific snapshot to expire.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireSnapshotId(long)}
+   * @param expireSnapshotId Id of the snapshot to expire
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  /**
+   * Expire all snapshots older than a given timestamp.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireOlderThan(long)}
+   * @param timestampMillis all snapshots before this time will be expired
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  /**
+   * Retain at least x snapshots when expiring
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}
+   * @param numSnapshots number of snapshots to leave
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    Preconditions.checkArgument(1 <= numSnapshots,
+        "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  /**
+   * The Consumer used on files which have been determined to be expired. By default uses a filesystem delete.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#deleteWith(Consumer)}
+   * @param newDeleteFunc Consumer which takes a path and deletes it
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    Dataset<Row> originalFiles = null;
+    try {
+      // Metadata before Expiration
+      originalFiles = buildValidFileDF().persist();
+      // Action to trigger persist
+      originalFiles.count();
+
+      // Perform Expiration
+      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+      if (expireSnapshotIdValue != null) {
+        expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+      }
+      if (expireOlderThanValue != null) {
+        expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+      }
+      if (retainLastValue != null) {
+        expireSnaps = expireSnaps.retainLast(retainLastValue);
+      }
+      expireSnaps.commit();
+
+      // Metadata after Expiration
+      Dataset<Row> validFiles = buildValidFileDF();
+      Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+      ExpireSnapshotsActionResult result = deleteFiles(filesToDelete.toLocalIterator());
+      return result;
+    } finally {
+      if (originalFiles != null) {
+        originalFiles.unpersist();
+      }
+    }
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {

Review comment:
       From @aokolnychyi 's previous comment
   
   > Right now, we don't remove old version files while expiring snapshots. On one hand, we keep old version files for debugging. On the other hand, we have to call RemoveOrphanFilesAction to actually delete them. This action behaves differently compared to RemoveSnapshots and may remove some version files as well. Since expiring snapshots produces a new version, we will delete at most one version file. So, I think ignoring version files from the analysis is the right way to go.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466563000



##########
File path: spark2/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction24.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.actions;
+
+public class TestExpireSnapshotsAction24 extends TestExpireSnapshotsAction{

Review comment:
       This is the concrete implementation of the Abstract class TestExpireSnapshotsAction, this is how we do Spark 2/3 testing with a separate file for each Spark Version




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468042769



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  /**
+   * A specific snapshot to expire.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireSnapshotId(long)}
+   * @param expireSnapshotId Id of the snapshot to expire
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  /**
+   * Expire all snapshots older than a given timestamp.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireOlderThan(long)}
+   * @param timestampMillis all snapshots before this time will be expired
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  /**
+   * Retain at least x snapshots when expiring
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}
+   * @param numSnapshots number of snapshots to leave
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    Preconditions.checkArgument(1 <= numSnapshots,
+        "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  /**
+   * The Consumer used on files which have been determined to be expired. By default uses a filesystem delete.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#deleteWith(Consumer)}
+   * @param newDeleteFunc Consumer which takes a path and deletes it
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    Dataset<Row> originalFiles = null;
+    try {
+      // Metadata before Expiration
+      originalFiles = buildValidFileDF().persist();
+      // Action to trigger persist
+      originalFiles.count();
+
+      // Perform Expiration
+      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+      if (expireSnapshotIdValue != null) {
+        expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+      }
+      if (expireOlderThanValue != null) {

Review comment:
       Nit: we usually add blank lines after `if` and loop control flow statements.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] mehtaashish23 commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
mehtaashish23 commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466542402



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);

Review comment:
       same here.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r465941279



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;

Review comment:
       Looks like we don't need this var anymore.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+

Review comment:
       nit: extra line here

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+

Review comment:
       nit: let's keep only 1 empty line, no need for 2.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,59 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg")
+        .load(allDataFilesMetadataTable)
+        .select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    Dataset<Row> manifestDF = spark.read().format("iceberg")

Review comment:
       nit: do we need this extra variable? My IDE always highlights such cases.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,59 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg")
+        .load(allDataFilesMetadataTable)
+        .select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    Dataset<Row> manifestDF = spark.read().format("iceberg")
+        .load(allManifestsMetadataTable)
+        .selectExpr("path as file_path");
+    return manifestDF;
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();
+
+    for (Snapshot snapshot : table.snapshots()) {
+      String manifestListLocation = snapshot.manifestListLocation();
+      if (manifestListLocation != null) {
+        manifestLists.add(manifestListLocation);
+      }
+    }
+
+    Dataset<Row> manifestListDF = spark

Review comment:
       nit: same here

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotActionResult.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.actions;
+
+public class ExpireSnapshotActionResult {

Review comment:
       I think it should be `Snapshots` instead of `Snapshot` to be consistent with the action name.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,59 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg")
+        .load(allDataFilesMetadataTable)
+        .select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    Dataset<Row> manifestDF = spark.read().format("iceberg")
+        .load(allManifestsMetadataTable)
+        .selectExpr("path as file_path");
+    return manifestDF;
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();
+
+    for (Snapshot snapshot : table.snapshots()) {
+      String manifestListLocation = snapshot.manifestListLocation();
+      if (manifestListLocation != null) {
+        manifestLists.add(manifestListLocation);
+      }
+    }
+
+    Dataset<Row> manifestListDF = spark
+        .createDataset(manifestLists, Encoders.STRING())
+        .toDF("file_path");
+
+    return manifestListDF;
+  }
+
+  protected Dataset<Row> buildOtherMetadataFileDF(SparkSession spark, TableOperations ops) {
+    List<String> otherMetadataFiles = Lists.newArrayList();
+    otherMetadataFiles.add(ops.metadataFileLocation("version-hint.text"));
+    TableMetadata metadata = ops.current();
+    otherMetadataFiles.add(metadata.metadataFileLocation());
+    for (TableMetadata.MetadataLogEntry previousMetadataFile : metadata.previousFiles()) {
+      otherMetadataFiles.add(previousMetadataFile.file());
+    }
+
+    Dataset<Row> otherMetadataFileDF = spark
+        .createDataset(otherMetadataFiles, Encoders.STRING())
+        .toDF("file_path");
+    return otherMetadataFileDF;

Review comment:
       nit: same here

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";

Review comment:
       Let's move static constants above instance variables.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;

Review comment:
       Looks like we can remove this one as well.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";

Review comment:
       It would also more natural to have underscores to separate words.
   For example, `DATA_FILE` and `MANIFEST_LIST`.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,59 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg")
+        .load(allDataFilesMetadataTable)
+        .select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    Dataset<Row> manifestDF = spark.read().format("iceberg")
+        .load(allManifestsMetadataTable)
+        .selectExpr("path as file_path");
+    return manifestDF;
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();
+
+    for (Snapshot snapshot : table.snapshots()) {
+      String manifestListLocation = snapshot.manifestListLocation();
+      if (manifestListLocation != null) {
+        manifestLists.add(manifestListLocation);
+      }
+    }
+
+    Dataset<Row> manifestListDF = spark
+        .createDataset(manifestLists, Encoders.STRING())
+        .toDF("file_path");
+
+    return manifestListDF;
+  }
+
+  protected Dataset<Row> buildOtherMetadataFileDF(SparkSession spark, TableOperations ops) {
+    List<String> otherMetadataFiles = Lists.newArrayList();
+    otherMetadataFiles.add(ops.metadataFileLocation("version-hint.text"));

Review comment:
       Let's add one empty line before and after adding version hint to have some logical grouping.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));

Review comment:
       Why `DataFile` and not `file_type`?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));
+  }
+
+  private Dataset<Row> getValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATAFILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFESTLIST))
+        .union(appendTypeString(buildOtherMetadataFileDF(spark, ops), OTHER));
+  }
+
+  private Set<String> getFilesOfType(List<Row> files, String type) {
+    return files.stream()
+        .filter(row -> row.getString(1).equals(type))
+        .map(row -> row.getString(0))
+        .collect(Collectors.toSet());
+  }
+
+  @Override
+  public ExpireSnapshotActionResult execute() {
+
+    Dataset<Row> originalFiles = getValidFileDF().persist();
+    originalFiles.count(); // Trigger Persist
+
+    localExpireSnapshots.commit();
+
+    Dataset<Row> validFiles = getValidFileDF();
+
+    List<Row> filesToDelete = originalFiles.except(validFiles).collectAsList();
+
+    LOG.warn("Deleting {} files", filesToDelete.size());
+    return new ExpireSnapshotActionResult(
+        deleteFiles(getFilesOfType(filesToDelete, DATAFILE), DATAFILE),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFEST), MANIFEST),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFESTLIST), MANIFESTLIST),
+        deleteFiles(getFilesOfType(filesToDelete, OTHER), OTHER));
+  }
+
+  private Long deleteFiles(Set<String> paths, String fileType) {
+    LOG.warn("{}s to delete: {}", fileType, Joiner.on(", ").join(paths));
+    AtomicReference<Long> deleteCount = new AtomicReference<>(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(ThreadPools.getWorkerPool())

Review comment:
       I think we have to match the behavior in `ExpireSnapshots` where the executor service is optional. I am not sure on the best name for the method, though.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));
+  }
+
+  private Dataset<Row> getValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATAFILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFESTLIST))
+        .union(appendTypeString(buildOtherMetadataFileDF(spark, ops), OTHER));
+  }
+
+  private Set<String> getFilesOfType(List<Row> files, String type) {
+    return files.stream()
+        .filter(row -> row.getString(1).equals(type))
+        .map(row -> row.getString(0))
+        .collect(Collectors.toSet());
+  }
+
+  @Override
+  public ExpireSnapshotActionResult execute() {
+
+    Dataset<Row> originalFiles = getValidFileDF().persist();
+    originalFiles.count(); // Trigger Persist
+
+    localExpireSnapshots.commit();
+
+    Dataset<Row> validFiles = getValidFileDF();
+
+    List<Row> filesToDelete = originalFiles.except(validFiles).collectAsList();
+
+    LOG.warn("Deleting {} files", filesToDelete.size());
+    return new ExpireSnapshotActionResult(
+        deleteFiles(getFilesOfType(filesToDelete, DATAFILE), DATAFILE),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFEST), MANIFEST),
+        deleteFiles(getFilesOfType(filesToDelete, MANIFESTLIST), MANIFESTLIST),
+        deleteFiles(getFilesOfType(filesToDelete, OTHER), OTHER));
+  }
+
+  private Long deleteFiles(Set<String> paths, String fileType) {
+    LOG.warn("{}s to delete: {}", fileType, Joiner.on(", ").join(paths));

Review comment:
       It may be okay to log this for manifests and manifest lists but there may be 10-100k data files to remove. Logging all of them may not be a good idea.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {

Review comment:
       nit: all other actions define overridden methods after the constructor, then public methods to configure the action, then public execute, then private methods. I think it makes to follow that here too.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));
+  }
+
+  private Dataset<Row> getValidFileDF() {

Review comment:
       I think `build` will be a better name than `get`.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467335828



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       I agree. This is more specific, and since it only applies when deleting the data files being more specific is good. I also think that a pool passed to `executeWith` would be used to parallelize the other operations.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-670583528


   @HeartSaVioR , done all tests are noted. Only one test had a significant modification the rests just have the "table.expire" switched with "Actions.forTable().expire. 
   
   Please let me know if there is anything you would like clarity on or if you have any more suggestions
   
   Thanks for your help in the review!


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468040906



##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {

Review comment:
       @HeartSaVioR Just to check with you, would it be ok if I moved the methods back into BaseAction?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468051885



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {

Review comment:
       A couple of things on these tests:
   1. The ported tests don't validate the action result in many cases. Can you add those checks?
   2. The new tests use Spark to write data, which I don't think is necessary. Using Spark to write data makes the tests harder to maintain because you have to go list files to find the ones you want, rather than constructing commits directly with known file paths (e.g., FILE_A, FILE_B, etc.).
   3. I find tests that use `ThreeColumnRecord` and values like `AAAA` to be a bit hard to read, since there isn't much difference between `AAAA` and `AAAAAA` or between `c2` and `c3`. If you remove Spark writes, then you could avoid needing this.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466524766



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();

Review comment:
       Yeah this is temporary, and I'm hoping that in most use cases it won't actually require a disk spill




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468033307



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -19,8 +19,16 @@
 
 package org.apache.iceberg.actions;
 
+import java.util.List;
 import org.apache.iceberg.MetadataTableType;
 import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.util.TableUtil;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+

Review comment:
       Nit: extra newline added.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468046813



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsActionResult.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.actions;
+
+public class ExpireSnapshotsActionResult {
+
+  private final Long dataFilesDeleted;
+  private final Long manifestFilesDeleted;
+  private final Long manifestListsDeleted;
+
+  public ExpireSnapshotsActionResult(Long dataFilesDeleted, Long manifestFilesDeleted, Long manifestListsDeleted) {
+    this.dataFilesDeleted = dataFilesDeleted;
+    this.manifestFilesDeleted = manifestFilesDeleted;
+    this.manifestListsDeleted = manifestListsDeleted;
+  }
+
+  public Long getDataFilesDeleted() {

Review comment:
       Minor: We don't use `get` because it doesn't generally add any value. In most cases, the right thing to do is to use a more descriptive verb, like `load` or `create`. If there isn't a more descriptive verb because it is just accessing a member field, then we omit `get`. That's more readable and the convention in most other JVM languages.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466001296



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,59 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg")
+        .load(allDataFilesMetadataTable)
+        .select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    Dataset<Row> manifestDF = spark.read().format("iceberg")

Review comment:
       Nope, I just have some strange habits, I'll clean it all up




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466565503



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       So let's do "Trace on Data Files" and "Debug" on everything else? I just want to make sure I'm not going against the pattern in RemoveSnapshots




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119542



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+

Review comment:
       Manual metadata rewrite code removed from here since it wasn't really part of the test and I couldn't easily move those functions into Spark module




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119103



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now

Review comment:
       Partitioning changed from TestTableBase to match the schema I was using here




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468163892



##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {
+
+  private TableUtil(){}

Review comment:
       Whole class is now removed




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467335599



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       I think it should be info. You're right that it isn't a warning because nothing went wrong. I would also say it isn't debug, because this is an important part of being able to monitor what happened. We don't need to know which files were deleted, but knowing how many were is a good thing to monitor. For example, if a user reports missing data, we'd go look at the logs for the last expiration and see whether this count is crazy high, indicating that we deleted too many files somehow.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468221566



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {

Review comment:
       All fixed up




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466490938



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {

Review comment:
       Can we add Javadoc to the class and methods as we have in `RewriteManifestsAction`?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration

Review comment:
       nit: same here

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration

Review comment:
       nit: I am not sure whether it is a convention but we tend to add a space between `//` and the start of the comment.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());

Review comment:
       nit: extra space after `=`

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {

Review comment:
       `paths` -> `expiredFiles`? Just an idea. 

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       @rdblue, I know this action follows what we had before and uses `warn` level. How appropriate is that, though? I think `warn` indicates that there is an unusual situation or something went wrong. Here, that's expected, no?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();

Review comment:
       By default, this will use memory and disk. I am OK with this. In the future, we will replace it with reading from an old version file.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();

Review comment:
       I think this has to be wrapped into try-finally. We will have to always unpersist if a commit fails, for example.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);

Review comment:
       Can we add tests for validating all params? I know we simply delegate but I think it would be better to check.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] HeartSaVioR commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467036413



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,47 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();
+    for (Snapshot snapshot : table.snapshots()) {
+      String manifestListLocation = snapshot.manifestListLocation();
+      if (manifestListLocation != null) {
+        manifestLists.add(manifestListLocation);
+      }
+    }
+
+    return spark.createDataset(manifestLists, Encoders.STRING()).toDF("file_path");
+  }
+
+  protected Dataset<Row> buildOtherMetadataFileDF(SparkSession spark, TableOperations ops) {
+    List<String> otherMetadataFiles = Lists.newArrayList();

Review comment:
       Ditto.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to list all Manifest and DataFiles. This

Review comment:
       nit: MetadataTables -> metadata tables

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after the expiration. This operation will

Review comment:
       nit: Dataframes which are anti-joined

##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,47 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();

Review comment:
       Sounds like creating `manifestLists` can be placed in core module, so that it can be reused from action for other execution engine, or even without engine-specific action.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.

Review comment:
       "apply that" seems redundant.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done

Review comment:
       Probably it'd be nice to describe about expiring snapshots and removing obsolete files sequentially, not mixing up.

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,781 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+

Review comment:
       nit: double empty lines

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. Deletes are still performed locally after retrieving the results
+ * from the SparkExecutors.

Review comment:
       nit: Spark executors

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done

Review comment:
       nit: Shuffle -> shuffle, `so` -> `, and`




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468181239



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }

Review comment:
       done, rightAfterCheckpoint()




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468040906



##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {

Review comment:
       @HeartSaVioR Just to check with you, would it be ok if I moved the methods back into BaseAction? I know you wanted these to be back in the core module.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-671977593


   Thanks everyone! I know that was a big one


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468012448



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark

Review comment:
       nit: I think we can skip `org.apache.iceberg` in the doc as it is already imported.

##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {

Review comment:
       I am OK with the way it is done. An alternative would be to add more methods to `TableMetadata` directly. It has access to snapshots and previous metadata files. We don't have access to `version-hint.text`, though. Any thoughts, @rdblue? 

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       Sounds good. I know @RussellSpitzer has a separate PR for renaming.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  /**
+   * A specific snapshot to expire.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireSnapshotId(long)}
+   * @param expireSnapshotId Id of the snapshot to expire
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  /**
+   * Expire all snapshots older than a given timestamp.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireOlderThan(long)}
+   * @param timestampMillis all snapshots before this time will be expired
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  /**
+   * Retain at least x snapshots when expiring
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}
+   * @param numSnapshots number of snapshots to leave
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    Preconditions.checkArgument(1 <= numSnapshots,
+        "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  /**
+   * The Consumer used on files which have been determined to be expired. By default uses a filesystem delete.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#deleteWith(Consumer)}
+   * @param newDeleteFunc Consumer which takes a path and deletes it
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    Dataset<Row> originalFiles = null;
+    try {
+      // Metadata before Expiration
+      originalFiles = buildValidFileDF().persist();
+      // Action to trigger persist
+      originalFiles.count();
+
+      // Perform Expiration
+      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+      if (expireSnapshotIdValue != null) {
+        expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+      }
+      if (expireOlderThanValue != null) {
+        expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+      }
+      if (retainLastValue != null) {
+        expireSnaps = expireSnaps.retainLast(retainLastValue);
+      }
+      expireSnaps.commit();
+
+      // Metadata after Expiration
+      Dataset<Row> validFiles = buildValidFileDF();
+      Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+      ExpireSnapshotsActionResult result = deleteFiles(filesToDelete.toLocalIterator());

Review comment:
       nit: unnecessary variable, can simply return

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}

Review comment:
       `Similar` -> `similar` or we need `.` after `Spark action`.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same

Review comment:
       nit: `to to`

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}

Review comment:
       nit: `org.apache.iceberg.` can be dropped.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;

Review comment:
       This must be a list as `expireSnapshotId` can be called multiple times just like in `RemoveSnapshots`.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  /**
+   * A specific snapshot to expire.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireSnapshotId(long)}

Review comment:
       nit: `org.apache.iceberg.` can be dropped in all javadocs.

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;

Review comment:
       We will need a test that catches this as well.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467121883



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)

Review comment:
       These files are directly copied out of the Core Test Base, only modification is in the partitioning path




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-670255565


   Updated with test coverage and logging changes


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467121189



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {
+    AssertHelpers.assertThrows("Should fail retain 0 snapshots " +
+            "because number of snapshots to retain cannot be zero",
+        IllegalArgumentException.class,
+        "Number of snapshots to retain must be at least 1, cannot be: 0",
+        () -> Actions.forTable(table).expireSnapshots().retainLast(0).execute());
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotAppend() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {
+    AssertHelpers.assertThrows("Should fail retain 0 snapshots " +
+            "because number of snapshots to retain cannot be zero",
+        IllegalArgumentException.class,
+        "Number of snapshots to retain must be at least 1, cannot be: 0",
+        () -> Actions.forTable(table).expireSnapshots().retainLast(0).execute());
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotAppend() {
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotFastAppend() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {
+    AssertHelpers.assertThrows("Should fail retain 0 snapshots " +
+            "because number of snapshots to retain cannot be zero",
+        IllegalArgumentException.class,
+        "Number of snapshots to retain must be at least 1, cannot be: 0",
+        () -> Actions.forTable(table).expireSnapshots().retainLast(0).execute());
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotAppend() {
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotFastAppend() {
+    table.updateProperties()
+        .set(TableProperties.MANIFEST_MERGE_ENABLED, "true")
+        .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1")
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+  }
+
+  /**
+   * Test on table below, and expiring the staged commit `B` using `expireOlderThan` API.
+   * Table: A - C
+   *          ` B (staged)
+   */
+  @Test
+  public void testWithExpiringDanglingStageCommit() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {
+    AssertHelpers.assertThrows("Should fail retain 0 snapshots " +
+            "because number of snapshots to retain cannot be zero",
+        IllegalArgumentException.class,
+        "Number of snapshots to retain must be at least 1, cannot be: 0",
+        () -> Actions.forTable(table).expireSnapshots().retainLast(0).execute());
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotAppend() {
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotFastAppend() {
+    table.updateProperties()
+        .set(TableProperties.MANIFEST_MERGE_ENABLED, "true")
+        .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1")
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+  }
+
+  /**
+   * Test on table below, and expiring the staged commit `B` using `expireOlderThan` API.
+   * Table: A - C
+   *          ` B (staged)
+   */
+  @Test
+  public void testWithExpiringDanglingStageCommit() {
+    // `A` commit
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    // `B` staged commit
+    table.newAppend()
+        .appendFile(FILE_B)
+        .stageOnly()
+        .commit();
+
+    TableMetadata base = ((BaseTable) table).operations().current();
+    Snapshot snapshotA = base.snapshots().get(0);
+    Snapshot snapshotB = base.snapshots().get(1);
+
+    // `C` commit
+    table.newAppend()
+        .appendFile(FILE_C)
+        .commit();
+
+    Set<String> deletedFiles = new HashSet<>();
+
+    // Expire all commits including dangling staged snapshot.
+    Actions.forTable(table).expireSnapshots()
+        .deleteWith(deletedFiles::add)
+        .expireOlderThan(snapshotB.timestampMillis() + 1)
+        .execute();
+
+    Set<String> expectedDeletes = new HashSet<>();
+    expectedDeletes.add(snapshotA.manifestListLocation());
+
+    // Files should be deleted of dangling staged snapshot
+    snapshotB.addedFiles().forEach(i -> {
+      expectedDeletes.add(i.path().toString());
+    });
+
+    // ManifestList should be deleted too
+    expectedDeletes.add(snapshotB.manifestListLocation());
+    snapshotB.dataManifests().forEach(file -> {
+      //Only the manifest of B should be deleted.
+      if (file.snapshotId() == snapshotB.snapshotId()) {
+        expectedDeletes.add(file.path());
+      }
+    });
+    Assert.assertSame("Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size());
+    //Take the diff
+    expectedDeletes.removeAll(deletedFiles);
+    Assert.assertTrue("Exactly same files should be deleted", expectedDeletes.isEmpty());
+  }
+
+  /**
+   * Expire cherry-pick the commit as shown below, when `B` is in table's current state
+   *  Table:
+   *  A - B - C <--current snapshot
+   *   `- D (source=B)
+   */
+  @Test
+  public void testWithCherryPickTableSnapshot() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467094194



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.

Review comment:
       Thanks for all the doc review here!




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468038800



##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {

Review comment:
       This is okay, but I don't see much utility for these methods other than in the actions, and methods were already added to `BaseAction`. I would probably make these private methods in `BaseAction` instead of adding a utility class.
   
   I think that would be better because we usually try to have util methods either use internals (`TableOperations`, `TableMetadata`) or the public API (`Table`) and not mix the two. Rather than rewrite one of these methods to use `TableMetadata`, I'd just move them.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467121563



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .retainLast(1)
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainZeroSnapshots() {
+    AssertHelpers.assertThrows("Should fail retain 0 snapshots " +
+            "because number of snapshots to retain cannot be zero",
+        IllegalArgumentException.class,
+        "Number of snapshots to retain must be at least 1, cannot be: 0",
+        () -> Actions.forTable(table).expireSnapshots().retainLast(0).execute());
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotAppend() {
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+
+  }
+
+  @Test
+  public void testScanExpiredManifestInValidSnapshotFastAppend() {
+    table.updateProperties()
+        .set(TableProperties.MANIFEST_MERGE_ENABLED, "true")
+        .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1")
+        .commit();
+
+    table.newAppend()
+        .appendFile(FILE_A)
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newOverwrite()
+        .addFile(FILE_C)
+        .deleteFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_D)
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .deleteWith(deletedFiles::add)
+        .execute();
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+  }
+
+  /**
+   * Test on table below, and expiring the staged commit `B` using `expireOlderThan` API.
+   * Table: A - C
+   *          ` B (staged)
+   */
+  @Test
+  public void testWithExpiringDanglingStageCommit() {
+    // `A` commit
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    // `B` staged commit
+    table.newAppend()
+        .appendFile(FILE_B)
+        .stageOnly()
+        .commit();
+
+    TableMetadata base = ((BaseTable) table).operations().current();
+    Snapshot snapshotA = base.snapshots().get(0);
+    Snapshot snapshotB = base.snapshots().get(1);
+
+    // `C` commit
+    table.newAppend()
+        .appendFile(FILE_C)
+        .commit();
+
+    Set<String> deletedFiles = new HashSet<>();
+
+    // Expire all commits including dangling staged snapshot.
+    Actions.forTable(table).expireSnapshots()
+        .deleteWith(deletedFiles::add)
+        .expireOlderThan(snapshotB.timestampMillis() + 1)
+        .execute();
+
+    Set<String> expectedDeletes = new HashSet<>();
+    expectedDeletes.add(snapshotA.manifestListLocation());
+
+    // Files should be deleted of dangling staged snapshot
+    snapshotB.addedFiles().forEach(i -> {
+      expectedDeletes.add(i.path().toString());
+    });
+
+    // ManifestList should be deleted too
+    expectedDeletes.add(snapshotB.manifestListLocation());
+    snapshotB.dataManifests().forEach(file -> {
+      //Only the manifest of B should be deleted.
+      if (file.snapshotId() == snapshotB.snapshotId()) {
+        expectedDeletes.add(file.path());
+      }
+    });
+    Assert.assertSame("Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size());
+    //Take the diff
+    expectedDeletes.removeAll(deletedFiles);
+    Assert.assertTrue("Exactly same files should be deleted", expectedDeletes.isEmpty());
+  }
+
+  /**
+   * Expire cherry-pick the commit as shown below, when `B` is in table's current state
+   *  Table:
+   *  A - B - C <--current snapshot
+   *   `- D (source=B)
+   */
+  @Test
+  public void testWithCherryPickTableSnapshot() {
+    // `A` commit
+    table.newAppend()
+        .appendFile(FILE_A)
+        .commit();
+    Snapshot snapshotA = table.currentSnapshot();
+
+    // `B` commit
+    Set<String> deletedAFiles = new HashSet<>();
+    table.newOverwrite()
+        .addFile(FILE_B)
+        .deleteFile(FILE_A)
+        .deleteWith(deletedAFiles::add)
+        .commit();
+    Assert.assertTrue("No files should be physically deleted", deletedAFiles.isEmpty());
+
+    // pick the snapshot 'B`
+    Snapshot snapshotB = table.currentSnapshot();
+
+    // `C` commit to let cherry-pick take effect, and avoid fast-forward of `B` with cherry-pick
+    table.newAppend()
+        .appendFile(FILE_C)
+        .commit();
+    Snapshot snapshotC = table.currentSnapshot();
+
+    // Move the table back to `A`
+    table.manageSnapshots()
+        .setCurrentSnapshot(snapshotA.snapshotId())
+        .commit();
+
+    // Generate A -> `D (B)`
+    table.manageSnapshots()
+        .cherrypick(snapshotB.snapshotId())
+        .commit();
+    Snapshot snapshotD = table.currentSnapshot();
+
+    // Move the table back to `C`
+    table.manageSnapshots()
+        .setCurrentSnapshot(snapshotC.snapshotId())
+        .commit();
+    List<String> deletedFiles = new ArrayList<>();
+
+    // Expire `C`
+    Actions.forTable(table).expireSnapshots()
+        .deleteWith(deletedFiles::add)
+        .expireOlderThan(snapshotC.timestampMillis() + 1)
+        .execute();
+
+    // Make sure no dataFiles are deleted for the B, C, D snapshot
+    Lists.newArrayList(snapshotB, snapshotC, snapshotD).forEach(i -> {
+      i.addedFiles().forEach(item -> {
+        Assert.assertFalse(deletedFiles.contains(item.path().toString()));
+      });
+    });
+  }
+
+  /**
+   * Test on table below, and expiring `B` which is not in current table state.
+   *  1) Expire `B`
+   *  2) All commit
+   * Table: A - C - D (B)
+   *          ` B (staged)
+   */
+  @Test
+  public void testWithExpiringStagedThenCherrypick() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468171612



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsActionResult.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.actions;
+
+public class ExpireSnapshotsActionResult {
+
+  private final Long dataFilesDeleted;
+  private final Long manifestFilesDeleted;
+  private final Long manifestListsDeleted;
+
+  public ExpireSnapshotsActionResult(Long dataFilesDeleted, Long manifestFilesDeleted, Long manifestListsDeleted) {
+    this.dataFilesDeleted = dataFilesDeleted;
+    this.manifestFilesDeleted = manifestFilesDeleted;
+    this.manifestListsDeleted = manifestListsDeleted;
+  }
+
+  public Long getDataFilesDeleted() {

Review comment:
       sgtm




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466012991



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));
+  }
+
+  private Dataset<Row> getValidFileDF() {

Review comment:
       sgtm




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468040171



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * Similar to {@link org.apache.iceberg.ExpireSnapshots#executeWith(ExecutorService)}
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  /**
+   * A specific snapshot to expire.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireSnapshotId(long)}
+   * @param expireSnapshotId Id of the snapshot to expire
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  /**
+   * Expire all snapshots older than a given timestamp.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#expireOlderThan(long)}
+   * @param timestampMillis all snapshots before this time will be expired
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  /**
+   * Retain at least x snapshots when expiring
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#retainLast(int)}
+   * @param numSnapshots number of snapshots to leave
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    Preconditions.checkArgument(1 <= numSnapshots,
+        "Number of snapshots to retain must be at least 1, cannot be: %s", numSnapshots);
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  /**
+   * The Consumer used on files which have been determined to be expired. By default uses a filesystem delete.
+   * Identical to {@link org.apache.iceberg.ExpireSnapshots#deleteWith(Consumer)}
+   * @param newDeleteFunc Consumer which takes a path and deletes it
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    Dataset<Row> originalFiles = null;
+    try {
+      // Metadata before Expiration
+      originalFiles = buildValidFileDF().persist();
+      // Action to trigger persist
+      originalFiles.count();
+
+      // Perform Expiration
+      ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+      if (expireSnapshotIdValue != null) {
+        expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+      }
+      if (expireOlderThanValue != null) {
+        expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+      }
+      if (retainLastValue != null) {
+        expireSnaps = expireSnaps.retainLast(retainLastValue);
+      }
+      expireSnaps.commit();
+
+      // Metadata after Expiration
+      Dataset<Row> validFiles = buildValidFileDF();
+      Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+      ExpireSnapshotsActionResult result = deleteFiles(filesToDelete.toLocalIterator());
+      return result;
+    } finally {
+      if (originalFiles != null) {
+        originalFiles.unpersist();
+      }
+    }
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {

Review comment:
       Why does this exclude other data files? I would expect this to use valid files and valid metadata files, as returned by the base action.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-671633153


   @aokolnychyi + @rdblue all your comments have been addressed, thanks again for the helpful checks and discussion


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466512790



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       I like this name better than `executeWith` we have in `RemoveSnapshots`. Shall we update `RemoveSnapshots` too before we release it? @fbocse @rdblue, what do you think?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467091047



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,47 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+    String allDataFilesMetadataTable = metadataTableName(MetadataTableType.ALL_DATA_FILES);
+    return spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+  }
+
+  protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+    String allManifestsMetadataTable = metadataTableName(MetadataTableType.ALL_MANIFESTS);
+    return spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path as file_path");
+  }
+
+  protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+    List<String> manifestLists = Lists.newArrayList();

Review comment:
       Sure I can move these over to a new TableUtil class in the core module




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466561501



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       I think "debug" is the most appropriate.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468066866



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();

Review comment:
       One more question: it seems passing *any* executor service will trigger `runParallel` in `Tasks`.
   In turn, that will call ``` for (final I item : items)``` that I assume would load all files on the driver?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466004645



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.actions;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.util.Tasks;
+import org.apache.iceberg.util.ThreadPools;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private final SparkSession spark;
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final TableOperations ops;
+  private final ExpireSnapshots localExpireSnapshots;
+  private final TableMetadata base;
+  private static final String DATAFILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFESTLIST = "Manifest List";
+  private static final String OTHER = "Other";
+
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+  private Consumer<String> deleteFunc = defaultDelete;
+
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+    this.base = ops.current();
+    this.localExpireSnapshots = table.expireSnapshots().cleanExpiredFiles(false);
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    localExpireSnapshots.expireSnapshotId(expireSnapshotId);
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    localExpireSnapshots.expireOlderThan(timestampMillis);
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    localExpireSnapshots.retainLast(numSnapshots);
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("DataFile"));

Review comment:
       Leftover from a prior version, I never call out the column by name again so I didn't change the column name when I changed the design




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-671660367


   I'd prefer if a couple methods were private, but that isn't important enough to block committing. I'll merge this.


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466568641



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();

Review comment:
       The commit in `ExpireSnapshots` can be unsuccessful. In that case, we should remove the cached state as we no longer need it.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467120726



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction

##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots, but explicitly remove the first snapshot
+    Actions.forTable(table).expireSnapshots()
+        .expireSnapshotId(firstSnapshotId)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithTooFewSnapshots() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    // Retain last 3 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t2)
+        .retainLast(3)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should still present",
+        firstSnapshotId, table.snapshot(firstSnapshotId).snapshotId());
+  }
+
+  @Test
+  public void testRetainLastKeepsExpiringSnapshot() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_D) // data_bucket=3
+        .commit();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have three snapshots.",
+        3, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNotNull("Second snapshot should present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testExpireOlderThanMultipleCalls() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    Snapshot thirdSnapshot = table.currentSnapshot();
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots and expire older than t3
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(secondSnapshot.timestampMillis())
+        .expireOlderThan(thirdSnapshot.timestampMillis())
+        .execute();
+
+    Assert.assertEquals("Should have one snapshots.",
+        1, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertNull("Second snapshot should not present.",
+        table.snapshot(secondSnapshot.snapshotId()));
+  }
+
+  @Test
+  public void testRetainLastMultipleCalls() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468035503



##########
File path: core/src/main/java/org/apache/iceberg/util/TableUtil.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class TableUtil {
+
+  private TableUtil(){}

Review comment:
       Nit: formatting here is off. We always add a space between method args and curly braces.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119652



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {

Review comment:
       New Test - Not Ported




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] HeartSaVioR commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-670546640


   I see the latest commit title now, `Port over all the tests from TestRemoveSnapshots`.
   
   Could you please leave some guidance comments in TestExpireSnapshotsAction so that reviewers can compare with the origin code of TestRemoveSnapshots or even skip reviewing some methods if the content is identical? Some applies on refactored code, but I already passed through the code so OK.


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466556043



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       I find the current behavior in `RemoveSnapshots` confusing. If I set an executor service with `executeWith`, I'd expect that to be used for all things since the name is generic. That does not happen because manifests are scanned using `ThreadPools.getWorkerPool()`.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468233968



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration table metadata. All of the same
+ * restrictions of Remove Snapshots also apply to this action.
+ * <p>
+ * This implementation uses the metadata tables for the table being expired to list all Manifest and DataFiles. This
+ * is made into a Dataframe which are anti-joined with the same list read after the expiration. This operation will
+ * require a shuffle so parallelism can be controlled through spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. The snapshot expiration will be fully committed before any deletes
+ * are issued. Deletes are still performed locally after retrieving the results from the Spark executors.
+ */
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();

Review comment:
       We decided to shift this to "null" so that we will use the single threaded execution path by default and we can figure out a non-eager iterator approach later




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-670016798


   cc @rdsr @prodeezy 


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467120666



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    for (int i = 0; i < 10; i++) {
+      writeDF(df, "overwrite");
+    }
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    checkExpirationResults(10L, 19L, 10L, results);
+  }
+
+  @Test
+  public void testRetainLastWithExpireOlderThan() {
+    long t0 = System.currentTimeMillis();
+    table.newAppend()
+        .appendFile(FILE_A) // data_bucket=0
+        .commit();
+    long firstSnapshotId = table.currentSnapshot().snapshotId();
+    long t1 = System.currentTimeMillis();
+    while (t1 <= table.currentSnapshot().timestampMillis()) {
+      t1 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_B) // data_bucket=1
+        .commit();
+
+    long t2 = System.currentTimeMillis();
+    while (t2 <= table.currentSnapshot().timestampMillis()) {
+      t2 = System.currentTimeMillis();
+    }
+
+    table.newAppend()
+        .appendFile(FILE_C) // data_bucket=2
+        .commit();
+
+    long t3 = System.currentTimeMillis();
+    while (t3 <= table.currentSnapshot().timestampMillis()) {
+      t3 = System.currentTimeMillis();
+    }
+
+    // Retain last 2 snapshots
+    Actions.forTable(table).expireSnapshots()
+        .expireOlderThan(t3)
+        .retainLast(2)
+        .execute();
+
+    Assert.assertEquals("Should have two snapshots.",
+        2, Lists.newArrayList(table.snapshots()).size());
+    Assert.assertEquals("First snapshot should not present.",
+        null, table.snapshot(firstSnapshotId));
+  }
+
+  @Test
+  public void testRetainLastWithExpireById() {

Review comment:
       Ported Test - Only change is using ExpireSnapshotsAction




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] mehtaashish23 commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
mehtaashish23 commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466529752



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);

Review comment:
       Shouldn't this be `trace` as well?

##########
File path: spark2/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction24.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.actions;
+
+public class TestExpireSnapshotsAction24 extends TestExpireSnapshotsAction{

Review comment:
       Unnecessary file?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       I wonder `executeDeleteWith` would become very specific to only use thread-pool for delete operation. What if we have some task in the operation, where the same thread pool can be re-used.  

##########
File path: spark3/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction3.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.spark.actions;
+
+import org.apache.iceberg.actions.TestExpireSnapshotsAction;
+
+public class TestExpireSnapshotsAction3 extends TestExpireSnapshotsAction {

Review comment:
       same: Unnecessary files?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466619159



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {

Review comment:
       Of course, I've set up most of the docs to link back to the ExpireSnapshots interface




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue merged pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264


   


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466556043



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {

Review comment:
       I find the current behavior in `RemoveSnapshots` confusing. If I set an executor service with `executeWith`, I'd expect that to be used for all things since the name is generic. That does not happen because manifests are scanned using `ThreadPools.getWorkerPool()` and the passed executor service is used only for deletes.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466562498



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);

Review comment:
       This mimics the RemoveSnapshot logic, which uses warn




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119880



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().execute();
+
+    checkExpirationResults(0L, 0L, 0L, results);
+  }
+
+  @Test
+  public void testCleanupRepeatedOverwrites() throws Exception {

Review comment:
       New test - Not Ported




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468261726



##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +50,66 @@ protected String metadataTableName(MetadataTableType type) {
       return tableName + "." + type;
     }
   }
+
+  /**
+   * Returns all the path locations of all Manifest Lists for a given table
+   * @param table the table
+   * @return the paths of the Manifest Lists
+   */
+  protected List<String> getManifestListPaths(Table table) {
+    List<String> manifestLists = Lists.newArrayList();
+    for (Snapshot snapshot : table.snapshots()) {
+      String manifestListLocation = snapshot.manifestListLocation();
+      if (manifestListLocation != null) {
+        manifestLists.add(manifestListLocation);
+      }
+    }
+    return manifestLists;
+  }
+
+  /**
+   * Returns all Metadata file paths which may not be in the current metadata. Specifically
+   * this includes "version-hint" files as well as entries in metadata.previousFiles.
+   * @param ops TableOperations for the table we will be getting paths from
+   * @return a list of paths to metadata files
+   */
+  protected List<String> getOtherMetadataFilePaths(TableOperations ops) {

Review comment:
       Same here. Since we aren't making these common utility methods, we should keep them private so we aren't committing to support them later on.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468052126



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();

Review comment:
       The table instance is used for the expire commit, right? If that's the case, then there should be no need to refresh here.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466563218



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction3.java
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.spark.actions;
+
+import org.apache.iceberg.actions.TestExpireSnapshotsAction;
+
+public class TestExpireSnapshotsAction3 extends TestExpireSnapshotsAction {

Review comment:
       Same as above, implementation of the TestExpireSnapshotsAction but within the Spark3 suite




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r468049031



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }

Review comment:
       Should this be a util method?




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466596580



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       We will need more context from @rdblue on `RemoveSnapshots`.




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#issuecomment-670064241


   We've been talking about the testing issue for a while, one of the biggest issues is that we cannot use the same code from TestRemoveSnapshots because it all relies on tons of inner class parameters and fields so it's extremely difficult (at least from my perspective) to setup the cherry pick cases. Even if we wanted to use the base class from spark-core for testing, we would then be unable to inherit from the Spark Base Action test class. 
   
   I'll take a look at getting in alternative versions of these tests


----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467119770



##########
File path: spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,780 @@
+/*
+ * 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.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+  private static final HadoopTables TABLES = new HadoopTables(new Configuration());
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "c1", Types.IntegerType.get()),
+      optional(2, "c2", Types.StringType.get()),
+      optional(3, "c3", Types.StringType.get())
+  );
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+  private static final List<ThreeColumnRecord> RECORDS = Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+  static final DataFile FILE_A = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-a.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=0") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_B = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-b.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=1") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_C = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-c.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=2") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+  static final DataFile FILE_D = DataFiles.builder(SPEC)
+      .withPath("/path/to/data-d.parquet")
+      .withFileSizeInBytes(10)
+      .withPartitionPath("c1=3") // easy way to set partition data for now
+      .withRecordCount(1)
+      .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private File tableDir;
+  private String tableLocation;
+  private Table table;
+
+  @Before
+  public void setupTableLocation() throws Exception {
+    this.tableDir = temp.newFolder();
+    this.tableLocation = tableDir.toURI().toString();
+    this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+  }
+
+  private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+    return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+  }
+
+  private void writeDF(Dataset<Row> df, String mode) {
+    df.select("c1", "c2", "c3")
+        .write()
+        .format("iceberg")
+        .mode(mode)
+        .save(tableLocation);
+  }
+
+  private void checkExpirationResults(Long expectedDatafiles, Long expectedManifestsDeleted,
+      Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+    Assert.assertEquals("Incorrect number of manifest files deleted",
+        expectedManifestsDeleted, results.getManifestFilesDeleted());
+    Assert.assertEquals("Incorrect number of datafiles deleted",
+        expectedDatafiles, results.getDataFilesDeleted());
+    Assert.assertEquals("Incorrect number of manifest lists deleted",
+        expectedManifestListsDeleted, results.getManifestListsDeleted());
+  }
+
+  @Test
+  public void testFilesCleaned() throws Exception {
+    Dataset<Row> df = buildDF(RECORDS);
+
+    writeDF(df, "append");
+
+    List<Path> expiredDataFiles = Files
+        .list(tableDir.toPath().resolve("data").resolve("c1=1"))
+        .collect(Collectors.toList());
+
+    Assert.assertEquals("There should be a data file to delete but there was none.",
+        2, expiredDataFiles.size());
+
+    writeDF(df, "overwrite");
+    writeDF(df, "append");
+
+    long end = System.currentTimeMillis();
+    while (end <= table.currentSnapshot().timestampMillis()) {
+      end = System.currentTimeMillis();
+    }
+
+    ExpireSnapshotsActionResult results =
+        Actions.forTable(table).expireSnapshots().expireOlderThan(end).execute();
+
+    table.refresh();
+
+    Assert.assertEquals("Table does not have 1 snapshot after expiration", 1, Iterables.size(table.snapshots()));
+
+    for (Path p : expiredDataFiles) {
+      Assert.assertFalse(String.format("File %s still exists but should have been deleted", p),
+          Files.exists(p));
+    }
+
+    checkExpirationResults(1L, 2L, 2L, results);
+  }
+
+  @Test
+  public void dataFilesCleanupWithParallelTasks() throws IOException {
+
+    table.newFastAppend()
+        .appendFile(FILE_A)
+        .commit();
+
+    table.newFastAppend()
+        .appendFile(FILE_B)
+        .commit();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D))
+        .commit();
+    long thirdSnapshotId = table.currentSnapshot().snapshotId();
+
+    table.newRewrite()
+        .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C))
+        .commit();
+    long fourthSnapshotId = table.currentSnapshot().snapshotId();
+
+    long t4 = System.currentTimeMillis();
+    while (t4 <= table.currentSnapshot().timestampMillis()) {
+      t4 = System.currentTimeMillis();
+    }
+
+    Set<String> deletedFiles = Sets.newHashSet();
+    Set<String> deleteThreads = ConcurrentHashMap.newKeySet();
+    AtomicInteger deleteThreadsIndex = new AtomicInteger(0);
+
+    Actions.forTable(table).expireSnapshots()
+        .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> {
+          Thread thread = new Thread(runnable);
+          thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement());
+          thread.setDaemon(true); // daemon threads will be terminated abruptly when the JVM exits
+          return thread;
+        }))
+        .expireOlderThan(t4)
+        .deleteWith(s -> {
+          deleteThreads.add(Thread.currentThread().getName());
+          deletedFiles.add(s);
+        })
+        .execute();
+
+    // Verifies that the delete methods ran in the threads created by the provided ExecutorService ThreadFactory
+    Assert.assertEquals(deleteThreads,
+        Sets.newHashSet("remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"));
+
+    Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString()));
+    Assert.assertTrue("FILE_B should be deleted", deletedFiles.contains(FILE_B.path().toString()));
+  }
+
+  @Test
+  public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception {

Review comment:
       New test - Not Ported




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466621380



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {

Review comment:
       sgtm




----------------------------------------------------------------
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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1264: Expire snapshots action

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r466524445



##########
File path: spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ExpireSnapshotsAction extends BaseAction<ExpireSnapshotsActionResult> {
+  private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsAction.class);
+
+  private static final String DATA_FILE = "Data File";
+  private static final String MANIFEST = "Manifest";
+  private static final String MANIFEST_LIST = "Manifest List";
+
+  // Creates an executor service that runs each task in the thread that invokes execute/submit.
+  private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = MoreExecutors.newDirectExecutorService();
+
+  private final SparkSession spark;
+  private final Table table;
+  private final TableOperations ops;
+  private final Consumer<String> defaultDelete = new Consumer<String>() {
+    @Override
+    public void accept(String file) {
+      ops.io().deleteFile(file);
+    }
+  };
+
+  private Long expireSnapshotIdValue = null;
+  private Long expireOlderThanValue = null;
+  private Integer retainLastValue = null;
+  private Consumer<String> deleteFunc = defaultDelete;
+  private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE;
+
+  ExpireSnapshotsAction(SparkSession spark, Table table) {
+    this.spark = spark;
+    this.table = table;
+    this.ops = ((HasTableOperations) table).operations();
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  /**
+   * An executor service used when deleting files. Only used during the local delete phase of this Spark action
+   * @param executorService the service to use
+   * @return this for method chaining
+   */
+  public ExpireSnapshotsAction executeDeleteWith(ExecutorService executorService) {
+    this.deleteExecutorService = executorService;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireSnapshotId(long expireSnapshotId) {
+    this.expireSnapshotIdValue = expireSnapshotId;
+    return this;
+  }
+
+  public ExpireSnapshotsAction expireOlderThan(long timestampMillis) {
+    this.expireOlderThanValue = timestampMillis;
+    return this;
+  }
+
+  public ExpireSnapshotsAction retainLast(int numSnapshots) {
+    this.retainLastValue = numSnapshots;
+    return this;
+  }
+
+  public ExpireSnapshotsAction deleteWith(Consumer<String> newDeleteFunc) {
+    this.deleteFunc = newDeleteFunc;
+    return this;
+  }
+
+  @Override
+  public ExpireSnapshotsActionResult execute() {
+    //Metadata before Expiration
+    Dataset<Row> originalFiles = buildValidFileDF().persist();
+    originalFiles.count(); // Action to trigger persist
+
+    //Perform Expiration
+    ExpireSnapshots expireSnaps = table.expireSnapshots().cleanExpiredFiles(false);
+    if (expireSnapshotIdValue != null) {
+      expireSnaps = expireSnaps.expireSnapshotId(expireSnapshotIdValue);
+    }
+    if (expireOlderThanValue != null) {
+      expireSnaps = expireSnaps.expireOlderThan(expireOlderThanValue);
+    }
+    if (retainLastValue != null) {
+      expireSnaps = expireSnaps.retainLast(retainLastValue);
+    }
+    expireSnaps.commit();
+
+    // Metadata after Expiration
+    Dataset<Row> validFiles = buildValidFileDF();
+    Dataset<Row> filesToDelete = originalFiles.except(validFiles);
+
+    ExpireSnapshotsActionResult result =  deleteFiles(filesToDelete.toLocalIterator());
+    originalFiles.unpersist();
+    return result;
+  }
+
+  private Dataset<Row> appendTypeString(Dataset<Row> ds, String type) {
+    return ds.select(new Column("file_path"), functions.lit(type).as("file_type"));
+  }
+
+  private Dataset<Row> buildValidFileDF() {
+    return appendTypeString(buildValidDataFileDF(spark), DATA_FILE)
+        .union(appendTypeString(buildManifestFileDF(spark), MANIFEST))
+        .union(appendTypeString(buildManifestListDF(spark, table), MANIFEST_LIST));
+  }
+
+  private ExpireSnapshotsActionResult deleteFiles(Iterator<Row> paths) {
+    AtomicLong dataFileCount = new AtomicLong(0L);
+    AtomicLong manifestCount = new AtomicLong(0L);
+    AtomicLong manifestListCount = new AtomicLong(0L);
+
+    Tasks.foreach(paths)
+        .retry(3).stopRetryOn(NotFoundException.class).suppressFailureWhenFinished()
+        .executeWith(deleteExecutorService)
+        .onFailure((fileInfo, exc) ->
+            LOG.warn("Delete failed for {}: {}", fileInfo.getString(1), fileInfo.getString(0), exc))
+        .run(fileInfo -> {
+          String file = fileInfo.getString(0);
+          String type = fileInfo.getString(1);
+          deleteFunc.accept(file);
+          switch (type) {
+            case DATA_FILE:
+              dataFileCount.incrementAndGet();
+              LOG.trace("Deleted Data File: {}", file);
+              break;
+            case MANIFEST:
+              manifestCount.incrementAndGet();
+              LOG.warn("Deleted Manifest: {}", file);
+              break;
+            case MANIFEST_LIST:
+              manifestListCount.incrementAndGet();
+              LOG.warn("Deleted Manifest List: {}", file);
+              break;
+          }
+        });
+    LOG.warn("Deleted {} total files", dataFileCount.get() + manifestCount.get() + manifestListCount.get());

Review comment:
       I was honestly confused as to why this, and the log manifest and manifest lists lines were all "warn" and not "debug" or "info"




----------------------------------------------------------------
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.

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



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