Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Retain table statistics during orphan files removal #5795

Merged
merged 1 commit into from
Sep 30, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,16 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;

public class GenericBlobMetadata implements BlobMetadata {

public static BlobMetadata from(org.apache.iceberg.puffin.BlobMetadata puffinMetadata) {
return new GenericBlobMetadata(
puffinMetadata.type(),
puffinMetadata.snapshotId(),
puffinMetadata.sequenceNumber(),
puffinMetadata.inputFields(),
puffinMetadata.properties());
}

private final String type;
private final long sourceSnapshotId;
private final long sourceSnapshotSequenceNumber;
Expand Down
15 changes: 15 additions & 0 deletions core/src/main/java/org/apache/iceberg/ReachableFileUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -113,4 +113,19 @@ public static List<String> manifestListLocations(Table table) {
}
return manifestListLocations;
}

/**
* Returns locations of statistics files in a table.
*
* @param table table for which statistics files needs to be listed
* @return the location of statistics files
*/
public static List<String> statisticsFilesLocations(Table table) {
List<String> statisticsFilesLocations = Lists.newArrayList();
for (StatisticsFile statisticsFile : table.statisticsFiles()) {
statisticsFilesLocations.add(statisticsFile.path());
}

return statisticsFilesLocations;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
List<String> otherMetadataFiles = Lists.newArrayList();
otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, true));
otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table));
return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
List<String> otherMetadataFiles = Lists.newArrayList();
otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, false));
otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table));
return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,9 @@

import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
Expand All @@ -37,18 +40,28 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.AssertHelpers;
import org.apache.iceberg.Files;
import org.apache.iceberg.GenericBlobMetadata;
import org.apache.iceberg.GenericStatisticsFile;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.StatisticsFile;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.actions.DeleteOrphanFiles;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.hadoop.HadoopCatalog;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.hadoop.HiddenPathFilter;
import org.apache.iceberg.puffin.Blob;
import org.apache.iceberg.puffin.Puffin;
import org.apache.iceberg.puffin.PuffinWriter;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
Expand All @@ -59,6 +72,7 @@
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.assertj.core.api.Assertions;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
Expand Down Expand Up @@ -644,4 +658,80 @@ public void testGarbageCollectionDisabled() {
"Cannot remove orphan files: GC is disabled",
() -> SparkActions.get().deleteOrphanFiles(table).execute());
}

@Test
public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
Table table =
TABLES.create(
SCHEMA,
PartitionSpec.unpartitioned(),
ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"),
tableLocation);

List<ThreeColumnRecord> records =
Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"));
Dataset<Row> df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1);
df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation);

table.refresh();
long snapshotId = table.currentSnapshot().snapshotId();
long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();

File statsLocation =
new File(new URI(tableLocation))
.toPath()
.resolve("data")
.resolve("some-stats-file")
.toFile();
StatisticsFile statisticsFile;
try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
puffinWriter.add(
new Blob(
"some-blob-type",
ImmutableList.of(1),
snapshotId,
snapshotSequenceNumber,
ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
puffinWriter.finish();
statisticsFile =
new GenericStatisticsFile(
snapshotId,
statsLocation.toString(),
puffinWriter.fileSize(),
puffinWriter.footerSize(),
puffinWriter.writtenBlobsMetadata().stream()
.map(GenericBlobMetadata::from)
.collect(ImmutableList.toImmutableList()));
}

Transaction transaction = table.newTransaction();
transaction.updateStatistics().setStatistics(snapshotId, statisticsFile).commit();
transaction.commitTransaction();

SparkActions.get()
.deleteOrphanFiles(table)
.olderThan(System.currentTimeMillis() + 1000)
.execute();

Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue();
Assertions.assertThat(statsLocation.length())
.as("stats file length")
.isEqualTo(statisticsFile.fileSizeInBytes());

transaction = table.newTransaction();
transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit();
transaction.commitTransaction();

DeleteOrphanFiles.Result result =
SparkActions.get()
.deleteOrphanFiles(table)
.olderThan(System.currentTimeMillis() + 1000)
.execute();
Iterable<String> orphanFileLocations = result.orphanFileLocations();
Assertions.assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1);
Assertions.assertThat(Iterables.getOnlyElement(orphanFileLocations))
.as("Deleted file")
.isEqualTo(statsLocation.toURI().toString());
Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,17 +21,35 @@
import static org.apache.iceberg.TableProperties.GC_ENABLED;
import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED;

