Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 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 @@ -17,9 +17,13 @@

package org.apache.spark.sql.connector.read;

import java.util.HashMap;
import java.util.Optional;
import java.util.OptionalLong;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics;

/**
* An interface to represent statistics for a data source, which is returned by
Expand All @@ -31,4 +35,7 @@
public interface Statistics {
OptionalLong sizeInBytes();
OptionalLong numRows();
default Optional<HashMap<NamedReference, ColumnStatistics>> columnStats() {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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;
import java.math.BigInteger;
import java.util.Optional;
import java.util.OptionalLong;

/**
* An interface to represent column statistics, which is part of
* {@link Statistics}.
*
* @since 3.4.0
*/
@Evolving
public interface ColumnStatistics {
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!

return Optional.empty();
}

default Optional<Object> min() {
return Optional.empty();
}

default Optional<Object> max() {
return Optional.empty();
}

default Optional<BigInteger> nullCount() {
return Optional.empty();
}

default OptionalLong avgLen() {
return OptionalLong.empty();
}

default OptionalLong maxLen() {
return OptionalLong.empty();
}

default Optional<Histogram> histogram() {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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 {
double height();
HistogramBin[] bins();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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 {
double lo();
double hi();
long ndv();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,12 @@
package org.apache.spark.sql.execution.datasources.v2

import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, SortOrder}
import org.apache.spark.sql.catalyst.plans.logical.{ExposesMetadataColumns, LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression, SortOrder}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, ExposesMetadataColumns, Histogram, HistogramBin, LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.catalyst.util.{truncatedString, CharVarcharUtils}
import org.apache.spark.sql.connector.catalog.{CatalogPlugin, FunctionCatalog, Identifier, MetadataColumn, SupportsMetadataColumns, Table, TableCapability}
import org.apache.spark.sql.connector.read.{Scan, Statistics => V2Statistics, SupportsReportStatistics}
import org.apache.spark.sql.connector.read.{Scan, SupportsReportStatistics}
import org.apache.spark.sql.connector.read.{Statistics => V2Statistics}
import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.Utils
Expand Down Expand Up @@ -91,7 +92,7 @@ case class DataSourceV2Relation(
table.asReadable.newScanBuilder(options).build() match {
case r: SupportsReportStatistics =>
val statistics = r.estimateStatistics()
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes)
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes, output)
case _ =>
Statistics(sizeInBytes = conf.defaultSizeInBytes)
}
Expand Down Expand Up @@ -142,7 +143,7 @@ case class DataSourceV2ScanRelation(
scan match {
case r: SupportsReportStatistics =>
val statistics = r.estimateStatistics()
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes)
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes, output)
case _ =>
Statistics(sizeInBytes = conf.defaultSizeInBytes)
}
Expand Down Expand Up @@ -173,7 +174,7 @@ case class StreamingDataSourceV2Relation(
override def computeStats(): Statistics = scan match {
case r: SupportsReportStatistics =>
val statistics = r.estimateStatistics()
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes)
DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes, output)
case _ =>
Statistics(sizeInBytes = conf.defaultSizeInBytes)
}
Expand Down Expand Up @@ -214,14 +215,52 @@ object DataSourceV2Relation {
def transformV2Stats(
v2Statistics: V2Statistics,
defaultRowCount: Option[BigInt],
defaultSizeInBytes: Long): Statistics = {
defaultSizeInBytes: Long,
output: Seq[Attribute] = Seq.empty): Statistics = {
val numRows: Option[BigInt] = if (v2Statistics.numRows().isPresent) {
Some(v2Statistics.numRows().getAsLong)
} else {
defaultRowCount
}

var colStats: Seq[(Attribute, ColumnStat)] = Seq.empty[(Attribute, ColumnStat)]
if (v2Statistics.columnStats().isPresent) {
val v2ColumnStat = v2Statistics.columnStats().get()
val keys = v2ColumnStat.keySet()

keys.forEach(key => {
val colStat = v2ColumnStat.get(key)
val distinct: Option[BigInt] =
if (colStat.distinctCount().isPresent) Some(colStat.distinctCount().get) else None
val min: Option[Any] = if (colStat.min().isPresent) Some(colStat.min().get) else None
val max: Option[Any] = if (colStat.max().isPresent) Some(colStat.max().get) else None
val nullCount: Option[BigInt] =
if (colStat.nullCount().isPresent) Some(colStat.nullCount().get()) else None
val avgLen: Option[Long] =
if (colStat.avgLen().isPresent) Some(colStat.avgLen().getAsLong) else None
val maxLen: Option[Long] =
if (colStat.maxLen().isPresent) Some(colStat.maxLen().getAsLong) else None
val histogram = if (colStat.histogram().isPresent) {
val v2Histogram = colStat.histogram().get()
val bins = v2Histogram.bins()
Some(Histogram(v2Histogram.height(),
bins.map(bin => HistogramBin(bin.lo, bin.hi, bin.ndv))))
} else {
None
}

val catalystColStat = ColumnStat(distinct, min, max, nullCount, avgLen, maxLen, histogram)

output.foreach(attribute => {
if (attribute.name.equals(key.describe())) {
colStats = colStats :+ (attribute -> catalystColStat)
}
})
})
}
Statistics(
sizeInBytes = v2Statistics.sizeInBytes().orElse(defaultSizeInBytes),
rowCount = numRows)
rowCount = numRows,
attributeStats = AttributeMap(colStats))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@

package org.apache.spark.sql.connector.catalog

import java.math.BigInteger
import java.time.{Instant, ZoneId}
import java.time.temporal.ChronoUnit
import java.util
import java.util.OptionalLong
import java.util.{HashMap, Optional, OptionalLong}

import scala.collection.mutable

Expand All @@ -33,6 +34,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}
Expand Down Expand Up @@ -273,7 +275,24 @@ abstract class InMemoryBaseTable(
}
}

