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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to