-
Notifications
You must be signed in to change notification settings - Fork 2.3k
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
Spark: Ensure that partition stats files are considered for GC procedures #9284
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -133,13 +133,13 @@ public static List<String> manifestListLocations(Table table, Set<Long> snapshot | |
} | ||
|
||
/** | ||
* Returns locations of statistics files in a table. | ||
* Returns locations of all 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) { | ||
return statisticsFilesLocations(table, statisticsFile -> true); | ||
return statisticsFilesLocationsForSnapshots(table, null); | ||
} | ||
|
||
/** | ||
|
@@ -148,12 +148,49 @@ public static List<String> statisticsFilesLocations(Table table) { | |
* @param table table for which statistics files needs to be listed | ||
* @param predicate predicate for filtering the statistics files | ||
* @return the location of statistics files | ||
* @deprecated use the {@code statisticsFilesLocationsForSnapshots(table, snapshotIds)} instead. | ||
*/ | ||
@Deprecated | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can't use the |
||
public static List<String> statisticsFilesLocations( | ||
Table table, Predicate<StatisticsFile> predicate) { | ||
return table.statisticsFiles().stream() | ||
.filter(predicate) | ||
.map(StatisticsFile::path) | ||
.collect(Collectors.toList()); | ||
} | ||
|
||
/** | ||
* Returns locations of all statistics files for a table matching the given snapshot IDs. | ||
* | ||
* @param table table for which statistics files needs to be listed | ||
* @param snapshotIds ids of snapshots for which statistics files will be returned. If null, | ||
* statistics files for all the snapshots will be returned. | ||
* @return the location of statistics files | ||
*/ | ||
public static List<String> statisticsFilesLocationsForSnapshots( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What do you think about calling it There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The deprecated method already has a name Hence, I had to keep different name. |
||
Table table, Set<Long> snapshotIds) { | ||
List<String> statsFileLocations = Lists.newArrayList(); | ||
Predicate<StatisticsFile> statsFilePredicate; | ||
aokolnychyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
Predicate<PartitionStatisticsFile> partitionStatsFilePredicate; | ||
if (snapshotIds == null) { | ||
aokolnychyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
statsFilePredicate = statisticsFile -> true; | ||
partitionStatsFilePredicate = partitionStatisticsFile -> true; | ||
} else { | ||
statsFilePredicate = statisticsFile -> snapshotIds.contains(statisticsFile.snapshotId()); | ||
partitionStatsFilePredicate = | ||
partitionStatisticsFile -> snapshotIds.contains(partitionStatisticsFile.snapshotId()); | ||
} | ||
|
||
table.statisticsFiles().stream() | ||
.filter(statsFilePredicate) | ||
.map(StatisticsFile::path) | ||
.forEach(statsFileLocations::add); | ||
|
||
table.partitionStatisticsFiles().stream() | ||
.filter(partitionStatsFilePredicate) | ||
.map(PartitionStatisticsFile::path) | ||
.forEach(statsFileLocations::add); | ||
|
||
return statsFileLocations; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,54 @@ | ||
/* | ||
* 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.io.IOException; | ||
import java.io.UncheckedIOException; | ||
import java.util.UUID; | ||
import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; | ||
import org.apache.iceberg.PartitionStatisticsFile; | ||
import org.apache.iceberg.io.FileIO; | ||
import org.apache.iceberg.io.PositionOutputStream; | ||
|
||
public class ProcedureUtil { | ||
|
||
private ProcedureUtil() {} | ||
|
||
static PartitionStatisticsFile writePartitionStatsFile( | ||
long snapshotId, String statsLocation, FileIO fileIO) { | ||
PositionOutputStream positionOutputStream; | ||
try { | ||
positionOutputStream = fileIO.newOutputFile(statsLocation).create(); | ||
positionOutputStream.close(); | ||
} catch (IOException e) { | ||
throw new UncheckedIOException(e); | ||
} | ||
|
||
return ImmutableGenericPartitionStatisticsFile.builder() | ||
.snapshotId(snapshotId) | ||
.fileSizeInBytes(42L) | ||
.path(statsLocation) | ||
.build(); | ||
} | ||
|
||
static String statsFileLocation(String tableLocation) { | ||
String statsFileName = "stats-file-" + UUID.randomUUID(); | ||
return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,13 +28,12 @@ | |
import java.time.Instant; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.UUID; | ||
import java.util.stream.Collectors; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.iceberg.GenericBlobMetadata; | ||
import org.apache.iceberg.GenericStatisticsFile; | ||
import org.apache.iceberg.PartitionStatisticsFile; | ||
import org.apache.iceberg.Snapshot; | ||
import org.apache.iceberg.StatisticsFile; | ||
import org.apache.iceberg.Table; | ||
|
@@ -445,7 +444,7 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { | |
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); | ||
Table table = validationCatalog.loadTable(tableIdent); | ||
String statsFileLocation1 = statsFileLocation(table.location()); | ||
String statsFileLocation1 = ProcedureUtil.statsFileLocation(table.location()); | ||
StatisticsFile statisticsFile1 = | ||
writeStatsFile( | ||
table.currentSnapshot().snapshotId(), | ||
|
@@ -456,7 +455,7 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { | |
|
||
sql("INSERT INTO %s SELECT 20, 'def'", tableName); | ||
table.refresh(); | ||
String statsFileLocation2 = statsFileLocation(table.location()); | ||
String statsFileLocation2 = ProcedureUtil.statsFileLocation(table.location()); | ||
StatisticsFile statisticsFile2 = | ||
writeStatsFile( | ||
table.currentSnapshot().snapshotId(), | ||
|
@@ -475,18 +474,9 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { | |
Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); | ||
|
||
table.refresh(); | ||
List<StatisticsFile> statsWithSnapshotId1 = | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since we are checking |
||
table.statisticsFiles().stream() | ||
.filter(statisticsFile -> statisticsFile.snapshotId() == statisticsFile1.snapshotId()) | ||
.collect(Collectors.toList()); | ||
Assertions.assertThat(statsWithSnapshotId1) | ||
.as( | ||
"Statistics file entry in TableMetadata should be deleted for the snapshot %s", | ||
statisticsFile1.snapshotId()) | ||
.isEmpty(); | ||
Assertions.assertThat(table.statisticsFiles()) | ||
.as( | ||
"Statistics file entry in TableMetadata should be present for the snapshot %s", | ||
"Statistics file entry in TableMetadata should be present only for the snapshot %s", | ||
statisticsFile2.snapshotId()) | ||
.extracting(StatisticsFile::snapshotId) | ||
.containsExactly(statisticsFile2.snapshotId()); | ||
|
@@ -500,7 +490,58 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { | |
.exists(); | ||
} | ||
|
||
private StatisticsFile writeStatsFile( | ||
@Test | ||
public void testExpireSnapshotsWithPartitionStatisticFiles() { | ||
sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); | ||
sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); | ||
Table table = validationCatalog.loadTable(tableIdent); | ||
String partitionStatsFileLocation1 = ProcedureUtil.statsFileLocation(table.location()); | ||
PartitionStatisticsFile partitionStatisticsFile1 = | ||
ProcedureUtil.writePartitionStatsFile( | ||
table.currentSnapshot().snapshotId(), partitionStatsFileLocation1, table.io()); | ||
table.updatePartitionStatistics().setPartitionStatistics(partitionStatisticsFile1).commit(); | ||
|
||
sql("INSERT INTO %s SELECT 20, 'def'", tableName); | ||
table.refresh(); | ||
String partitionStatsFileLocation2 = ProcedureUtil.statsFileLocation(table.location()); | ||
PartitionStatisticsFile partitionStatisticsFile2 = | ||
ProcedureUtil.writePartitionStatsFile( | ||
table.currentSnapshot().snapshotId(), partitionStatsFileLocation2, table.io()); | ||
table.updatePartitionStatistics().setPartitionStatistics(partitionStatisticsFile2).commit(); | ||
|
||
waitUntilAfter(table.currentSnapshot().timestampMillis()); | ||
|
||
Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); | ||
List<Object[]> output = | ||
sql( | ||
"CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", | ||
catalogName, currentTimestamp, tableIdent); | ||
Assertions.assertThat(output.get(0)[5]) | ||
.as("should be 1 deleted partition statistics file") | ||
.isEqualTo(1L); | ||
|
||
table.refresh(); | ||
Assertions.assertThat(table.partitionStatisticsFiles()) | ||
.as( | ||
"partition statistics file entry in TableMetadata should be present only for the snapshot %s", | ||
partitionStatisticsFile2.snapshotId()) | ||
.extracting(PartitionStatisticsFile::snapshotId) | ||
.containsExactly(partitionStatisticsFile2.snapshotId()); | ||
|
||
Assertions.assertThat(new File(partitionStatsFileLocation1)) | ||
.as( | ||
"partition statistics file should not exist for snapshot %s", | ||
partitionStatisticsFile1.snapshotId()) | ||
.doesNotExist(); | ||
|
||
Assertions.assertThat(new File(partitionStatsFileLocation2)) | ||
.as( | ||
"partition statistics file should exist for snapshot %s", | ||
partitionStatisticsFile2.snapshotId()) | ||
.exists(); | ||
} | ||
|
||
private static StatisticsFile writeStatsFile( | ||
long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) | ||
throws IOException { | ||
try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { | ||
|
@@ -523,9 +564,4 @@ private StatisticsFile writeStatsFile( | |
.collect(ImmutableList.toImmutableList())); | ||
} | ||
} | ||
|
||
private String statsFileLocation(String tableLocation) { | ||
String statsFileName = "stats-file-" + UUID.randomUUID(); | ||
return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we add a note on when it was deprecated (1.5.0) and when it will be removed (1.6.0)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have added this already in previous commit.