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/12/04 11:52:49 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request #1874: Expire snapshots procedure

aokolnychyi opened a new pull request #1874:
URL: https://github.com/apache/iceberg/pull/1874


   This PR adds a stored procedure to expire snapshots.
   
   The main author of this change is @liukun4515. I took commits from #1819, rebased, and added more changes to match the current state of procedures.
   
   Resolves #1597.


----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.extensions;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+
+public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase {
+
+  public TestExpireSnapshotsProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testExpireSnapshotsInEmptyTable() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name());
+    assertEquals("Should not delete any files", ImmutableList.of(row(0L, 0L, 0L)), output);
+  }
+
+  @Test
+  public void testExpireSnapshotsUsingPositionalArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot firstSnapshot = table.currentSnapshot();
+
+    waitUntilAfter(firstSnapshot.timestampMillis());
+
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis()));
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    // expire without retainLast param
+    List<Object[]> output1 = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name(), secondSnapshotTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output1);
+
+    table.refresh();
+
+    Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots()));
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName);
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(3L, "c"), row(4L, "d")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    table.refresh();
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    // expire with retainLast param
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 2)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(2L, 2L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotUsingNamedArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 1)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotsGCDisabled() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED);
+
+    AssertHelpers.assertThrows("Should reject call",
+        ValidationException.class, "Cannot expire snapshots: GC is disabled",
+        () -> sql("CALL %s.system.expire_snapshots('%s', '%s')", catalogName,
+            tableIdent.namespace(), tableIdent.name()));
+  }
+
+  @Test
+  public void testInvalidExpireSnapshotsCases() {
+    AssertHelpers.assertThrows("Should not allow mixed args",

Review comment:
       Not sure we need some of the more generic checks here, like testing if named and pos args cannot be mixed




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.procedures;
+
+import org.apache.iceberg.actions.Actions;
+import org.apache.iceberg.actions.ExpireSnapshotsAction;
+import org.apache.iceberg.actions.ExpireSnapshotsActionResult;
+import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.util.DateTimeUtils;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * A procedure that expires snapshots in a table.
+ *
+ * @see Actions#expireSnapshots()
+ */
+public class ExpireSnapshotsProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] {
+      ProcedureParameter.required("namespace", DataTypes.StringType),
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.optional("older_than", DataTypes.TimestampType),
+      ProcedureParameter.optional("retain_last", DataTypes.IntegerType)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("num_deleted_data_files", DataTypes.LongType, true, Metadata.empty()),
+      new StructField("num_deleted_manifest_files", DataTypes.LongType, true, Metadata.empty()),
+      new StructField("num_deleted_manifest_lists", DataTypes.LongType, true, Metadata.empty())

Review comment:
       Done.




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.extensions;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+
+public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase {
+
+  public TestExpireSnapshotsProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testExpireSnapshotsInEmptyTable() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name());
+    assertEquals("Should not delete any files", ImmutableList.of(row(0L, 0L, 0L)), output);
+  }
+
+  @Test
+  public void testExpireSnapshotsUsingPositionalArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot firstSnapshot = table.currentSnapshot();
+
+    waitUntilAfter(firstSnapshot.timestampMillis());
+
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis()));
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    // expire without retainLast param
+    List<Object[]> output1 = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name(), secondSnapshotTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output1);
+
+    table.refresh();
+
+    Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots()));
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName);
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(3L, "c"), row(4L, "d")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    table.refresh();
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    // expire with retainLast param
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 2)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(2L, 2L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotUsingNamedArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 1)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotsGCDisabled() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED);
+
+    AssertHelpers.assertThrows("Should reject call",
+        ValidationException.class, "Cannot expire snapshots: GC is disabled",
+        () -> sql("CALL %s.system.expire_snapshots('%s', '%s')", catalogName,
+            tableIdent.namespace(), tableIdent.name()));
+  }
+
+  @Test
+  public void testInvalidExpireSnapshotsCases() {
+    AssertHelpers.assertThrows("Should not allow mixed args",

Review comment:
       Sounds fine, It doesn't really matter since they run so quickly.




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.extensions;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+
+public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase {
+
+  public TestExpireSnapshotsProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testExpireSnapshotsInEmptyTable() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name());
+    assertEquals("Should not delete any files", ImmutableList.of(row(0L, 0L, 0L)), output);
+  }
+
+  @Test
+  public void testExpireSnapshotsUsingPositionalArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot firstSnapshot = table.currentSnapshot();
+
+    waitUntilAfter(firstSnapshot.timestampMillis());
+
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis()));
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    // expire without retainLast param
+    List<Object[]> output1 = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name(), secondSnapshotTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output1);
+
+    table.refresh();
+
+    Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots()));
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName);
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(3L, "c"), row(4L, "d")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    table.refresh();
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    // expire with retainLast param
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 2)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(2L, 2L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotUsingNamedArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 1)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotsGCDisabled() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED);
+
+    AssertHelpers.assertThrows("Should reject call",
+        ValidationException.class, "Cannot expire snapshots: GC is disabled",
+        () -> sql("CALL %s.system.expire_snapshots('%s', '%s')", catalogName,
+            tableIdent.namespace(), tableIdent.name()));
+  }
+
+  @Test
+  public void testInvalidExpireSnapshotsCases() {
+    AssertHelpers.assertThrows("Should not allow mixed args",

Review comment:
       It is a bit weird but we have such checks in other procedures so I added for consistency. 




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java
##########
@@ -96,6 +75,36 @@ public StructType outputType() {
     return OUTPUT_TYPE;
   }
 
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String namespace = args.getString(0);
+    String tableName = args.getString(1);
+    Long olderThanMillis = args.isNullAt(2) ? null : DateTimeUtils.toMillis(args.getLong(2));
+    Integer retainLastNum = args.isNullAt(3) ? null : args.getInt(3);
+
+    return modifyIcebergTable(namespace, tableName, table -> {
+      Actions actions = Actions.forTable(table);
+
+      ExpireSnapshotsAction action = actions.expireSnapshots();
+
+      if (olderThanMillis != null) {
+        action.expireOlderThan(olderThanMillis);

Review comment:
       Ah! Excellent, I forgot about that
   




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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


   cc @RussellSpitzer @liukun4515 @rdblue 


----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.extensions;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+
+public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase {
+
+  public TestExpireSnapshotsProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testExpireSnapshotsInEmptyTable() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name());
+    assertEquals("Should not delete any files", ImmutableList.of(row(0L, 0L, 0L)), output);
+  }
+
+  @Test
+  public void testExpireSnapshotsUsingPositionalArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot firstSnapshot = table.currentSnapshot();
+
+    waitUntilAfter(firstSnapshot.timestampMillis());
+
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis()));
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    // expire without retainLast param
+    List<Object[]> output1 = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name(), secondSnapshotTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output1);
+
+    table.refresh();
+
+    Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots()));
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName);
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(3L, "c"), row(4L, "d")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    table.refresh();
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    // expire with retainLast param
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 2)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(2L, 2L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotUsingNamedArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 1)",

