Skip to content

Commit

Permalink
Add statistics information in table snapshot
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Aug 30, 2022
1 parent 6a1f351 commit 9179106
Show file tree
Hide file tree
Showing 29 changed files with 1,065 additions and 3 deletions.
3 changes: 3 additions & 0 deletions .palantir/revapi.yml
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,9 @@ acceptedBreaks:
- code: "java.method.addedToInterface"
new: "method java.lang.String org.apache.iceberg.expressions.Reference<T>::name()"
justification: "All subclasses implement name"
- code: "java.method.addedToInterface"
new: "method java.util.List<org.apache.iceberg.StatisticsFile> org.apache.iceberg.Table::statisticsFiles()"
justification: "new API method"
- code: "java.method.removed"
old: "method org.apache.iceberg.OverwriteFiles org.apache.iceberg.OverwriteFiles::validateNoConflictingAppends(org.apache.iceberg.expressions.Expression)"
justification: "Deprecations for 1.0 release"
Expand Down
7 changes: 7 additions & 0 deletions api/src/main/java/org/apache/iceberg/Table.java
Original file line number Diff line number Diff line change
Expand Up @@ -308,6 +308,13 @@ default UpdateStatistics updateStatistics() {
/** Returns a {@link LocationProvider} to provide locations for new data files. */
LocationProvider locationProvider();

/**
* Returns the current statistics files for the table
*
* @return the current statistics files for the table
*/
List<StatisticsFile> statisticsFiles();

/**
* Returns the current refs for the table
*
Expand Down
4 changes: 3 additions & 1 deletion api/src/main/java/org/apache/iceberg/UpdateStatistics.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@
*/
package org.apache.iceberg;

import java.util.List;

/** API for updating statistics files in a table. */
public interface UpdateStatistics extends PendingUpdate<Snapshot> {
public interface UpdateStatistics extends PendingUpdate<List<StatisticsFile>> {
/**
* Set the table's statistics file for given snapshot, replacing the previous statistics file for
* the snapshot if any exists.
Expand Down
6 changes: 6 additions & 0 deletions core/src/main/java/org/apache/iceberg/BaseMetadataTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.iceberg.encryption.EncryptionManager;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.LocationProvider;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;

/**
Expand Down Expand Up @@ -158,6 +159,11 @@ public List<HistoryEntry> history() {
return table().history();
}

@Override
public List<StatisticsFile> statisticsFiles() {
return ImmutableList.of();
}

@Override
public Map<String, SnapshotRef> refs() {
return table().refs();
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/java/org/apache/iceberg/BaseTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,11 @@ public LocationProvider locationProvider() {
return operations().locationProvider();
}

@Override
public List<StatisticsFile> statisticsFiles() {
return ops.current().statisticsFiles();
}

@Override
public Map<String, SnapshotRef> refs() {
return ops.current().refs();
Expand Down
13 changes: 13 additions & 0 deletions core/src/main/java/org/apache/iceberg/BaseTransaction.java
Original file line number Diff line number Diff line change
Expand Up @@ -216,6 +216,14 @@ public DeleteFiles newDelete() {
return delete;
}

@Override
public UpdateStatistics updateStatistics() {
checkLastOperationCommitted("UpdateStatistics");
UpdateStatistics updateStatistics = new SetStatistics(transactionOps);
updates.add(updateStatistics);
return updateStatistics;
}

@Override
public ExpireSnapshots expireSnapshots() {
checkLastOperationCommitted("ExpireSnapshots");
Expand Down Expand Up @@ -747,6 +755,11 @@ public LocationProvider locationProvider() {
return transactionOps.locationProvider();
}

@Override
public List<StatisticsFile> statisticsFiles() {
return current.statisticsFiles();
}

@Override
public Map<String, SnapshotRef> refs() {
return current.refs();
Expand Down
13 changes: 12 additions & 1 deletion core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.iceberg;

import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand All @@ -26,7 +27,17 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;

public class GenericBlobMetadata implements BlobMetadata {
public class GenericBlobMetadata implements BlobMetadata, Serializable {

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
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,14 @@
*/
package org.apache.iceberg;

import java.io.Serializable;
import java.util.List;
import java.util.Objects;
import java.util.StringJoiner;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;

public class GenericStatisticsFile implements StatisticsFile {
public class GenericStatisticsFile implements StatisticsFile, Serializable {
private final long snapshotId;
private final String path;
private final long fileSizeInBytes;
Expand Down
14 changes: 14 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,18 @@ 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;
}
}
7 changes: 7 additions & 0 deletions core/src/main/java/org/apache/iceberg/SerializableTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ public class SerializableTable implements Table, Serializable {
private final FileIO io;
private final EncryptionManager encryption;
private final LocationProvider locationProvider;
private final List<StatisticsFile> statisticsFiles;
private final Map<String, SnapshotRef> refs;

private transient volatile Table lazyTable = null;
Expand All @@ -82,6 +83,7 @@ protected SerializableTable(Table table) {
this.io = fileIO(table);
this.encryption = table.encryption();
this.locationProvider = table.locationProvider();
this.statisticsFiles = table.statisticsFiles();
this.refs = table.refs();
}

Expand Down Expand Up @@ -237,6 +239,11 @@ public LocationProvider locationProvider() {
return locationProvider;
}

@Override
public List<StatisticsFile> statisticsFiles() {
return statisticsFiles;
}

@Override
public Map<String, SnapshotRef> refs() {
return refs;
Expand Down
77 changes: 77 additions & 0 deletions core/src/main/java/org/apache/iceberg/SetStatistics.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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;

import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Stream;
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.Maps;

public class SetStatistics implements UpdateStatistics {
private final TableOperations ops;
private final Map<Long, Optional<StatisticsFile>> statisticsToSet = Maps.newHashMap();

public SetStatistics(TableOperations ops) {
this.ops = ops;
}

@Override
public UpdateStatistics setStatistics(long snapshotId, StatisticsFile statisticsFile) {
Preconditions.checkArgument(snapshotId == statisticsFile.snapshotId());
statisticsToSet.put(snapshotId, Optional.of(statisticsFile));
return this;
}

@Override
public UpdateStatistics removeStatistics(long snapshotId) {
statisticsToSet.put(snapshotId, Optional.empty());
return this;
}

@Override
public List<StatisticsFile> apply() {
return Stream.concat(
// Retained statistics
ops.current().statisticsFiles().stream()
.filter(
statisticsFile -> !statisticsToSet.containsKey(statisticsFile.snapshotId())),
// New statistics
statisticsToSet.values().stream().filter(Optional::isPresent).map(Optional::get))
.collect(ImmutableList.toImmutableList());
}

@Override
public void commit() {
TableMetadata base = ops.current(); // or ops.refresh() ?
TableMetadata.Builder builder = TableMetadata.buildFrom(base);
statisticsToSet.forEach(
(snapshotId, statistics) -> {
if (statistics.isPresent()) {
builder.setStatistics(snapshotId, statistics.get());
} else {
builder.removeStatistics(snapshotId);
}
});
TableMetadata newMetadata = builder.build();
ops.commit(base, newMetadata);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,7 @@ public static String toJson(TableMetadata metadata) {
}
}

@SuppressWarnings("checkstyle:CyclomaticComplexity")
public static void toJson(TableMetadata metadata, JsonGenerator generator) throws IOException {
generator.writeStartObject();

Expand Down Expand Up @@ -225,6 +226,12 @@ public static void toJson(TableMetadata metadata, JsonGenerator generator) throw
}
generator.writeEndArray();

generator.writeArrayFieldStart(STATISTICS);
for (StatisticsFile statisticsFile : metadata.statisticsFiles()) {
StatisticsFileParser.toJson(statisticsFile, generator);
}
generator.writeEndArray();

generator.writeArrayFieldStart(SNAPSHOT_LOG);
for (HistoryEntry logEntry : metadata.snapshotLog()) {
generator.writeStartObject();
Expand Down
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

0 comments on commit 9179106

Please sign in to comment.