Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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.Map;
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<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.

return Optional.empty();
}
}
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;

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() {
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() {
return Optional.empty();
}
}
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();

/**
* @return equi-height histogram bins
*/
HistogramBin[] bins();
}
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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@
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}
Expand Down Expand Up @@ -91,7 +91,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 +142,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 +173,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 +214,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().getAsLong) 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().getAsLong) 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 @@ -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

Expand All @@ -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}
Expand Down Expand Up @@ -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],
Expand All @@ -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])
Expand All @@ -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)
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.

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 = {
Expand Down
Loading