Skip to content
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
29 changes: 27 additions & 2 deletions core/src/main/java/org/apache/iceberg/RewriteTablePathUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -129,8 +129,8 @@ public static TableMetadata replacePaths(
metadataLogEntries,
metadata.refs(),
updatePathInStatisticsFiles(metadata.statisticsFiles(), sourcePrefix, targetPrefix),
// TODO: update partition statistics file paths
metadata.partitionStatisticsFiles(),
updatePathInPartitionStatisticsFiles(
metadata.partitionStatisticsFiles(), sourcePrefix, targetPrefix),
metadata.nextRowId(),
metadata.encryptionKeys(),
metadata.changes());
Expand Down Expand Up @@ -175,6 +175,31 @@ private static List<StatisticsFile> updatePathInStatisticsFiles(
.collect(Collectors.toList());
}

/**
* This method updates the file paths in a list of PartitionStatisticsFile. It replaces the
* sourcePrefix in the file paths with the targetPrefix.
*
* @param partitionStatisticsFiles The list of PartitionStatisticsFile to update.
* @param sourcePrefix The prefix to be replaced in the file paths.
* @param targetPrefix The new prefix to replace the sourcePrefix in the file paths.
* @return A new list of PartitionStatisticsFile with updated file paths.
*/
private static List<PartitionStatisticsFile> updatePathInPartitionStatisticsFiles(
List<PartitionStatisticsFile> partitionStatisticsFiles,
String sourcePrefix,
String targetPrefix) {

return partitionStatisticsFiles.stream()
.map(
existing ->
ImmutableGenericPartitionStatisticsFile.builder()
.snapshotId(existing.snapshotId())
.path(newPath(existing.path(), sourcePrefix, targetPrefix))
.fileSizeInBytes(existing.fileSizeInBytes())
.build())
.collect(Collectors.toList());
}

