Skip to content
Closed
Show file tree
Hide file tree
Changes from 16 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 @@ -86,7 +86,7 @@ statement
| CREATE TABLE (IF NOT EXISTS)? target=tableIdentifier
LIKE source=tableIdentifier #createTableLike
| ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS
(identifier | FOR COLUMNS identifierSeq?)? #analyze
(identifier | FOR COLUMNS identifierSeq)? #analyze
| ALTER (TABLE | VIEW) from=tableIdentifier
RENAME TO to=tableIdentifier #renameTable
| ALTER (TABLE | VIEW) tableIdentifier
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;

import org.apache.spark.sql.catalyst.plans.logical.Except;
Copy link
Contributor

Choose a reason for hiding this comment

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

unnecessary change

import org.apache.spark.sql.types.*;
import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.array.ByteArrayMethods;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -32,19 +38,77 @@ 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(dataType: DataType, statRow: InternalRow) {

def forNumeric[T <: AtomicType](dataType: T): NumericColumnStat[T] = {
Copy link
Contributor

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?

Copy link
Contributor Author

@wzhfy wzhfy Oct 1, 2016

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.

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(dataType: DataType, str: String): ColumnStat = {
// use Base64 for decoding
ColumnStat(dataType, InternalRow(Base64.decodeBase64(str)))
}
}

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
Expand Up @@ -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) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not make the identifierSeq non-optional in the grammar? Saves a lot of typing:)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, good idea

AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier), noscan = false)
Copy link
Contributor

Choose a reason for hiding this comment

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

when will we hit this branch?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

analyze table without noncan

} else {
AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString, noscan = false)
AnalyzeColumnCommand(
visitTableIdentifier(ctx.tableIdentifier),
visitIdentifierSeq(ctx.identifierSeq()))
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,180 @@
/*
* 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.{InternalRow, 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 in the current database to generate statistics,
Copy link
Contributor

Choose a reason for hiding this comment

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

... in the current database ... users can specify the database in table name right? I think we can just say given table

* 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)
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

@wzhfy wzhfy Sep 27, 2016

Choose a reason for hiding this comment

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

We need to refresh the stats property of catalogTable in cached data source tables, alterTable won't refresh that.

Copy link
Contributor

Choose a reason for hiding this comment

The 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 caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis
Copy link
Contributor

Choose a reason for hiding this comment

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

val resolver = sparkSession.sessionState.conf.resolver
...
relation.output.find(attr => resolver(attr.name, col))

Copy link
Contributor Author

Choose a reason for hiding this comment

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

nice!

columnNames.foreach { col =>
val exprOption = relation.output.find { attr =>
if (caseSensitive) attr.name == col else attr.name.equalsIgnoreCase(col)
}
val expr = exprOption.getOrElse(throw new AnalysisException(s"Invalid column name: $col."))
// do deduplication
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, hive does so. @hvanhovell and @gatorsmile also agreed on this.

Copy link
Member

Choose a reason for hiding this comment

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

How about issuing logWarning? I still think a silent drop is a concern

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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)) {
Copy link
Member

Choose a reason for hiding this comment

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

Deduplication lacks case sensitivity handling.

Copy link
Contributor Author

@wzhfy wzhfy Sep 22, 2016

Choose a reason for hiding this comment

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

@gatorsmile There's no need to handle case sensitivity here, because in attributesToAnalyze we have the name in schema. For example, if the column name in schema is "abc", for columns "abc" and "ABC", both their names in expr are "abc".

attributesToAnalyze += expr
}
}

// 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))
Copy link
Contributor

Choose a reason for hiding this comment

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

what will happen if the table is empty?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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) =>
(expr.name, ColumnStatStruct.unwrapStruct(statsRow, i + 1, expr, ndvMaxErr, rowCount))
Copy link
Contributor

Choose a reason for hiding this comment

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

we only need the struct numFields here right? Can we follow what we did in ColumnStat?

val numFields = dataType match {
       case BinaryType | BooleanType => 3
       case _ => 4
     }

then we don't need unwrapStruct

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we can calculate numField here, then ColumnStat.apply can be

def apply(numFields: Int, str: String)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good advice! Thanks.

}.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 = {
val approxNdv = HyperLogLogPlusPlus(e, relativeSD)
// the approximate ndv should not be larger than the number of rows
If(LessThanOrEqual(approxNdv, Count(one)), approxNdv, Count(one))
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: Use Least expression. That should be quicker, and should be (of course) less code :)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That's much better! Thanks!

}
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()
case e: Expression => e
Copy link
Contributor

Choose a reason for hiding this comment

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

this case is unnecessary

}
})
}

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] = {
Copy link
Member

Choose a reason for hiding this comment

The 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 apply(e: Attribute, relativeSD: Double): CreateStruct = e.dataType match {
Copy link
Member

Choose a reason for hiding this comment

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

Nit: e -> attr

// 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}.")
}

def unwrapStruct(
row: InternalRow,
offset: Int,
e: Expression,
relativeSD: Double,
rowCount: Long): ColumnStat = {
val numFields = e.dataType match {
case _: NumericType | TimestampType | DateType => numericColumnStat(e, relativeSD).length
case StringType => stringColumnStat(e, relativeSD).length
case BinaryType => binaryColumnStat(e).length
case BooleanType => booleanColumnStat(e).length
}
val struct = row.getStruct(offset, numFields)
ColumnStat(e.dataType, struct)
}
}
Loading