-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17073] [SQL] generate column-level statistics #15090
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
Changes from all commits
b9d2af6
18f7bfd
230f1d3
924a41d
22fc9fc
10b51a8
b5cd2ff
f279370
12d94cd
d1def03
410125a
b43115c
bb19f72
2b64549
08df669
7cd8f14
377a491
1d3dd62
3335af6
06819dd
95c2d2f
734abad
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,12 @@ | |
|
|
||
| package org.apache.spark.sql.catalyst.plans.logical | ||
|
|
||
| import org.apache.commons.codec.binary.Base64 | ||
|
|
||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
| /** | ||
| * Estimates of various statistics. The default estimation logic simply lazily multiplies the | ||
| * corresponding statistic produced by the children. To override this behavior, override | ||
|
|
@@ -32,19 +38,80 @@ package org.apache.spark.sql.catalyst.plans.logical | |
| * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it | ||
| * defaults to the product of children's `sizeInBytes`. | ||
| * @param rowCount Estimated number of rows. | ||
| * @param colStats Column-level statistics. | ||
| * @param isBroadcastable If true, output is small enough to be used in a broadcast join. | ||
| */ | ||
| case class Statistics( | ||
| sizeInBytes: BigInt, | ||
| rowCount: Option[BigInt] = None, | ||
| colStats: Map[String, ColumnStat] = Map.empty, | ||
| isBroadcastable: Boolean = false) { | ||
|
|
||
| override def toString: String = "Statistics(" + simpleString + ")" | ||
|
|
||
| /** Readable string representation for the Statistics. */ | ||
| def simpleString: String = { | ||
| Seq(s"sizeInBytes=$sizeInBytes", | ||
| if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "", | ||
| s"isBroadcastable=$isBroadcastable" | ||
| ).filter(_.nonEmpty).mkString("", ", ", "") | ||
| ).filter(_.nonEmpty).mkString(", ") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Statistics for a column. | ||
| */ | ||
| case class ColumnStat(statRow: InternalRow) { | ||
|
|
||
| def forNumeric[T <: AtomicType](dataType: T): NumericColumnStat[T] = { | ||
| NumericColumnStat(statRow, dataType) | ||
| } | ||
| def forString: StringColumnStat = StringColumnStat(statRow) | ||
| def forBinary: BinaryColumnStat = BinaryColumnStat(statRow) | ||
| def forBoolean: BooleanColumnStat = BooleanColumnStat(statRow) | ||
|
|
||
| override def toString: String = { | ||
| // use Base64 for encoding | ||
| Base64.encodeBase64String(statRow.asInstanceOf[UnsafeRow].getBytes) | ||
| } | ||
| } | ||
|
|
||
| object ColumnStat { | ||
| def apply(numFields: Int, str: String): ColumnStat = { | ||
| // use Base64 for decoding | ||
| val bytes = Base64.decodeBase64(str) | ||
| val unsafeRow = new UnsafeRow(numFields) | ||
|
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. we can just write
Contributor
Author
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. it only returns an empty UnsafeRow: row.pointTo(new byte[numBytes], numBytes);
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. ah i see, sorry I read the code wrong |
||
| unsafeRow.pointTo(bytes, bytes.length) | ||
| ColumnStat(unsafeRow) | ||
| } | ||
| } | ||
|
|
||
| case class NumericColumnStat[T <: AtomicType](statRow: InternalRow, dataType: T) { | ||
| // The indices here must be consistent with `ColumnStatStruct.numericColumnStat`. | ||
| val numNulls: Long = statRow.getLong(0) | ||
| val max: T#InternalType = statRow.get(1, dataType).asInstanceOf[T#InternalType] | ||
| val min: T#InternalType = statRow.get(2, dataType).asInstanceOf[T#InternalType] | ||
| val ndv: Long = statRow.getLong(3) | ||
| } | ||
|
|
||
| case class StringColumnStat(statRow: InternalRow) { | ||
| // The indices here must be consistent with `ColumnStatStruct.stringColumnStat`. | ||
| val numNulls: Long = statRow.getLong(0) | ||
| val avgColLen: Double = statRow.getDouble(1) | ||
| val maxColLen: Long = statRow.getLong(2) | ||
| val ndv: Long = statRow.getLong(3) | ||
| } | ||
|
|
||
| case class BinaryColumnStat(statRow: InternalRow) { | ||
| // The indices here must be consistent with `ColumnStatStruct.binaryColumnStat`. | ||
| val numNulls: Long = statRow.getLong(0) | ||
| val avgColLen: Double = statRow.getDouble(1) | ||
| val maxColLen: Long = statRow.getLong(2) | ||
| } | ||
|
|
||
| case class BooleanColumnStat(statRow: InternalRow) { | ||
| // The indices here must be consistent with `ColumnStatStruct.booleanColumnStat`. | ||
| val numNulls: Long = statRow.getLong(0) | ||
| val numTrues: Long = statRow.getLong(1) | ||
| val numFalses: Long = statRow.getLong(2) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -87,19 +87,27 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { | |
| } | ||
|
|
||
| /** | ||
| * Create an [[AnalyzeTableCommand]] command. This currently only implements the NOSCAN | ||
| * option (other options are passed on to Hive) e.g.: | ||
| * Create an [[AnalyzeTableCommand]] command or an [[AnalyzeColumnCommand]] command. | ||
| * Example SQL for analyzing table : | ||
| * {{{ | ||
| * ANALYZE TABLE table COMPUTE STATISTICS NOSCAN; | ||
| * ANALYZE TABLE table COMPUTE STATISTICS [NOSCAN]; | ||
| * }}} | ||
| * Example SQL for analyzing columns : | ||
| * {{{ | ||
| * ANALYZE TABLE table COMPUTE STATISTICS FOR COLUMNS column1, column2; | ||
| * }}} | ||
| */ | ||
| override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { | ||
| if (ctx.partitionSpec == null && | ||
| ctx.identifier != null && | ||
| ctx.identifier.getText.toLowerCase == "noscan") { | ||
| AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) | ||
| AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier)) | ||
| } else if (ctx.identifierSeq() == null) { | ||
|
||
| AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier), noscan = false) | ||
|
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. when will we hit this branch?
Contributor
Author
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. analyze table without |
||
| } else { | ||
| AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString, noscan = false) | ||
| AnalyzeColumnCommand( | ||
| visitTableIdentifier(ctx.tableIdentifier), | ||
| visitIdentifierSeq(ctx.identifierSeq())) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,175 @@ | ||
| /* | ||
| * 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.execution.command | ||
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, ColumnStat, LogicalPlan, Statistics} | ||
| import org.apache.spark.sql.execution.datasources.LogicalRelation | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
|
|
||
| /** | ||
| * Analyzes the given columns of the given table to generate statistics, which will be used in | ||
| * query optimizations. | ||
| */ | ||
| case class AnalyzeColumnCommand( | ||
| tableIdent: TableIdentifier, | ||
| columnNames: Seq[String]) extends RunnableCommand { | ||
|
|
||
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
| val sessionState = sparkSession.sessionState | ||
| val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) | ||
| val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) | ||
| val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdentWithDB)) | ||
|
|
||
| relation match { | ||
| case catalogRel: CatalogRelation => | ||
| updateStats(catalogRel.catalogTable, | ||
| AnalyzeTableCommand.calculateTotalSize(sessionState, catalogRel.catalogTable)) | ||
|
|
||
| case logicalRel: LogicalRelation if logicalRel.catalogTable.isDefined => | ||
| updateStats(logicalRel.catalogTable.get, logicalRel.relation.sizeInBytes) | ||
|
|
||
| case otherRelation => | ||
| throw new AnalysisException("ANALYZE TABLE is not supported for " + | ||
| s"${otherRelation.nodeName}.") | ||
| } | ||
|
|
||
| def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit = { | ||
| val (rowCount, columnStats) = computeColStats(sparkSession, relation) | ||
| val statistics = Statistics( | ||
| sizeInBytes = newTotalSize, | ||
| rowCount = Some(rowCount), | ||
| colStats = columnStats ++ catalogTable.stats.map(_.colStats).getOrElse(Map())) | ||
| sessionState.catalog.alterTable(catalogTable.copy(stats = Some(statistics))) | ||
| // Refresh the cached data source table in the catalog. | ||
| sessionState.catalog.refreshTable(tableIdentWithDB) | ||
|
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. why do we need to refresh table for statistics updating? AFAIK, refresh table is used when the table data files changed, and we need to list files again and invalidate table cache.
Contributor
Author
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. We need to refresh the
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. ah i see |
||
| } | ||
|
|
||
| Seq.empty[Row] | ||
| } | ||
|
|
||
| def computeColStats( | ||
| sparkSession: SparkSession, | ||
| relation: LogicalPlan): (Long, Map[String, ColumnStat]) = { | ||
|
|
||
| // check correctness of column names | ||
| val attributesToAnalyze = mutable.MutableList[Attribute]() | ||
| val duplicatedColumns = mutable.MutableList[String]() | ||
| val resolver = sparkSession.sessionState.conf.resolver | ||
| columnNames.foreach { col => | ||
| val exprOption = relation.output.find(attr => resolver(attr.name, col)) | ||
| val expr = exprOption.getOrElse(throw new AnalysisException(s"Invalid column name: $col.")) | ||
| // do deduplication | ||
|
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. is it a common behaviour? i.e. if users specify duplicated columns, we just deduplicate silently instead of reporting error.
Contributor
Author
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. Yes, hive does so. @hvanhovell and @gatorsmile also agreed on this.
Member
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. How about issuing
Contributor
Author
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. @gatorsmile OK, let me add a log for it. |
||
| if (!attributesToAnalyze.contains(expr)) { | ||
| attributesToAnalyze += expr | ||
| } else { | ||
| duplicatedColumns += col | ||
| } | ||
| } | ||
| if (duplicatedColumns.nonEmpty) { | ||
| logWarning(s"Duplicated columns ${duplicatedColumns.mkString("(", ", ", ")")} detected " + | ||
| s"when analyzing columns ${columnNames.mkString("(", ", ", ")")}, ignoring them.") | ||
|
Member
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. How about this? logWarning("A duplicate column name was detected in `ANALYZE TABLE` statement. " +
s"Input columns: ${columnNames.mkString("(", ", ", ")")}. " +
s"Duplicate columns: ${duplicatedColumns.mkString("(", ", ", ")")}.") |
||
| } | ||
|
|
||
| // Collect statistics per column. | ||
| // The first element in the result will be the overall row count, the following elements | ||
| // will be structs containing all column stats. | ||
| // The layout of each struct follows the layout of the ColumnStats. | ||
| val ndvMaxErr = sparkSession.sessionState.conf.ndvMaxError | ||
| val expressions = Count(Literal(1)).toAggregateExpression() +: | ||
| attributesToAnalyze.map(ColumnStatStruct(_, ndvMaxErr)) | ||
| val namedExpressions = expressions.map(e => Alias(e, e.toString)()) | ||
| val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, namedExpressions, relation)) | ||
|
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. what will happen if the table is empty?
Contributor
Author
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. rowCount, ndv will be 0, max, min will be null, see the last test case in StatisticsColumnSuite. |
||
| .queryExecution.toRdd.collect().head | ||
|
|
||
| // unwrap the result | ||
| val rowCount = statsRow.getLong(0) | ||
| val columnStats = attributesToAnalyze.zipWithIndex.map { case (expr, i) => | ||
| val numFields = ColumnStatStruct.numStatFields(expr.dataType) | ||
| (expr.name, ColumnStat(statsRow.getStruct(i + 1, numFields))) | ||
| }.toMap | ||
| (rowCount, columnStats) | ||
| } | ||
| } | ||
|
|
||
| object ColumnStatStruct { | ||
| val zero = Literal(0, LongType) | ||
| val one = Literal(1, LongType) | ||
|
|
||
| def numNulls(e: Expression): Expression = if (e.nullable) Sum(If(IsNull(e), one, zero)) else zero | ||
| def max(e: Expression): Expression = Max(e) | ||
| def min(e: Expression): Expression = Min(e) | ||
| def ndv(e: Expression, relativeSD: Double): Expression = { | ||
| // the approximate ndv should never be larger than the number of rows | ||
| Least(Seq(HyperLogLogPlusPlus(e, relativeSD), Count(one))) | ||
| } | ||
| def avgLength(e: Expression): Expression = Average(Length(e)) | ||
| def maxLength(e: Expression): Expression = Max(Length(e)) | ||
| def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) | ||
| def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) | ||
|
|
||
| def getStruct(exprs: Seq[Expression]): CreateStruct = { | ||
| CreateStruct(exprs.map { expr: Expression => | ||
| expr.transformUp { | ||
| case af: AggregateFunction => af.toAggregateExpression() | ||
| } | ||
| }) | ||
| } | ||
|
|
||
| def numericColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { | ||
| Seq(numNulls(e), max(e), min(e), ndv(e, relativeSD)) | ||
| } | ||
|
|
||
| def stringColumnStat(e: Expression, relativeSD: Double): Seq[Expression] = { | ||
| Seq(numNulls(e), avgLength(e), maxLength(e), ndv(e, relativeSD)) | ||
| } | ||
|
|
||
| def binaryColumnStat(e: Expression): Seq[Expression] = { | ||
| Seq(numNulls(e), avgLength(e), maxLength(e)) | ||
| } | ||
|
|
||
| def booleanColumnStat(e: Expression): Seq[Expression] = { | ||
|
Member
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. All the above 13 functions should be private, right? |
||
| Seq(numNulls(e), numTrues(e), numFalses(e)) | ||
| } | ||
|
|
||
| def numStatFields(dataType: DataType): Int = { | ||
| dataType match { | ||
| case BinaryType | BooleanType => 3 | ||
| case _ => 4 | ||
| } | ||
| } | ||
|
|
||
| def apply(e: Attribute, relativeSD: Double): CreateStruct = e.dataType match { | ||
|
Member
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. Nit: |
||
| // Use aggregate functions to compute statistics we need. | ||
| case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(e, relativeSD)) | ||
| case StringType => getStruct(stringColumnStat(e, relativeSD)) | ||
| case BinaryType => getStruct(binaryColumnStat(e)) | ||
| case BooleanType => getStruct(booleanColumnStat(e)) | ||
| case otherType => | ||
| throw new AnalysisException("Analyzing columns is not supported for column " + | ||
| s"${e.name} of data type: ${e.dataType}.") | ||
| } | ||
| } | ||
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.
We have
NumericType, can we use that?Uh oh!
There was an error while loading. Please reload this page.
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.
But we need to include DateType and TimestampType also, so I use AtomicType.