szehon-ho commented on code in PR #10659: URL: https://github.com/apache/iceberg/pull/10659#discussion_r1674693714
########## spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java: ########## @@ -97,6 +117,36 @@ public static Object[][] parameters() { }; } + @BeforeAll Review Comment: Why is it needed? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java: ########## @@ -175,7 +181,25 @@ 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, Maps.newHashMap()); + } + + Map<NamedReference, ColumnStatistics> map = Maps.newHashMap(); + + if (readConf.enableColumnStats()) { + List<StatisticsFile> files = table.statisticsFiles(); + if (!files.isEmpty()) { + List<BlobMetadata> metadataList = (files.get(0)).blobMetadata(); + + for (BlobMetadata blobMetadata : metadataList) { + long ndv = Long.parseLong(blobMetadata.properties().get("ndv")); + ColumnStatistics colStats = new ColStats(ndv, null, null, 0L, 0L, 0L, null); Review Comment: Should we fill the other values from the manifest file? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java: ########## @@ -175,7 +181,25 @@ 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, Maps.newHashMap()); + } + + Map<NamedReference, ColumnStatistics> map = Maps.newHashMap(); Review Comment: Can we have a more descriptive name? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java: ########## @@ -175,7 +181,25 @@ 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, Maps.newHashMap()); + } + + Map<NamedReference, ColumnStatistics> map = Maps.newHashMap(); + + if (readConf.enableColumnStats()) { + List<StatisticsFile> files = table.statisticsFiles(); + if (!files.isEmpty()) { + List<BlobMetadata> metadataList = (files.get(0)).blobMetadata(); + + for (BlobMetadata blobMetadata : metadataList) { + long ndv = Long.parseLong(blobMetadata.properties().get("ndv")); Review Comment: I think we should be more defensive if there is no value, or numberformatexception? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java: ########## @@ -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 column statistics are enabled when estimating statistics Review Comment: the comment seems not meaningful at first read. Can we be more precise? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/ColStats.java: ########## @@ -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 ColStats implements ColumnStatistics { Review Comment: Nit: this is a bit confusing, ColStats is just abbreviate the Spark class name. How about SparkColumnStatistics? (Looks like in the package there are a lot of class called SparkXXX in this situation) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org