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

Spark 3.5: Spark action to compute the partition stats #9437

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Spark 3.5: Spark action to compute the partition stats
  • Loading branch information
ajantha-bhat committed Mar 21, 2024
commit 36fc685908b4ac8cec11a53f1ca4112a7e07e160
Original file line number Diff line number Diff line change
Expand Up @@ -70,4 +70,10 @@ default RewritePositionDeleteFiles rewritePositionDeletes(Table table) {
throw new UnsupportedOperationException(
this.getClass().getName() + " does not implement rewritePositionDeletes");
}

/** Instantiates an action to compute partition statistics and register it to table metadata. */
default ComputePartitionStats computePartitionStatistics(Table table) {
throw new UnsupportedOperationException(
this.getClass().getName() + " does not implement computePartitionStatistics");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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 org.apache.iceberg.PartitionStatisticsFile;

/** An action to compute and register partition stats. */
public interface ComputePartitionStats
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Question: Do we have to support branches/tags/snapshot IDs? It is not required in the initial version but I guess it makes sense in general?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should support it. Since stats is mapped to snapshot ID. It should be easy to work with branch and tags. I will handle this in a follow up.

extends Action<ComputePartitionStats, ComputePartitionStats.Result> {

/** The action result that contains a summary of the execution. */
interface Result {
/**
* Returns the output file which is registered to the table metadata, null if the table is
* non-partitioned or empty.
*/
PartitionStatisticsFile outputFile();
}
}
99 changes: 99 additions & 0 deletions core/src/main/java/org/apache/iceberg/PartitionEntry.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,15 @@
*/
package org.apache.iceberg;

import java.util.List;
import java.util.Map;
import java.util.Objects;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.PartitionUtil;

public class PartitionEntry implements IndexedRecord {
private PartitionData partitionData;
Expand Down Expand Up @@ -261,6 +265,24 @@ private static Schema prepareAvroSchema(Types.StructType partitionType) {
return AvroSchemaUtil.convert(icebergSchema(partitionType), "partitionEntry");
}

public synchronized PartitionEntry update(PartitionEntry entry) {
this.specId = Math.max(this.specId, entry.specId);
this.dataRecordCount += entry.dataRecordCount;
this.dataFileCount += entry.dataFileCount;
this.dataFileSizeInBytes += entry.dataFileSizeInBytes;
this.posDeleteRecordCount += entry.posDeleteRecordCount;
this.posDeleteFileCount += entry.posDeleteFileCount;
this.eqDeleteRecordCount += entry.eqDeleteRecordCount;
this.eqDeleteFileCount += entry.eqDeleteFileCount;
this.totalRecordCount += entry.totalRecordCount;
if (this.lastUpdatedAt < entry.lastUpdatedAt) {
this.lastUpdatedAt = entry.lastUpdatedAt();
this.lastUpdatedSnapshotId = entry.lastUpdatedSnapshotId;
}

return this;
}

public static class Builder {
private PartitionData partitionData;
private int specId;
Expand Down Expand Up @@ -358,4 +380,81 @@ public PartitionEntry build() {
return partition;
}
}

public static CloseableIterable<PartitionEntry> fromManifest(Table table, ManifestFile manifest) {
CloseableIterable<? extends ManifestEntry<? extends ContentFile<?>>> entries =
CloseableIterable.transform(
ManifestFiles.open(manifest, table.io(), table.specs())
.select(scanColumns(manifest.content())) // don't select stats columns
.liveEntries(),
t ->
(ManifestEntry<? extends ContentFile<?>>)
// defensive copy of manifest entry without stats columns
t.copyWithoutStats());

Types.StructType partitionType = Partitioning.partitionType(table);
return CloseableIterable.transform(
entries, entry -> fromManifestEntry(entry, table, partitionType));
}

private static PartitionEntry fromManifestEntry(
ManifestEntry<?> entry, Table table, Types.StructType partitionType) {
PartitionEntry.Builder builder = PartitionEntry.builder();
builder
.withSpecId(entry.file().specId())
.withPartitionData(coercedPartitionData(entry.file(), table.specs(), partitionType));
Snapshot snapshot = table.snapshot(entry.snapshotId());
if (snapshot != null) {
builder
.withLastUpdatedSnapshotId(snapshot.snapshotId())
.withLastUpdatedAt(snapshot.timestampMillis());
}

switch (entry.file().content()) {
case DATA:
builder
.withDataFileCount(1)
.withDataRecordCount(entry.file().recordCount())
.withDataFileSizeInBytes(entry.file().fileSizeInBytes());
break;
case POSITION_DELETES:
builder.withPosDeleteFileCount(1).withPosDeleteRecordCount(entry.file().recordCount());
break;
case EQUALITY_DELETES:
builder.withEqDeleteFileCount(1).withEqDeleteRecordCount(entry.file().recordCount());
break;
default:
throw new UnsupportedOperationException(
"Unsupported file content type: " + entry.file().content());
}

// TODO: optionally compute TOTAL_RECORD_COUNT based on the flag
return builder.build();
}

private static PartitionData coercedPartitionData(
ContentFile<?> file, Map<Integer, PartitionSpec> specs, Types.StructType partitionType) {
// keep the partition data as per the unified spec by coercing
StructLike partition =
PartitionUtil.coercePartition(partitionType, specs.get(file.specId()), file.partition());
PartitionData data = new PartitionData(partitionType);
for (int i = 0; i < partitionType.fields().size(); i++) {
Object val = partition.get(i, partitionType.fields().get(i).type().typeId().javaClass());
if (val != null) {
data.set(i, val);
}
}
return data;
}

private static List<String> scanColumns(ManifestContent content) {
switch (content) {
case DATA:
return BaseScan.SCAN_COLUMNS;
case DELETES:
return BaseScan.DELETE_SCAN_COLUMNS;
default:
throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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 org.immutables.value.Value;

@Value.Enclosing
@SuppressWarnings("ImmutablesStyle")
@Value.Style(
typeImmutableEnclosing = "ImmutableComputePartitionStats",
visibilityString = "PUBLIC",
builderVisibilityString = "PUBLIC")
interface BaseComputePartitionStats extends ComputePartitionStats {

@Value.Immutable
interface Result extends ComputePartitionStats.Result {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionData;
Expand All @@ -49,10 +50,12 @@ private PartitionStatsUtil() {}

public static OutputFile newPartitionStatsFile(
TableOperations ops, long snapshotId, FileFormat format) {
// TODO: UUID is temp, remove it.
return ops.io()
.newOutputFile(
ops.metadataFileLocation(
format.addExtension(String.format("partition-stats-%d", snapshotId))));
format.addExtension(
String.format("partition-stats-%s-%d", UUID.randomUUID(), snapshotId))));
}

public static void writePartitionStatsFile(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,13 @@
package org.apache.iceberg.spark.actions;

import static org.apache.iceberg.MetadataTableType.ALL_MANIFESTS;
import static org.apache.iceberg.MetadataTableType.ENTRIES;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.first;
import static org.apache.spark.sql.functions.lit;
import static org.apache.spark.sql.functions.max;
import static org.apache.spark.sql.functions.sum;
import static org.apache.spark.sql.functions.when;

import java.util.Collection;
import java.util.Iterator;
Expand All @@ -42,6 +47,7 @@
import org.apache.iceberg.MetadataTableType;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.ReachableFileUtil;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.StaticTableOperations;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableMetadata;
Expand Down Expand Up @@ -72,6 +78,8 @@
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.apache.spark.sql.types.DataTypes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -164,6 +172,66 @@ protected Dataset<FileInfo> contentFileDS(Table table, Set<Long> snapshotIds) {
return manifestBeanDS.flatMap(new ReadManifest(tableBroadcast), FileInfo.ENCODER);
}

protected Dataset<Row> partitionEntryDS(Table table) {
Dataset<Row> dataset =
loadMetadataTable(table, ENTRIES)
.filter(col("status").$less(2))
.select(
col("data_file.spec_id").as("SPEC_ID"),
col("data_file.partition").as("PARTITION_DATA"),
when(col("data_file.content").equalTo(0), col("data_file.record_count"))
.otherwise(lit(0))
.as("DATA_RECORD_COUNT"),
when(col("data_file.content").equalTo(0), lit(1))
.otherwise(lit(0))
.as("DATA_FILE_COUNT"),
when(col("data_file.content").equalTo(0), col("data_file.file_size_in_bytes"))
.otherwise(lit(0))
.as("DATA_FILE_SIZE_IN_BYTES"),
when(col("data_file.content").equalTo(1), col("data_file.record_count"))
.otherwise(lit(0))
.as("POSITION_DELETE_RECORD_COUNT"),
when(col("data_file.content").equalTo(1), lit(1))
.otherwise(lit(0))
.as("POSITION_DELETE_FILE_COUNT"),
when(col("data_file.content").equalTo(2), col("data_file.record_count"))
.otherwise(lit(0))
.as("EQUALITY_DELETE_RECORD_COUNT"),
when(col("data_file.content").equalTo(2), lit(1))
.otherwise(lit(0))
.as("EQUALITY_DELETE_FILE_COUNT"),
functions
.udf(
(Long snapshotId) -> lastUpdatedTime(snapshotId, table), DataTypes.LongType)
.apply(col("snapshot_id"))
.as("LAST_UPDATED_AT"),
col("snapshot_id").as("LAST_UPDATED_SNAPSHOT_ID"),
lit(0)
.alias("TOTAL_RECORD_COUNT")); // TODO: not sure if this can be computed by this
// distributed algorithm. This was meant to be
// effective count after applying deletes.

return dataset
.groupBy(col("PARTITION_DATA"))
.agg(
max(col("LAST_UPDATED_SNAPSHOT_ID")).as("LAST_UPDATED_SNAPSHOT_ID"),
first(col("LAST_UPDATED_AT")).as("LAST_UPDATED_AT"),
max(col("SPEC_ID")).as("SPEC_ID"),
sum(col("DATA_FILE_COUNT")).as("DATA_FILE_COUNT"),
sum(col("DATA_RECORD_COUNT")).as("DATA_RECORD_COUNT"),
sum(col("DATA_FILE_SIZE_IN_BYTES")).as("DATA_FILE_SIZE_IN_BYTES"),
sum(col("POSITION_DELETE_FILE_COUNT")).as("POSITION_DELETE_FILE_COUNT"),
sum(col("POSITION_DELETE_RECORD_COUNT")).as("POSITION_DELETE_RECORD_COUNT"),
sum(col("EQUALITY_DELETE_FILE_COUNT")).as("EQUALITY_DELETE_FILE_COUNT"),
sum(col("EQUALITY_DELETE_RECORD_COUNT")).as("EQUALITY_DELETE_RECORD_COUNT"),
sum(col("TOTAL_RECORD_COUNT")).as("TOTAL_RECORD_COUNT"));
}

public static long lastUpdatedTime(long snapshotId, Table table) {
Snapshot snapshot = table.snapshot(snapshotId);
return snapshot == null ? 0 : snapshot.timestampMillis();
}

protected Dataset<FileInfo> manifestDS(Table table) {
return manifestDS(table, null);
}
Expand Down
Loading