HeartSaVioR commented on a change in pull request #1264:
URL: https://github.com/apache/iceberg/pull/1264#discussion_r467036413
##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,47 @@ protected String metadataTableName(MetadataTableType type) {
return tableName + "." + type;
}
}
+
+ protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+ String allDataFilesMetadataTable =
metadataTableName(MetadataTableType.ALL_DATA_FILES);
+ return
spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+ }
+
+ protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+ String allManifestsMetadataTable =
metadataTableName(MetadataTableType.ALL_MANIFESTS);
+ return
spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path
as file_path");
+ }
+
+ protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+ List<String> manifestLists = Lists.newArrayList();
+ for (Snapshot snapshot : table.snapshots()) {
+ String manifestListLocation = snapshot.manifestListLocation();
+ if (manifestListLocation != null) {
+ manifestLists.add(manifestListLocation);
+ }
+ }
+
+ return spark.createDataset(manifestLists,
Encoders.STRING()).toDF("file_path");
+ }
+
+ protected Dataset<Row> buildOtherMetadataFileDF(SparkSession spark,
TableOperations ops) {
+ List<String> otherMetadataFiles = Lists.newArrayList();
Review comment:
Ditto.
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to
list all Manifest and DataFiles. This
Review comment:
nit: MetadataTables -> metadata tables
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to
list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after
the expiration. This operation will
Review comment:
nit: Dataframes which are anti-joined
##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -41,4 +51,47 @@ protected String metadataTableName(MetadataTableType type) {
return tableName + "." + type;
}
}
+
+ protected Dataset<Row> buildValidDataFileDF(SparkSession spark) {
+ String allDataFilesMetadataTable =
metadataTableName(MetadataTableType.ALL_DATA_FILES);
+ return
spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+ }
+
+ protected Dataset<Row> buildManifestFileDF(SparkSession spark) {
+ String allManifestsMetadataTable =
metadataTableName(MetadataTableType.ALL_MANIFESTS);
+ return
spark.read().format("iceberg").load(allManifestsMetadataTable).selectExpr("path
as file_path");
+ }
+
+ protected Dataset<Row> buildManifestListDF(SparkSession spark, Table table) {
+ List<String> manifestLists = Lists.newArrayList();
Review comment:
Sounds like creating `manifestLists` can be placed in core module, so
that it can be reused from action for other execution engine, or even without
engine-specific action.
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
Review comment:
"apply that" seems redundant.
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to
list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after
the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through
spark.sql.shuffle.partitions. The expiration is done
Review comment:
Probably it'd be nice to describe about expiring snapshots and removing
obsolete files sequentially, not mixing up.
##########
File path:
spark/src/test/java/org/apache/iceberg/actions/TestExpireSnapshotsAction.java
##########
@@ -0,0 +1,781 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.SparkTestBase;
+import org.apache.iceberg.spark.source.ThreeColumnRecord;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestExpireSnapshotsAction extends SparkTestBase {
+
+ private static final HadoopTables TABLES = new HadoopTables(new
Configuration());
+ private static final Schema SCHEMA = new Schema(
+ optional(1, "c1", Types.IntegerType.get()),
+ optional(2, "c2", Types.StringType.get()),
+ optional(3, "c3", Types.StringType.get())
+ );
+
+ private static final PartitionSpec SPEC =
PartitionSpec.builderFor(SCHEMA).identity("c1").build();
+
+ private static final List<ThreeColumnRecord> RECORDS =
Lists.newArrayList(new ThreeColumnRecord(1, "AAAA", "AAAA"));
+
+ static final DataFile FILE_A = DataFiles.builder(SPEC)
+ .withPath("/path/to/data-a.parquet")
+ .withFileSizeInBytes(10)
+ .withPartitionPath("c1=0") // easy way to set partition data for now
+ .withRecordCount(1)
+ .build();
+ static final DataFile FILE_B = DataFiles.builder(SPEC)
+ .withPath("/path/to/data-b.parquet")
+ .withFileSizeInBytes(10)
+ .withPartitionPath("c1=1") // easy way to set partition data for now
+ .withRecordCount(1)
+ .build();
+ static final DataFile FILE_C = DataFiles.builder(SPEC)
+ .withPath("/path/to/data-c.parquet")
+ .withFileSizeInBytes(10)
+ .withPartitionPath("c1=2") // easy way to set partition data for now
+ .withRecordCount(1)
+ .build();
+ static final DataFile FILE_D = DataFiles.builder(SPEC)
+ .withPath("/path/to/data-d.parquet")
+ .withFileSizeInBytes(10)
+ .withPartitionPath("c1=3") // easy way to set partition data for now
+ .withRecordCount(1)
+ .build();
+
+ @Rule
+ public TemporaryFolder temp = new TemporaryFolder();
+
+ private File tableDir;
+ private String tableLocation;
+ private Table table;
+
+ @Before
+ public void setupTableLocation() throws Exception {
+ this.tableDir = temp.newFolder();
+ this.tableLocation = tableDir.toURI().toString();
+ this.table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation);
+ }
+
+ private Dataset<Row> buildDF(List<ThreeColumnRecord> records) {
+ return spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
+ }
+
+ private void writeDF(Dataset<Row> df, String mode) {
+ df.select("c1", "c2", "c3")
+ .write()
+ .format("iceberg")
+ .mode(mode)
+ .save(tableLocation);
+ }
+
+ private void checkExpirationResults(Long expectedDatafiles, Long
expectedManifestsDeleted,
+ Long expectedManifestListsDeleted, ExpireSnapshotsActionResult results) {
+
+ Assert.assertEquals("Incorrect number of manifest files deleted",
+ expectedManifestsDeleted, results.getManifestFilesDeleted());
+ Assert.assertEquals("Incorrect number of datafiles deleted",
+ expectedDatafiles, results.getDataFilesDeleted());
+ Assert.assertEquals("Incorrect number of manifest lists deleted",
+ expectedManifestListsDeleted, results.getManifestListsDeleted());
+ }
+
Review comment:
nit: double empty lines
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to
list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after
the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through
spark.sql.shuffle.partitions. The expiration is done
+ * locally using a direct call to RemoveSnapshots. Deletes are still performed
locally after retrieving the results
+ * from the SparkExecutors.
Review comment:
nit: Spark executors
##########
File path:
spark/src/main/java/org/apache/iceberg/actions/ExpireSnapshotsAction.java
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.actions;
+
+import java.util.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import org.apache.iceberg.ExpireSnapshots;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import
org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.functions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An action which performs the same operation as {@link
org.apache.iceberg.ExpireSnapshots} but uses Spark
+ * to to determine the delta in files between the pre and post-expiration
table metadata. All of the same
+ * restrictions apply that apply to Remove Snapshots.
+ * <p>
+ * This implementation uses the MetadataTables for the table being expired to
list all Manifest and DataFiles. This
+ * is made into a Dataframe which is antiJoined with the same list read after
the expiration. This operation will
+ * require a Shuffle so parallelism can be controlled through
spark.sql.shuffle.partitions. The expiration is done
Review comment:
nit: Shuffle -> shuffle, `so` -> `, and`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]