case class InMemoryStats(sizeInBytes: OptionalLong, numRows: OptionalLong) extends Statistics
case class InMemoryStats(
sizeInBytes: OptionalLong,
numRows: OptionalLong,
override val columnStats: Optional[HashMap[NamedReference, ColumnStatistics]])
extends Statistics

case class InMemoryColumnStats (
override val distinctCount: Optional[BigInteger],
override val min: Optional[AnyRef],
override val max: Optional[AnyRef],
override val nullCount: Optional[BigInteger],
override val avgLen: OptionalLong,
override val maxLen: OptionalLong,
override val histogram: Optional[Histogram]) 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],
Expand All @@ -285,7 +304,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])
Expand All @@ -294,7 +313,30 @@ abstract class InMemoryBaseTable(
val objectHeaderSizeInBytes = 12L
val rowSizeInBytes = objectHeaderSizeInBytes + schema.defaultSize
val sizeInBytes = numRows * rowSizeInBytes
InMemoryStats(OptionalLong.of(sizeInBytes), OptionalLong.of(numRows))

val map = new util.HashMap[NamedReference, ColumnStatistics]()
val colNames = readSchema.fields.map(_.name)
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.

// 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.

val colStats = InMemoryColumnStats(
Optional.of[BigInteger](BigInteger.valueOf(5)),
Optional.of[AnyRef](Integer.valueOf(0)),
Optional.of[AnyRef](Integer.valueOf(5)),
Optional.of[BigInteger](BigInteger.valueOf(0)),
OptionalLong.of(111L),
OptionalLong.of(1111L),
Optional.of[Histogram](InMemoryHistogram(5, Array(bin1, bin2, bin3, bin4, bin5)))
)
map.put(fieldReference, colStats)
}

InMemoryStats(OptionalLong.of(sizeInBytes), OptionalLong.of(numRows), Optional.of(map))
}

override def outputPartitioning(): Partitioning = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM
import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership
import org.apache.spark.sql.errors.QueryErrorsBase
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION}
Expand Down Expand Up @@ -2772,6 +2773,39 @@ 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.

spark.sql("CREATE TABLE testcat.test (id bigint NOT NULL, data string)")
spark.sql("INSERT INTO testcat.test values (1, 'test1'), (2, 'test2'), (3, 'test3')," +
" (4, 'test4'), (5, 'test5')")
val df = spark.sql("select * from testcat.test")

df.queryExecution.optimizedPlan.collect {
case scan: DataSourceV2ScanRelation =>
val stats = scan.stats
assert(stats.sizeInBytes == 200)
assert(stats.rowCount.get == 5)
val colStats = stats.attributeStats.values.toArray
assert(colStats.length == 2)
colStats.foreach(stat => {
assert(stat.distinctCount.get == 5)
assert(stat.min.get == 0)
assert(stat.max.get == 5)
assert(stat.nullCount.get == 0)
assert(stat.avgLen.get == 111L)
assert(stat.maxLen.get == 1111L)
assert(stat.histogram.get.height == 5)
val bins = stat.histogram.get.bins
var i = 0
bins.foreach(bin => {
assert(bin.hi == 2.0 + 2 * i)
assert(bin.lo == 1.0 + 2 * i)
assert(bin.ndv == 5)
i += 1
})
})
}
}

private def testNotSupportedV2Command(sqlCommand: String, sqlParams: String): Unit = {
checkError(
exception = intercept[AnalysisException] {
Expand Down