-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-41378][SQL] Support Column Stats in DS v2 #38904
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
Closed
Closed
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
5b5019f
Suport Col Stats in DS v2
huaxingao c77252e
fix mima
huaxingao 2a1422b
address comments
huaxingao a6089ec
remove unnecessary change
huaxingao 0cddab9
address comments
huaxingao 5a872d8
address comments and fix tests failure
huaxingao File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
...catalyst/src/main/java/org/apache/spark/sql/connector/read/colstats/ColumnStatistics.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,83 @@ | ||
| /* | ||
| * 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.spark.sql.connector.read.colstats; | ||
huaxingao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| import java.util.Optional; | ||
| import java.util.OptionalLong; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
| import org.apache.spark.sql.connector.read.Statistics; | ||
|
|
||
| /** | ||
| * An interface to represent column statistics, which is part of | ||
| * {@link Statistics}. | ||
| * | ||
| * @since 3.4.0 | ||
| */ | ||
| @Evolving | ||
| public interface ColumnStatistics { | ||
|
|
||
| /** | ||
| * @return number of distinct values in the column | ||
| */ | ||
| default OptionalLong distinctCount() { | ||
| return OptionalLong.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return minimum value in the column | ||
| */ | ||
| default Optional<Object> min() { | ||
| return Optional.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return maximum value in the column | ||
| */ | ||
| default Optional<Object> max() { | ||
| return Optional.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return number of nulls in the column | ||
| */ | ||
| default OptionalLong nullCount() { | ||
| return OptionalLong.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return average length of the values in the column | ||
| */ | ||
| default OptionalLong avgLen() { | ||
huaxingao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return OptionalLong.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return maximum length of the values in the column | ||
| */ | ||
| default OptionalLong maxLen() { | ||
| return OptionalLong.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * @return histogram of the values in the column | ||
| */ | ||
| default Optional<Histogram> histogram() { | ||
huaxingao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return Optional.empty(); | ||
| } | ||
| } | ||
40 changes: 40 additions & 0 deletions
40
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/colstats/Histogram.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,40 @@ | ||
| /* | ||
| * 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.spark.sql.connector.read.colstats; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * An interface to represent an equi-height histogram, which is a part of | ||
| * {@link ColumnStatistics}. Equi-height histogram represents the distribution of | ||
| * a column's values by a sequence of bins. | ||
| * | ||
| * @since 3.4.0 | ||
| */ | ||
| @Evolving | ||
| public interface Histogram { | ||
| /** | ||
| * @return number of rows in each bin | ||
| */ | ||
| double height(); | ||
huaxingao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| /** | ||
| * @return equi-height histogram bins | ||
| */ | ||
| HistogramBin[] bins(); | ||
| } | ||
43 changes: 43 additions & 0 deletions
43
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/colstats/HistogramBin.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,43 @@ | ||
| /* | ||
| * 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.spark.sql.connector.read.colstats; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
|
|
||
| /** | ||
| * An interface to represent a bin in an equi-height histogram. | ||
| * | ||
| * @since 3.4.0 | ||
| */ | ||
| @Evolving | ||
| public interface HistogramBin { | ||
| /** | ||
| * @return lower bound of the value range in this bin | ||
| */ | ||
| double lo(); | ||
|
|
||
| /** | ||
| * @return higher bound of the value range in this bin | ||
| */ | ||
| double hi(); | ||
|
|
||
| /** | ||
| * @return approximate number of distinct values in this bin | ||
| */ | ||
| long ndv(); | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,7 @@ package org.apache.spark.sql.connector.catalog | |
| import java.time.{Instant, ZoneId} | ||
| import java.time.temporal.ChronoUnit | ||
| import java.util | ||
| import java.util.OptionalLong | ||
| import java.util.{Optional, OptionalLong} | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
|
|
@@ -33,6 +33,7 @@ import org.apache.spark.sql.connector.distributions.{Distribution, Distributions | |
| import org.apache.spark.sql.connector.expressions._ | ||
| import org.apache.spark.sql.connector.metric.{CustomMetric, CustomTaskMetric} | ||
| import org.apache.spark.sql.connector.read._ | ||
| import org.apache.spark.sql.connector.read.colstats.{ColumnStatistics, Histogram, HistogramBin} | ||
| import org.apache.spark.sql.connector.read.partitioning.{KeyGroupedPartitioning, Partitioning, UnknownPartitioning} | ||
| import org.apache.spark.sql.connector.write._ | ||
| import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} | ||
|
|
@@ -273,7 +274,19 @@ abstract class InMemoryBaseTable( | |
| } | ||
| } | ||
|
|
||
| case class InMemoryStats(sizeInBytes: OptionalLong, numRows: OptionalLong) extends Statistics | ||
| case class InMemoryStats( | ||
| sizeInBytes: OptionalLong, | ||
| numRows: OptionalLong, | ||
| override val columnStats: Optional[util.Map[NamedReference, ColumnStatistics]]) | ||
| extends Statistics | ||
|
|
||
| case class InMemoryColumnStats( | ||
| override val distinctCount: OptionalLong, | ||
| override val nullCount: OptionalLong) extends ColumnStatistics | ||
|
|
||
| case class InMemoryHistogramBin(lo: Double, hi: Double, ndv: Long) extends HistogramBin | ||
|
|
||
| case class InMemoryHistogram(height: Double, bins: Array[HistogramBin]) extends Histogram | ||
|
|
||
| abstract class BatchScanBaseClass( | ||
| var data: Seq[InputPartition], | ||
|
|
@@ -285,7 +298,7 @@ abstract class InMemoryBaseTable( | |
|
|
||
| override def estimateStatistics(): Statistics = { | ||
| if (data.isEmpty) { | ||
| return InMemoryStats(OptionalLong.of(0L), OptionalLong.of(0L)) | ||
| return InMemoryStats(OptionalLong.of(0L), OptionalLong.of(0L), Optional.empty()) | ||
| } | ||
|
|
||
| val inputPartitions = data.map(_.asInstanceOf[BufferedRows]) | ||
|
|
@@ -294,7 +307,39 @@ abstract class InMemoryBaseTable( | |
| val objectHeaderSizeInBytes = 12L | ||
| val rowSizeInBytes = objectHeaderSizeInBytes + schema.defaultSize | ||
| val sizeInBytes = numRows * rowSizeInBytes | ||
| InMemoryStats(OptionalLong.of(sizeInBytes), OptionalLong.of(numRows)) | ||
|
|
||
| val numOfCols = tableSchema.fields.length | ||
| val dataTypes = tableSchema.fields.map(_.dataType) | ||
| val colValueSets = new Array[util.HashSet[Object]](numOfCols) | ||
| val numOfNulls = new Array[Long](numOfCols) | ||
| for (i <- 0 until numOfCols) { | ||
| colValueSets(i) = new util.HashSet[Object] | ||
| } | ||
|
|
||
| inputPartitions.foreach(inputPartition => | ||
| inputPartition.rows.foreach(row => | ||
| for (i <- 0 until numOfCols) { | ||
| colValueSets(i).add(row.get(i, dataTypes(i))) | ||
| if (row.isNullAt(i)) { | ||
| numOfNulls(i) += 1 | ||
| } | ||
| } | ||
| ) | ||
| ) | ||
|
|
||
| val map = new util.HashMap[NamedReference, ColumnStatistics]() | ||
| val colNames = tableSchema.fields.map(_.name) | ||
| var i = 0 | ||
| for (col <- colNames) { | ||
| val fieldReference = FieldReference(col) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| val colStats = InMemoryColumnStats( | ||
| OptionalLong.of(colValueSets(i).size()), | ||
| OptionalLong.of(numOfNulls(i))) | ||
| map.put(fieldReference, colStats) | ||
| i = i + 1 | ||
| } | ||
|
|
||
| InMemoryStats(OptionalLong.of(sizeInBytes), OptionalLong.of(numRows), Optional.of(map)) | ||
| } | ||
|
|
||
| override def outputPartitioning(): Partitioning = { | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we use empty map to indicate no column stats? Catalyst column stats also use map directly.