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

[SPARK-41378][SQL] Support Column Stats in DS v2 #38904

Closed
wants to merge 6 commits into from

Conversation

huaxingao
Copy link
Contributor

What changes were proposed in this pull request?

Support Col Stats in DS v2

Why are the changes needed?

Currently only Table stats is supported in DS V2. Column stats should be supported too.

Does this PR introduce any user-facing change?

Yes
ColumnStatistics interface is introduced and added as a part of Statistics

How was this patch tested?

new test

@github-actions github-actions bot added the SQL label Dec 5, 2022
@github-actions github-actions bot added the BUILD label Dec 5, 2022
@github-actions github-actions bot removed the BUILD label Dec 5, 2022
@dongjoon-hyun
Copy link
Member

Thank you for updates.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

@huaxingao
Copy link
Contributor Author

also cc @cloud-fan

Copy link
Member

@sunchao sunchao left a comment

Choose a reason for hiding this comment

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

Also curious how this is to be used by Spark

Comment on lines 321 to 326
// put some fake data for testing only
val bin1 = InMemoryHistogramBin(1, 2, 5L)
val bin2 = InMemoryHistogramBin(3, 4, 5L)
val bin3 = InMemoryHistogramBin(5, 6, 5L)
val bin4 = InMemoryHistogramBin(7, 8, 5L)
val bin5 = InMemoryHistogramBin(9, 10, 5L)
Copy link
Member

Choose a reason for hiding this comment

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

Hmm, not sure if fake statistics cause will cause unexpected result later? Ideally we should compute real statistics like sizeInBytes and numRows from data .

Copy link
Member

Choose a reason for hiding this comment

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

If it's too complicated, maybe we can just compute max/min for test purpose.

Copy link
Contributor Author

@huaxingao huaxingao Dec 7, 2022

Choose a reason for hiding this comment

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

I removed the fake data and computed NDV and null Count for testing purpose.

@huaxingao
Copy link
Contributor Author

Also curious how this is to be used by Spark

The newly added ColumnStatistics is converted to logical ColumnStat in this method and is used in CBO

/**
* @return number of distinct values in the column
*/
default Optional<BigInteger> distinctCount() {
Copy link
Contributor

Choose a reason for hiding this comment

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

In CBO, we need the distinct count as a BigInteger because the estimated row count can be very large due to join, generate, etc. But for a single table, do we really need BigInteger?

Copy link
Member

Choose a reason for hiding this comment

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

So, do you suggest java.util.OptionalLong?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yup

Copy link
Member

Choose a reason for hiding this comment

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

+1

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

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 OptionalLong. Thanks for the suggestion!

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM from my side.

@dongjoon-hyun
Copy link
Member

Merged to master for Apache Spark 3.4.0. Thank you, @huaxingao and all!

@huaxingao
Copy link
Contributor Author

Thank you all very much!

@huaxingao huaxingao deleted the colStats branch December 7, 2022 21:23
@@ -31,4 +35,7 @@
public interface Statistics {
OptionalLong sizeInBytes();
OptionalLong numRows();
default Optional<Map<NamedReference, ColumnStatistics>> columnStats() {
Copy link
Contributor

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.

val colNames = tableSchema.fields.map(_.name)
var i = 0
for (col <- colNames) {
val fieldReference = FieldReference(col)
Copy link
Contributor

Choose a reason for hiding this comment

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

FieldReference.column(col) as it's plain column name, while FieldReference.apply parses the string.

@@ -2772,6 +2773,26 @@ class DataSourceV2SQLSuiteV1Filter
}
}

test("SPARK-41378: test column stats") {
Copy link
Member

Choose a reason for hiding this comment

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

This test fails with Scala 2.13:

- SPARK-41378: test column stats *** FAILED *** (19 milliseconds)
  5 did not equal 3 (DataSourceV2SQLSuite.scala:2789)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
  at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
  at org.scalatest.Assertions$.newAssertionFailedException(Assertions.scala:1231)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:1295)
  at org.apache.spark.sql.connector.DataSourceV2SQLSuiteV1Filter$$anonfun$$nestedInanonfun$new$386$1.applyOrElse(DataSourceV2SQLSuite.scala:2789)
  at org.apache.spark.sql.connector.DataSourceV2SQLSuiteV1Filter$$anonfun$$nestedInanonfun$new$386$1.applyOrElse(DataSourceV2SQLSuite.scala:2782)
  at scala.PartialFunction$Lifted.apply(PartialFunction.scala:338)
  at scala.PartialFunction$Lifted.apply(PartialFunction.scala:334)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$collect$1(TreeNode.scala:326)
  at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$collect$1$adapted(TreeNode.scala:326)
  at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:285)
  at org.apache.spark.sql.catalyst.trees.TreeNode.collect(TreeNode.scala:326)
  at org.apache.spark.sql.connector.DataSourceV2SQLSuiteV1Filter.$anonfun$new$386(DataSourceV2SQLSuite.scala:2782)
  at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
  at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
  at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
  at org.scalatest.Transformer.apply(Transformer.scala:22)
  at org.scalatest.Transformer.apply(Transformer.scala:20)
  at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
  at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:207)
  at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
  at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)

https://github.com/apache/spark/actions/runs/3670384591/jobs/6204890447
https://github.com/apache/spark/actions/runs/3665545037/jobs/6196700142
https://github.com/apache/spark/actions/runs/3660066892/jobs/6186794437

Mind taking a look please?

Copy link
Member

Choose a reason for hiding this comment

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

Let me take a look at this.

Copy link
Member

Choose a reason for hiding this comment

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

cloud-fan pushed a commit that referenced this pull request Dec 12, 2022
### What changes were proposed in this pull request?
follow-up PR

### Why are the changes needed?
to address comments
#38904 (comment)
#38904 (comment)

### Does this PR introduce _any_ user-facing change?

Change the return type of `columnStats()` from `Optional<Map>` to `Map`.

### How was this patch tested?
existing test

Closes #39027 from huaxingao/colstats_followup.

Authored-by: huaxingao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
beliefer pushed a commit to beliefer/spark that referenced this pull request Dec 18, 2022
### What changes were proposed in this pull request?
Support Col Stats in DS v2

### Why are the changes needed?
Currently only Table stats is supported in DS V2. Column stats should be supported too.

### Does this PR introduce _any_ user-facing change?
Yes
`ColumnStatistics` interface is introduced and added as a part of `Statistics`

### How was this patch tested?
new test

Closes apache#38904 from huaxingao/colStats.

Authored-by: huaxingao <huaxin_gao@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
beliefer pushed a commit to beliefer/spark that referenced this pull request Dec 18, 2022
### What changes were proposed in this pull request?
follow-up PR

### Why are the changes needed?
to address comments
apache#38904 (comment)
apache#38904 (comment)

### Does this PR introduce _any_ user-facing change?

Change the return type of `columnStats()` from `Optional<Map>` to `Map`.

### How was this patch tested?
existing test

Closes apache#39027 from huaxingao/colstats_followup.

Authored-by: huaxingao <huaxin_gao@apple.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
7 participants