import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.sql.Timestamp;
import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.iceberg.AssertHelpers;
import org.apache.iceberg.Files;
import org.apache.iceberg.GenericBlobMetadata;
import org.apache.iceberg.GenericStatisticsFile;
import org.apache.iceberg.StatisticsFile;
import org.apache.iceberg.Table;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.exceptions.ValidationException;
import org.apache.iceberg.puffin.Blob;
import org.apache.iceberg.puffin.Puffin;
import org.apache.iceberg.puffin.PuffinWriter;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.spark.Spark3Util;
import org.apache.spark.sql.AnalysisException;
import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException;
import org.assertj.core.api.Assertions;
import org.junit.After;
import org.junit.Assert;
import org.junit.Rule;
Expand Down Expand Up @@ -332,4 +350,116 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() {
"CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)",
catalogName, tableIdent, -1));
}

@Test
public void testRemoveOrphanFilesWithStatisticFiles() throws Exception {
if (!catalogName.equals("spark_catalog")) {
sql(
"CREATE TABLE %s USING iceberg "
+ "TBLPROPERTIES('format-version'='2') "
+ "AS SELECT 10 int, 'abc' data",
tableName);
} else {
// give a fresh location to Hive tables as Spark will not clean up the table location
// correctly while dropping tables through spark_catalog
sql(
"CREATE TABLE %s USING iceberg LOCATION '%s' "
+ "TBLPROPERTIES('format-version'='2') "
+ "AS SELECT 10 int, 'abc' data",
tableName, temp.newFolder());
}

Table table = Spark3Util.loadIcebergTable(spark, tableName);

String statsFileName = "stats-file-" + UUID.randomUUID();
File statsLocation =
(new URI(table.location()).isAbsolute()
? new File(new URI(table.location()))
: new File(table.location()))
.toPath()
.resolve("data")
.resolve(statsFileName)
.toFile();
StatisticsFile statisticsFile;
try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) {
long snapshotId = table.currentSnapshot().snapshotId();
long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber();
puffinWriter.add(
new Blob(
"some-blob-type",
ImmutableList.of(1),
snapshotId,
snapshotSequenceNumber,
ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8))));
puffinWriter.finish();
statisticsFile =
new GenericStatisticsFile(
snapshotId,
statsLocation.toString(),
puffinWriter.fileSize(),
puffinWriter.footerSize(),
puffinWriter.writtenBlobsMetadata().stream()
.map(GenericBlobMetadata::from)
.collect(ImmutableList.toImmutableList()));
}

Transaction transaction = table.newTransaction();
transaction
.updateStatistics()
.setStatistics(statisticsFile.snapshotId(), statisticsFile)
.commit();
transaction.commitTransaction();

// wait to ensure files are old enough
waitUntilAfter(System.currentTimeMillis());
Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis()));

List<Object[]> output =
sql(
"CALL %s.system.remove_orphan_files("
+ "table => '%s',"
+ "older_than => TIMESTAMP '%s')",
catalogName, tableIdent, currentTimestamp);
Assertions.assertThat(output).as("Should be no orphan files").isEmpty();

Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue();
Assertions.assertThat(statsLocation.length())
.as("stats file length")
.isEqualTo(statisticsFile.fileSizeInBytes());

transaction = table.newTransaction();
transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit();
transaction.commitTransaction();

output =
sql(
"CALL %s.system.remove_orphan_files("
+ "table => '%s',"
+ "older_than => TIMESTAMP '%s')",
catalogName, tableIdent, currentTimestamp);
Assertions.assertThat(output).as("Should be orphan files").hasSize(1);
Assertions.assertThat(Iterables.getOnlyElement(output))
.as("Deleted files")
.containsExactly(statsLocation.toURI().toString());
Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse();
}

private static File tableLocation(Table table) {
// Depending on test case, location is URI or a local path
String location = table.location();
File file = new File(location);
try {
URI uri = new URI(location);
if (uri.getScheme() != null) {
// Location is a well-formed URI
file = new File(uri);
}
} catch (URISyntaxException ignored) {
// Ignore
}

Preconditions.checkState(
file.isDirectory(), "Table location '%s' does not point to a directory", location);
return file;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,7 @@ protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
List<String> otherMetadataFiles = Lists.newArrayList();
otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, true));
otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table));
return spark().createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@ protected Dataset<Row> buildOtherMetadataFileDF(Table table) {
List<String> otherMetadataFiles = Lists.newArrayList();
otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, false));
otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table));
otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table));
return spark.createDataset(otherMetadataFiles, Encoders.STRING()).toDF("file_path");
}

Expand Down
Loading