private static List<TableMetadata.MetadataLogEntry> updatePathInMetadataLogs(
TableMetadata metadata, String sourcePrefix, String targetPrefix) {
List<TableMetadata.MetadataLogEntry> metadataLogEntries =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.iceberg.HasTableOperations;
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.PartitionStatisticsFile;
import org.apache.iceberg.RewriteTablePathUtil;
import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter;
import org.apache.iceberg.RewriteTablePathUtil.RewriteResult;
Expand Down Expand Up @@ -273,11 +274,6 @@ private String rebuildMetadata() {
TableMetadata endMetadata =
((HasTableOperations) newStaticTable(endVersionName, table.io())).operations().current();

Preconditions.checkArgument(
endMetadata.partitionStatisticsFiles() == null
|| endMetadata.partitionStatisticsFiles().isEmpty(),
"Partition statistics files are not supported yet.");

// rebuild version files
RewriteResult<Snapshot> rewriteVersionResult = rewriteVersionFiles(endMetadata);
Set<Snapshot> deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite());
Expand Down Expand Up @@ -385,6 +381,9 @@ private Set<Pair<String, String>> rewriteVersionFile(
// include statistics files in copy plan
result.addAll(
statsFileCopyPlan(metadata.statisticsFiles(), newTableMetadata.statisticsFiles()));
result.addAll(
partitionStatsFileCopyPlan(
metadata.partitionStatisticsFiles(), newTableMetadata.partitionStatisticsFiles()));
return result;
}

Expand All @@ -409,6 +408,27 @@ private Set<Pair<String, String>> statsFileCopyPlan(
return result;
}

private Set<Pair<String, String>> partitionStatsFileCopyPlan(
List<PartitionStatisticsFile> beforeStats, List<PartitionStatisticsFile> afterStats) {
Set<Pair<String, String>> result = Sets.newHashSet();
if (beforeStats.isEmpty()) {
return result;
}

Preconditions.checkArgument(
beforeStats.size() == afterStats.size(),
"Before and after path rewrite, partition statistic files count should be same");
for (int i = 0; i < beforeStats.size(); i++) {
PartitionStatisticsFile before = beforeStats.get(i);
PartitionStatisticsFile after = afterStats.get(i);
Preconditions.checkArgument(
before.fileSizeInBytes() == after.fileSizeInBytes(),
"Before and after path rewrite, partition statistic file size should be same");
result.add(Pair.of(before.path(), after.path()));
}
return result;
}

/**
* Rewrite a manifest list representing a snapshot.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,14 +38,12 @@
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DeleteFile;
import org.apache.iceberg.HasTableOperations;
import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.StaticTableOperations;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.TableMetadataParser;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.TestHelpers;
import org.apache.iceberg.actions.ActionsProvider;
Expand Down Expand Up @@ -902,36 +900,35 @@ public void testInvalidArgs() {
}

@Test
public void testPartitionStatisticFile() throws IOException {
public void testTableWithManyPartitionStatisticFile() throws IOException {
String sourceTableLocation = newTableLocation();
Map<String, String> properties = Maps.newHashMap();
properties.put("format-version", "2");
String tableName = "v2tblwithPartStats";
Table sourceTable =
createMetastoreTable(sourceTableLocation, properties, "default", tableName, 0);
createMetastoreTable(sourceTableLocation, properties, "default", tableName, 0, "c1");

TableMetadata metadata = currentMetadata(sourceTable);
TableMetadata withPartStatistics =
TableMetadata.buildFrom(metadata)
.setPartitionStatistics(
ImmutableGenericPartitionStatisticsFile.builder()
.snapshotId(11L)
.path("/some/partition/stats/file.parquet")
.fileSizeInBytes(42L)
.build())
.build();
int iterations = 10;
for (int i = 0; i < iterations; i++) {
sql("insert into hive.default.%s values (%s, 'AAAAAAAAAA', 'AAAA')", tableName, i);
sourceTable.refresh();
actions().computePartitionStats(sourceTable).execute();
}

OutputFile file = sourceTable.io().newOutputFile(metadata.metadataFileLocation());
TableMetadataParser.overwrite(withPartStatistics, file);
sourceTable.refresh();
assertThat(sourceTable.partitionStatisticsFiles()).hasSize(iterations);

assertThatThrownBy(
() ->
actions()
.rewriteTablePath(sourceTable)
.rewriteLocationPrefix(sourceTableLocation, targetTableLocation())
.execute())
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining("Partition statistics files are not supported yet");
String targetTableLocation = targetTableLocation();
RewriteTablePath.Result result =
actions()
.rewriteTablePath(sourceTable)
.rewriteLocationPrefix(sourceTableLocation, targetTableLocation)
.execute();
checkFileNum(
iterations * 2 + 1, iterations, iterations, 0, iterations, iterations * 6 + 1, result);

findAndAssertFileInFileList(
result, "partition-stats", sourceTableLocation, targetTableLocation);
}

@Test
Expand Down Expand Up @@ -988,29 +985,7 @@ public void testStatisticsFileSourcePath() throws IOException {

checkFileNum(3, 1, 1, 1, 7, result);

// Read the file list to verify statistics file paths
List<Tuple2<String, String>> filesToMove = readPathPairList(result.fileListLocation());

// Find the statistics file entry in the file list using stream
Tuple2<String, String> statsFilePathPair =
filesToMove.stream()
.filter(pathPair -> pathPair._1().endsWith(".stats"))
.findFirst()
.orElse(null);

assertThat(statsFilePathPair).as("Should find statistics file in file list").isNotNull();

// Verify the source path points to the actual source location, not staging
assertThat(statsFilePathPair._1())
.as("Statistics file source should point to source table location and NOT staging")
.startsWith(sourceTableLocation)
.contains("/metadata/")
.doesNotContain("staging");

// Verify the target path is correctly rewritten
assertThat(statsFilePathPair._2())
.as("Statistics file target should point to target table location")
.startsWith(targetTableLocation);
findAndAssertFileInFileList(result, ".stats", sourceTableLocation, targetTableLocation);
}

@Test
Expand Down Expand Up @@ -1265,14 +1240,32 @@ protected void checkFileNum(
int manifestFileCount,
int totalCount,
RewriteTablePath.Result result) {
checkFileNum(versionFileCount, manifestListCount, manifestFileCount, 0, totalCount, result);
checkFileNum(versionFileCount, manifestListCount, manifestFileCount, 0, 0, totalCount, result);
}

protected void checkFileNum(
int versionFileCount,
int manifestListCount,
int manifestFileCount,
int statisticsFileCount,
int totalCount,
RewriteTablePath.Result result) {
checkFileNum(
versionFileCount,
manifestListCount,
manifestFileCount,
statisticsFileCount,
0,
totalCount,
result);
}

protected void checkFileNum(
int versionFileCount,
int manifestListCount,
int manifestFileCount,
int statisticsFileCount,
int partitionFileCount,
int totalCount,
RewriteTablePath.Result result) {
List<String> filesToMove =
Expand Down Expand Up @@ -1302,6 +1295,9 @@ protected void checkFileNum(
assertThat(filesToMove.stream().filter(f -> f.endsWith(".stats")))
.as("Wrong rebuilt Statistic file count")
.hasSize(statisticsFileCount);
assertThat(filesToMove.stream().filter(f -> f.contains("partition-stats")))
.as("Wrong rebuilt Partition Statistic file count")
.hasSize(partitionFileCount);
assertThat(filesToMove).as("Wrong total file count").hasSize(totalCount);
}

Expand Down Expand Up @@ -1355,41 +1351,79 @@ private Table createMetastoreTable(
String namespace,
String tableName,
int snapshotNumber) {
return createMetastoreTable(location, properties, namespace, tableName, snapshotNumber, null);
}

private Table createMetastoreTable(
String location,
Map<String, String> properties,
String namespace,
String tableName,
int snapshotNumber,
String partitionColumn) {
spark.conf().set("spark.sql.catalog.hive", SparkCatalog.class.getName());
spark.conf().set("spark.sql.catalog.hive.type", "hive");
spark.conf().set("spark.sql.catalog.hive.default-namespace", "default");
spark.conf().set("spark.sql.catalog.hive.cache-enabled", "false");

// Generate and execute CREATE TABLE SQL
String createTableSQL =
generateCreateTableSQL(location, properties, namespace, tableName, partitionColumn);
sql(createTableSQL);

for (int i = 0; i < snapshotNumber; i++) {
sql("insert into hive.%s.%s values (%s, 'AAAAAAAAAA', 'AAAA')", namespace, tableName, i);
}
return catalog.loadTable(TableIdentifier.of(namespace, tableName));
}

/**
* Generates SQL statement for creating an Iceberg table
*
* @param location location the storage location path for the table, can be empty
* @param properties key-value pairs of table properties for setting table metadata
* @param namespace the namespace (database name)
* @param tableName the name of the table to be created
* @param partitionColumn the partition column name, must be one of c1, c2, or c3; can be null or
* empty string for non-partitioned table
* @return CREATE TABLE SQL statement string
*/
private String generateCreateTableSQL(
String location,
Map<String, String> properties,
String namespace,
String tableName,
String partitionColumn) {
StringBuilder propertiesStr = new StringBuilder();
properties.forEach((k, v) -> propertiesStr.append("'" + k + "'='" + v + "',"));
String tblProperties =
propertiesStr.substring(0, propertiesStr.length() > 0 ? propertiesStr.length() - 1 : 0);

sql("DROP TABLE IF EXISTS hive.%s.%s", namespace, tableName);
if (tblProperties.isEmpty()) {
String sqlStr =
String.format(
"CREATE TABLE hive.%s.%s (c1 bigint, c2 string, c3 string)", namespace, tableName);
if (!location.isEmpty()) {
sqlStr = String.format("%s USING iceberg LOCATION '%s'", sqlStr, location);
}
sql(sqlStr);

StringBuilder createTableSql = new StringBuilder();
createTableSql
.append("CREATE TABLE hive.")
.append(namespace)
.append(".")
.append(tableName)
.append(" (c1 bigint, c2 string, c3 string)");

if (partitionColumn != null && !partitionColumn.isEmpty()) {
createTableSql.append(" USING iceberg PARTITIONED BY (").append(partitionColumn).append(")");
} else {
String sqlStr =
String.format(
"CREATE TABLE hive.%s.%s (c1 bigint, c2 string, c3 string)", namespace, tableName);
if (!location.isEmpty()) {
sqlStr = String.format("%s USING iceberg LOCATION '%s'", sqlStr, location);
}

sqlStr = String.format("%s TBLPROPERTIES (%s)", sqlStr, tblProperties);
sql(sqlStr);
createTableSql.append(" USING iceberg");
}

for (int i = 0; i < snapshotNumber; i++) {
sql("insert into hive.%s.%s values (%s, 'AAAAAAAAAA', 'AAAA')", namespace, tableName, i);
if (!location.isEmpty()) {
createTableSql.append(" LOCATION '").append(location).append("'");
}
return catalog.loadTable(TableIdentifier.of(namespace, tableName));

if (!tblProperties.isEmpty()) {
createTableSql.append(" TBLPROPERTIES (").append(tblProperties).append(")");
}

return createTableSql.toString();
}

private static String fileName(String path) {
Expand Down Expand Up @@ -1434,4 +1468,36 @@ private void removeBroadcastValuesFromLocalBlockManager(long id) {
blockInfoManager.removeBlock(blockId);
blockManager.memoryStore().remove(blockId);
}

private void findAndAssertFileInFileList(
RewriteTablePath.Result result,
String fileIdentifier,
String sourceTableLocation,
String targetTableLocation) {

List<Tuple2<String, String>> filesToMove = readPathPairList(result.fileListLocation());

// Find the file path pair that contains the specified file identifier
Tuple2<String, String> filePathPair =
filesToMove.stream()
.filter(pair -> pair._1().contains(fileIdentifier))
.findFirst()
.orElse(null);

// Assert that the file was found in the list
assertThat(filePathPair).as("Should find " + fileIdentifier + " file in file list").isNotNull();

// Validate source path: should point to source table location, contain metadata, and not
// staging
assertThat(filePathPair._1())
.as(fileIdentifier + " source should point to source table location and NOT staging")
.startsWith(sourceTableLocation)
.contains("/metadata/")
.doesNotContain("staging");

// Validate target path: should point to target table location
assertThat(filePathPair._2())
.as(fileIdentifier + " target should point to target table location")
.startsWith(targetTableLocation);
}
}