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

Support Spark Column Stats #10659

Merged
merged 9 commits into from
Jul 31, 2024
Merged
Show file tree
Hide file tree
Changes from 3 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 @@ -347,4 +347,12 @@ private boolean executorCacheLocalityEnabledInternal() {
.defaultValue(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT)
.parse();
}

public boolean enableColumnStats() {
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
return confParser
.booleanConf()
.sessionConf(SparkSQLProperties.ENABLE_COLUMN_STATS)
.defaultValue(SparkSQLProperties.ENABLE_COLUMN_STATS_DEFAULT)
.parse();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -90,4 +90,8 @@ private SparkSQLProperties() {}
public static final String EXECUTOR_CACHE_LOCALITY_ENABLED =
"spark.sql.iceberg.executor-cache.locality.enabled";
public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false;

// Controls whether to calculate column statistics and report them to Spark
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
public static final String ENABLE_COLUMN_STATS = "spark.sql.iceberg.enable-column-stats";
public static final boolean ENABLE_COLUMN_STATS_DEFAULT = false;
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
}
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics {
public Statistics estimateStatistics() {
long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum();
long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount);
return new Stats(sizeInBytes, rowsCount);
return new Stats(sizeInBytes, rowsCount, Collections.emptyMap());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* 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.source;

import java.util.Optional;
import java.util.OptionalLong;
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;
import org.apache.spark.sql.connector.read.colstats.Histogram;

class SparkColumnStatistics implements ColumnStatistics {

private final OptionalLong distinctCount;
private final Optional<Object> min;
private final Optional<Object> max;
private final OptionalLong nullCount;
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
private final OptionalLong avgLen;
private final OptionalLong maxLen;
private final Optional<Histogram> histogram;

SparkColumnStatistics(
long distinctCount,
Object min,
Object max,
long nullCount,
long avgLen,
long maxLen,
Histogram histogram) {
this.distinctCount = OptionalLong.of(distinctCount);
this.min = Optional.ofNullable(min);
this.max = Optional.ofNullable(max);
this.nullCount = OptionalLong.of(nullCount);
this.avgLen = OptionalLong.of(avgLen);
this.maxLen = OptionalLong.of(maxLen);
this.histogram = Optional.ofNullable(histogram);
}

@Override
public OptionalLong distinctCount() {
return distinctCount;
}

@Override
public Optional<Object> min() {
return min;
}

@Override
public Optional<Object> max() {
return max;
}

@Override
public OptionalLong nullCount() {
return nullCount;
}

@Override
public OptionalLong avgLen() {
return avgLen;
}

@Override
public OptionalLong maxLen() {
return maxLen;
}

@Override
public Optional<Histogram> histogram() {
return histogram;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,18 @@
import java.util.Map;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import org.apache.iceberg.BlobMetadata;
import org.apache.iceberg.ScanTask;
import org.apache.iceberg.ScanTaskGroup;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SnapshotSummary;
import org.apache.iceberg.StatisticsFile;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.metrics.ScanReport;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.spark.Spark3Util;
import org.apache.iceberg.spark.SparkReadConf;
import org.apache.iceberg.spark.SparkSchemaUtil;
Expand Down Expand Up @@ -75,13 +78,17 @@
import org.apache.iceberg.util.TableScanUtil;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.connector.expressions.FieldReference;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.metric.CustomMetric;
import org.apache.spark.sql.connector.metric.CustomTaskMetric;
import org.apache.spark.sql.connector.read.Batch;
import org.apache.spark.sql.connector.read.Scan;
import org.apache.spark.sql.connector.read.Statistics;
import org.apache.spark.sql.connector.read.SupportsReportStatistics;
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
import org.apache.spark.sql.internal.SQLConf;
import org.apache.spark.sql.types.StructType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -91,6 +98,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics {

private final JavaSparkContext sparkContext;
private final Table table;
private final SparkSession spark;
private final SparkReadConf readConf;
private final boolean caseSensitive;
private final Schema expectedSchema;
Expand All @@ -111,6 +119,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics {
Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch());
SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema);

this.spark = spark;
this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
this.table = table;
this.readConf = readConf;
Expand Down Expand Up @@ -175,7 +184,37 @@ public Statistics estimateStatistics() {
protected Statistics estimateStatistics(Snapshot snapshot) {
// its a fresh table, no data
if (snapshot == null) {
return new Stats(0L, 0L);
return new Stats(0L, 0L, Collections.emptyMap());
}

boolean cboEnabled =
Boolean.parseBoolean(spark.conf().get(SQLConf.CBO_ENABLED().key(), "false"));
Map<NamedReference, ColumnStatistics> colStatsMap = null;

Choose a reason for hiding this comment

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

Thinking back to this change, may be cleaner to just set this to empty map and the reassign to Maps.newHashMap on 195 as is.

colStatsMap = Collections.emptyMap
if (report & cbo) {
colStatsMap = Maps.newHashMap
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed. Thanks!

if (readConf.enableColumnStats() && cboEnabled) {
colStatsMap = Maps.newHashMap();
List<StatisticsFile> files = table.statisticsFiles();
if (!files.isEmpty()) {
List<BlobMetadata> metadataList = (files.get(0)).blobMetadata();
huaxingao marked this conversation as resolved.
Show resolved Hide resolved

for (BlobMetadata blobMetadata : metadataList) {
int id = blobMetadata.fields().get(0);
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we check if there is more than one field here(for example, ndv stats is collected for say field1 and field2) and not propagate the stats if so?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I can't find detailed documentation for BlobMetadata in the spec. My understanding is that for the blob type apache-datasketches-theta-v1, ndv is required. I assume that ndv is set in the properties as shown in the following example. If my assumption is correct, it seems to me that we can only have one field in the fields, establishing a one-to-one relationship with the ndv in the properties.

new GenericBlobMetadata(
    APACHE_DATASKETCHES_THETA_V1,
    snapshotId,
    1,
    ImmutableList.of(1),
    ImmutableMap.of("ndv", "4"))

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Member

Choose a reason for hiding this comment

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

Correct, the apache-datasketches-theta-v1 should be calculated on one field.
And yes, there should be the ndv property set. The property may seem somewhat redundant within the Puffin file, but allow faster access to the information at SELECT-time. More importantly, the properties are propagated to the table metadata and so a query planner accesses the NDV information without opening the Puffin file at all.

String colName = table.schema().findColumnName(id);
NamedReference ref = FieldReference.column(colName);
huaxingao marked this conversation as resolved.
Show resolved Hide resolved

long ndv = 0;
String ndvStr = blobMetadata.properties().get("ndv");
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
if (ndvStr != null && !ndvStr.isEmpty()) {
ndv = Long.parseLong(blobMetadata.properties().get("ndv"));
} else {
LOG.debug("ndv is not set in BlobMetadata for column {}", colName);
}

// TODO: Fill min, max and null from the manifest file
Copy link
Member

Choose a reason for hiding this comment

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

Can we remove the TODO and make an Issue?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Created issue

ColumnStatistics colStats = new SparkColumnStatistics(ndv, null, null, 0L, 0L, 0L, null);
Copy link
Contributor

Choose a reason for hiding this comment

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

When ndv is not set, we are sending in 0. Is that intended?

Copy link
Member

Choose a reason for hiding this comment

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

Yeah I didn't see notes in the Spark docs about what 0 would mean in this context.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Checked Spark code, should be None if ndv is not available. I changed the code accordingly.

Copy link
Member

Choose a reason for hiding this comment

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

Looks like it is still 0?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changed to null.


colStatsMap.put(ref, colStats);
}
}
}

// estimate stats using snapshot summary only for partitioned tables
Expand All @@ -186,12 +225,13 @@ protected Statistics estimateStatistics(Snapshot snapshot) {
snapshot.snapshotId(),
table.name());
long totalRecords = totalRecords(snapshot);
return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords);
return new Stats(
SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords, colStatsMap);
}

long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum();
long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount);
return new Stats(sizeInBytes, rowsCount);
return new Stats(sizeInBytes, rowsCount, colStatsMap);
}

private long totalRecords(Snapshot snapshot) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,21 @@
*/
package org.apache.iceberg.spark.source;

import java.util.Map;
import java.util.OptionalLong;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.read.Statistics;
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;

class Stats implements Statistics {
private final OptionalLong sizeInBytes;
private final OptionalLong numRows;
private final Map<NamedReference, ColumnStatistics> colstats;

Stats(long sizeInBytes, long numRows) {
Stats(long sizeInBytes, long numRows, Map<NamedReference, ColumnStatistics> colstats) {
this.sizeInBytes = OptionalLong.of(sizeInBytes);
this.numRows = OptionalLong.of(numRows);
this.colstats = colstats;
}

@Override
Expand All @@ -39,4 +44,9 @@ public OptionalLong sizeInBytes() {
public OptionalLong numRows() {
return numRows;
}

@Override
public Map<NamedReference, ColumnStatistics> columnStats() {
return colstats;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,21 @@
import static org.apache.spark.sql.functions.expr;
import static org.assertj.core.api.Assertions.assertThat;

import java.util.List;
import java.util.Map;
import org.apache.iceberg.GenericBlobMetadata;
import org.apache.iceberg.GenericStatisticsFile;
import org.apache.iceberg.Parameter;
import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Parameters;
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.spark.Spark3Util;
import org.apache.iceberg.spark.SparkCatalogConfig;
import org.apache.iceberg.spark.SparkSQLProperties;
import org.apache.iceberg.spark.TestBaseWithCatalog;
import org.apache.iceberg.spark.functions.BucketFunction;
import org.apache.iceberg.spark.functions.DaysFunction;
Expand All @@ -44,6 +51,7 @@
import org.apache.iceberg.spark.functions.TruncateFunction;
import org.apache.iceberg.spark.functions.YearsFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
Expand All @@ -60,6 +68,8 @@
import org.apache.spark.sql.connector.read.ScanBuilder;
import org.apache.spark.sql.connector.read.Statistics;
import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters;
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;
import org.apache.spark.sql.internal.SQLConf;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -130,6 +140,57 @@ public void testEstimatedRowCount() throws NoSuchTableException {
assertThat(stats.numRows().getAsLong()).isEqualTo(10000L);
}

@TestTemplate
public void testColStats() throws NoSuchTableException {
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName);

List<SimpleRecord> records =
Lists.newArrayList(
new SimpleRecord(1, "a"),
new SimpleRecord(2, "b"),
new SimpleRecord(3, "a"),
new SimpleRecord(4, "b"));
spark
.createDataset(records, Encoders.bean(SimpleRecord.class))
.coalesce(1)
.writeTo(tableName)
.append();

Table table = validationCatalog.loadTable(tableIdent);
long snapshotId = table.currentSnapshot().snapshotId();

GenericStatisticsFile statisticsFile =
new GenericStatisticsFile(
snapshotId,
"/test/statistics/file.puffin",
100,
42,
ImmutableList.of(
new GenericBlobMetadata(
"stats-type", snapshotId, 1, ImmutableList.of(1), ImmutableMap.of("ndv", "4")),
new GenericBlobMetadata(
"stats-type",
snapshotId,
1,
ImmutableList.of(2),
ImmutableMap.of("ndv", "2"))));

table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit();
SparkScanBuilder scanBuilder =
new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty());
SparkScan scan = (SparkScan) scanBuilder.build();

Map<String, String> sqlConf = ImmutableMap.of(SparkSQLProperties.ENABLE_COLUMN_STATS, "true");

Map<String, String> sqlConf2 =
ImmutableMap.of(
SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.ENABLE_COLUMN_STATS, "true");

checkStatistics(scan, 4L, false);
withSQLConf(sqlConf, () -> checkStatistics(scan, 4L, false));
withSQLConf(sqlConf2, () -> checkStatistics(scan, 4L, true));
}

@TestTemplate
public void testUnpartitionedYears() throws Exception {
createUnpartitionedTable(spark, tableName);
Expand Down Expand Up @@ -734,6 +795,19 @@ private Expression[] expressions(Expression... expressions) {
return expressions;
}

private void checkStatistics(SparkScan scan, long expectedRowCount, boolean expectedColumnStats) {
huaxingao marked this conversation as resolved.
Show resolved Hide resolved
Copy link
Member

@RussellSpitzer RussellSpitzer Jul 26, 2024

Choose a reason for hiding this comment

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

This may be over parameterized, we only use this function 2 times and we use them for relatively different things. So I'm not sure it makes things clearer. It's also missing a bit to be completely parameterized.

So if it was completely parameterized we would want probably two functions like

checkStatisticsNotReported(scan)
checkReportedStatistics(scan, rowCount, map<String, Int> distinctValueCounts)

As it is we hard code in the column name "id" and the distinct value count "4l" but parameterize the expectedRowCount. So we couldn't really re-use this function for any reason.

Now since we only actually use this function for 1 example above it may be ok to not even parameterize it at all. So I think there are 2 ways to go here.

Find more uses for the function and fully parameterize
Deparameterize and inline.

I think we have a few more use cases we should probably test out just for fun so maybe route 1 is better?

For example

Test where 1 column has NDV and the other does not.
Test where stats reporting is enabled, but the table does not have stats
Test with different distinct values for columns in the table

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for the suggestion! Changed to route 1

Statistics stats = scan.estimateStatistics();
assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount);

Map<NamedReference, ColumnStatistics> columnStats = stats.columnStats();
if (expectedColumnStats) {
assertThat(columnStats.get(FieldReference.column("id")).distinctCount().getAsLong())
.isEqualTo(4L);
assertThat(columnStats.get(FieldReference.column("data")).distinctCount().getAsLong())
.isEqualTo(2L);
}
}

private static LiteralValue<Integer> intLit(int value) {
return LiteralValue.apply(value, DataTypes.IntegerType);
}
Expand Down
Loading