Skip to content

Iceberg: Table-level column stats filter support #5724

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
Expand All @@ -42,6 +41,7 @@
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.collections.MapUtils;
Expand Down Expand Up @@ -186,7 +186,6 @@
import org.apache.iceberg.puffin.BlobMetadata;
import org.apache.iceberg.puffin.Puffin;
import org.apache.iceberg.puffin.PuffinCompressionCodec;
import org.apache.iceberg.puffin.PuffinReader;
import org.apache.iceberg.puffin.PuffinWriter;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
Expand All @@ -201,7 +200,6 @@
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.SerializationUtil;
import org.apache.iceberg.util.SnapshotUtil;
Expand Down Expand Up @@ -637,20 +635,27 @@ private boolean writeColStats(List<ColumnStatistics> colStats, Table tbl) {
long snapshotId = tbl.currentSnapshot().snapshotId();
long snapshotSequenceNumber = tbl.currentSnapshot().sequenceNumber();

colStats.forEach(statsObj -> {
byte[] serializeColStats = SerializationUtils.serialize(statsObj);
puffinWriter.add(
new Blob(
ColumnStatisticsObj.class.getSimpleName(),
ImmutableList.of(1),
snapshotId,
snapshotSequenceNumber,
ByteBuffer.wrap(serializeColStats),
PuffinCompressionCodec.NONE,
ImmutableMap.of("partition",
String.valueOf(statsObj.getStatsDesc().getPartName()))
));
colStats.forEach(stats -> {
boolean isTblLevel = stats.getStatsDesc().isIsTblLevel();

for (Serializable statsObj : isTblLevel ? stats.getStatsObj() : Collections.singletonList(stats)) {
byte[] serializeColStats = SerializationUtils.serialize(statsObj);
puffinWriter.add(
new Blob(
ColumnStatisticsObj.class.getSimpleName(),
ImmutableList.of(isTblLevel ? tbl.spec().schema().findField(
((ColumnStatisticsObj) statsObj).getColName()).fieldId() : 1),
snapshotId,
snapshotSequenceNumber,
ByteBuffer.wrap(serializeColStats),
PuffinCompressionCodec.NONE,
isTblLevel ?
ImmutableMap.of("specId", String.valueOf(tbl.spec().specId())) :
ImmutableMap.of("partition", String.valueOf(stats.getStatsDesc().getPartName()))
));
}
});

puffinWriter.finish();

statisticsFile =
Expand Down Expand Up @@ -693,17 +698,28 @@ private boolean canProvideColStats(Table table, long snapshotId) {
}

@Override
public List<ColumnStatisticsObj> getColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable) {
public List<ColumnStatisticsObj> getColStatistics(org.apache.hadoop.hive.ql.metadata.Table hmsTable,
List<String> colNames) {
Table table = IcebergTableUtil.getTable(conf, hmsTable.getTTable());

Snapshot snapshot = IcebergTableUtil.getTableSnapshot(table, hmsTable);
if (snapshot == null) {
return Lists.newArrayList();
}

ColumnStatistics emptyStats = new ColumnStatistics();
if (snapshot != null) {
return IcebergTableUtil.getColStatsPath(table, snapshot.snapshotId())
.map(statsPath -> readColStats(table, statsPath, null).get(0))
.orElse(emptyStats).getStatsObj();
Predicate<BlobMetadata> filter;
if (colNames != null) {
Set<String> columns = Sets.newHashSet(colNames);
filter = metadata -> {
int specId = Integer.parseInt(metadata.properties().get("specId"));
String column = table.specs().get(specId).schema().findColumnName(metadata.inputFields().get(0));
return columns.contains(column);
};
} else {
filter = null;
}
return emptyStats.getStatsObj();

return IcebergTableUtil.readColStats(table, snapshot.snapshotId(), filter);
}

@Override
Expand All @@ -720,9 +736,10 @@ public AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table hms
MetastoreConf.ConfVars.STATS_NDV_DENSITY_FUNCTION);
double ndvTuner = MetastoreConf.getDoubleVar(getConf(), MetastoreConf.ConfVars.STATS_NDV_TUNER);

List<ColumnStatistics> partStats = IcebergTableUtil.getColStatsPath(table, snapshot.snapshotId())
.map(statsPath -> readColStats(table, statsPath, Sets.newHashSet(partNames)))
.orElse(Collections.emptyList());
Set<String> partitions = Sets.newHashSet(partNames);
Predicate<BlobMetadata> filter = metadata -> partitions.contains(metadata.properties().get("partition"));

List<ColumnStatistics> partStats = IcebergTableUtil.readColStats(table, snapshot.snapshotId(), filter);

partStats.forEach(colStats ->
colStats.getStatsObj().removeIf(statsObj -> !colNames.contains(statsObj.getColName())));
Expand All @@ -736,30 +753,6 @@ public AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table hms
return new AggrStats(colStatsList, partStats.size());
}

private List<ColumnStatistics> readColStats(Table table, Path statsPath, Set<String> partNames) {
List<ColumnStatistics> colStats = Lists.newArrayList();

try (PuffinReader reader = Puffin.read(table.io().newInputFile(statsPath.toString())).build()) {
List<BlobMetadata> blobMetadata = reader.fileMetadata().blobs();

if (partNames != null) {
blobMetadata = blobMetadata.stream()
.filter(metadata -> partNames.contains(metadata.properties().get("partition")))
.collect(Collectors.toList());
}
Iterator<ByteBuffer> it = Iterables.transform(reader.readAll(blobMetadata), Pair::second).iterator();
LOG.info("Using col stats from : {}", statsPath);

while (it.hasNext()) {
byte[] byteBuffer = ByteBuffers.toByteArray(it.next());
colStats.add(SerializationUtils.deserialize(byteBuffer));
}
} catch (Exception e) {
LOG.warn(" Unable to read col stats: ", e);
}
return colStats;
}

@Override
public boolean canComputeQueryUsingStats(Partish partish) {
org.apache.hadoop.hive.ql.metadata.Table hmsTable = partish.getTable();
Expand Down Expand Up @@ -799,22 +792,24 @@ private boolean shouldRewriteColStats(Table tbl) {
private void checkAndMergeColStats(List<ColumnStatistics> statsNew, Table tbl) throws InvalidObjectException {
Long previousSnapshotId = tbl.currentSnapshot().parentId();
if (previousSnapshotId != null && canProvideColStats(tbl, previousSnapshotId)) {
List<ColumnStatistics> statsOld = IcebergTableUtil.getColStatsPath(tbl, previousSnapshotId)
.map(statsPath -> readColStats(tbl, statsPath, null))
.orElse(Collections.emptyList());

boolean isTblLevel = statsNew.get(0).getStatsDesc().isIsTblLevel();
Map<String, ColumnStatistics> oldStatsMap = Maps.newHashMap();

List<?> statsOld = IcebergTableUtil.readColStats(tbl, previousSnapshotId, null);

if (!isTblLevel) {
for (ColumnStatistics statsObjOld : statsOld) {
for (ColumnStatistics statsObjOld : (List<ColumnStatistics>) statsOld) {
oldStatsMap.put(statsObjOld.getStatsDesc().getPartName(), statsObjOld);
}
} else {
statsOld = Collections.singletonList(
new ColumnStatistics(null, (List<ColumnStatisticsObj>) statsOld));
}
for (ColumnStatistics statsObjNew : statsNew) {
String partitionKey = statsObjNew.getStatsDesc().getPartName();
ColumnStatistics statsObjOld = isTblLevel ?
statsOld.get(0) : oldStatsMap.get(partitionKey);
(ColumnStatistics) statsOld.get(0) : oldStatsMap.get(partitionKey);

if (statsObjOld != null && statsObjOld.getStatsObjSize() != 0 && !statsObjNew.getStatsObj().isEmpty()) {
MetaStoreServerUtils.mergeColStats(statsObjNew, statsObjOld);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,20 @@
package org.apache.iceberg.mr.hive;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.time.ZoneId;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.function.BinaryOperator;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
Expand Down Expand Up @@ -77,11 +81,18 @@
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.mr.Catalogs;
import org.apache.iceberg.mr.InputFormatConfig;
import org.apache.iceberg.puffin.BlobMetadata;
import org.apache.iceberg.puffin.Puffin;
import org.apache.iceberg.puffin.PuffinReader;
import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable;
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;
import org.apache.iceberg.types.Conversions;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.SnapshotUtil;
import org.apache.iceberg.util.StructProjection;
Expand Down Expand Up @@ -547,4 +558,31 @@ public static TransformSpec getTransformSpec(Table table, String transformName,
return spec;
}

public static <T> List<T> readColStats(Table table, Long snapshotId, Predicate<BlobMetadata> filter) {
List<T> colStats = Lists.newArrayList();

Optional<Path> statsPath = IcebergTableUtil.getColStatsPath(table, snapshotId);
if (!statsPath.isPresent()) {
return colStats;
}
try (PuffinReader reader = Puffin.read(table.io().newInputFile(statsPath.toString())).build()) {
List<BlobMetadata> blobMetadata = reader.fileMetadata().blobs();

if (filter != null) {
blobMetadata = blobMetadata.stream().filter(filter)
.collect(Collectors.toList());
}
Iterator<ByteBuffer> it = Iterables.transform(reader.readAll(blobMetadata), Pair::second).iterator();
LOG.info("Using col stats from : {}", statsPath);

while (it.hasNext()) {
byte[] byteBuffer = ByteBuffers.toByteArray(it.next());
colStats.add(SerializationUtils.deserialize(byteBuffer));
}
} catch (Exception e) {
LOG.warn(" Unable to read col stats: ", e);
}
return colStats;
}

}
2 changes: 1 addition & 1 deletion ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
Original file line number Diff line number Diff line change
Expand Up @@ -6121,7 +6121,7 @@ public List<ColumnStatisticsObj> getTableColumnStatistics(
List<ColumnStatisticsObj> retv = null;
try {
if (tbl.isNonNative() && tbl.getStorageHandler().canProvideColStatistics(tbl)) {
return tbl.getStorageHandler().getColStatistics(tbl);
return tbl.getStorageHandler().getColStatistics(tbl, colNames);
}
if (checkTransactional) {
AcidUtils.TableSnapshot tableSnapshot = AcidUtils.getTableSnapshot(conf, tbl);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -286,12 +286,19 @@ default boolean canProvidePartitionStatistics(org.apache.hadoop.hive.ql.metadata
/**
* Returns column statistics (upper/lower bounds, number of Null/NaN values, NDVs, histogram).
* @param table table object
* @param colNames list of column names
* @return list of ColumnStatisticsObj objects
*/
default List<ColumnStatisticsObj> getColStatistics(org.apache.hadoop.hive.ql.metadata.Table table) {
default List<ColumnStatisticsObj> getColStatistics(org.apache.hadoop.hive.ql.metadata.Table table,
List<String> colNames) {
return null;
}

@Deprecated
default List<ColumnStatisticsObj> getColStatistics(org.apache.hadoop.hive.ql.metadata.Table table) {
return getColStatistics(table, null);
}

/**
* Returns an aggregated column statistics for the supplied partition list
* @param table table object
Expand Down
Loading