Review comment:
       Named args?




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.procedures;
+
+import org.apache.iceberg.actions.Actions;
+import org.apache.iceberg.actions.ExpireSnapshotsAction;
+import org.apache.iceberg.actions.ExpireSnapshotsActionResult;
+import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.util.DateTimeUtils;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * A procedure that expires snapshots in a table.
+ *
+ * @see Actions#expireSnapshots()
+ */
+public class ExpireSnapshotsProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] {
+      ProcedureParameter.required("namespace", DataTypes.StringType),
+      ProcedureParameter.required("table", DataTypes.StringType),
+      ProcedureParameter.optional("older_than", DataTypes.TimestampType),
+      ProcedureParameter.optional("retain_last", DataTypes.IntegerType)
+  };
+
+  private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{
+      new StructField("num_deleted_data_files", DataTypes.LongType, true, Metadata.empty()),
+      new StructField("num_deleted_manifest_files", DataTypes.LongType, true, Metadata.empty()),
+      new StructField("num_deleted_manifest_lists", DataTypes.LongType, true, Metadata.empty())

Review comment:
       Don't we usually name columns like this with `_count` instead of `num_`? I would expect `deleted_data_files_count`, like we have `added_data_files_count` in manifests or `record_count` in data 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] aokolnychyi commented on a change in pull request #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java
##########
@@ -96,6 +75,36 @@ public StructType outputType() {
     return OUTPUT_TYPE;
   }
 
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String namespace = args.getString(0);
+    String tableName = args.getString(1);
+    Long olderThanMillis = args.isNullAt(2) ? null : DateTimeUtils.toMillis(args.getLong(2));
+    Integer retainLastNum = args.isNullAt(3) ? null : args.getInt(3);
+
+    return modifyIcebergTable(namespace, tableName, table -> {
+      Actions actions = Actions.forTable(table);
+
+      ExpireSnapshotsAction action = actions.expireSnapshots();
+
+      if (olderThanMillis != null) {
+        action.expireOlderThan(olderThanMillis);

Review comment:
       The action delegates to `ExpireSnapshots` table API that, in turn, respects the default table props we added recently. 




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.extensions;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+
+public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase {
+
+  public TestExpireSnapshotsProcedure(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testExpireSnapshotsInEmptyTable() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name());
+    assertEquals("Should not delete any files", ImmutableList.of(row(0L, 0L, 0L)), output);
+  }
+
+  @Test
+  public void testExpireSnapshotsUsingPositionalArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot firstSnapshot = table.currentSnapshot();
+
+    waitUntilAfter(firstSnapshot.timestampMillis());
+
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot secondSnapshot = table.currentSnapshot();
+    Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis()));
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    // expire without retainLast param
+    List<Object[]> output1 = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s')",
+        catalogName, tableIdent.namespace(), tableIdent.name(), secondSnapshotTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(0L, 0L, 1L)),
+        output1);
+
+    table.refresh();
+
+    Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots()));
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName);
+    assertEquals("Should have expected rows",
+        ImmutableList.of(row(3L, "c"), row(4L, "d")),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    table.refresh();
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    // expire with retainLast param
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 2)",
+        catalogName, tableIdent.namespace(), tableIdent.name(), currentTimestamp);
+    assertEquals("Procedure output must match",
+        ImmutableList.of(row(2L, 2L, 1L)),
+        output);
+  }
+
+  @Test
+  public void testExpireSnapshotUsingNamedArgs() {
+    sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName);
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots()));
+
+    waitUntilAfter(table.currentSnapshot().timestampMillis());
+
+    Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));
+
+    List<Object[]> output = sql(
+        "CALL %s.system.expire_snapshots('%s', '%s', TIMESTAMP '%s', 1)",

Review comment:
       Done.




----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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


   


----------------------------------------------------------------
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 #1874: Spark: Add ExpireSnapshotsProcedure

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java
##########
@@ -96,6 +75,36 @@ public StructType outputType() {
     return OUTPUT_TYPE;
   }
 
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String namespace = args.getString(0);
+    String tableName = args.getString(1);
+    Long olderThanMillis = args.isNullAt(2) ? null : DateTimeUtils.toMillis(args.getLong(2));
+    Integer retainLastNum = args.isNullAt(3) ? null : args.getInt(3);
+
+    return modifyIcebergTable(namespace, tableName, table -> {
+      Actions actions = Actions.forTable(table);
+
+      ExpireSnapshotsAction action = actions.expireSnapshots();
+
+      if (olderThanMillis != null) {
+        action.expireOlderThan(olderThanMillis);

Review comment:
       So we are going with the action default expire time? Just checking because I thougt some folks wanted that changed




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