Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/15090#discussion_r78740027 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala --- @@ -0,0 +1,209 @@ +/* + * 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 +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} +import org.apache.spark.sql.catalyst.plans.logical.{BasicColStats, Statistics} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + + +/** + * Analyzes the given columns of the given table in the current database to generate statistics, + * which will be used in query optimizations. + */ +case class AnalyzeColumnCommand( + tableName: String, + columnNames: Seq[String]) extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val sessionState = sparkSession.sessionState + val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) + val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) + + // check correctness for column names + val attributeNames = relation.output.map(_.name.toLowerCase) + val invalidColumns = columnNames.filterNot { col => attributeNames.contains(col.toLowerCase)} + if (invalidColumns.nonEmpty) { + throw new AnalysisException(s"Invalid columns for table $tableName: $invalidColumns.") + } + + relation match { + case catalogRel: CatalogRelation => + updateStats(catalogRel.catalogTable, + AnalyzeTableCommand.calculateTotalSize(sparkSession, catalogRel.catalogTable)) + + case logicalRel: LogicalRelation if logicalRel.catalogTable.isDefined => + updateStats(logicalRel.catalogTable.get, logicalRel.relation.sizeInBytes) + + case otherRelation => + throw new AnalysisException(s"ANALYZE TABLE is not supported for " + + s"${otherRelation.nodeName}.") + } + + def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit = { + val lowerCaseNames = columnNames.map(_.toLowerCase) + val attributes = + relation.output.filter(attr => lowerCaseNames.contains(attr.name.toLowerCase)) + + // collect column statistics + val aggColumns = mutable.ArrayBuffer[Column](count(Column("*"))) + attributes.foreach(entry => aggColumns ++= statsAgg(entry.name, entry.dataType)) + val statsRow: InternalRow = Dataset.ofRows(sparkSession, relation).select(aggColumns: _*) + .queryExecution.toRdd.collect().head + + // We also update table-level stats to prevent inconsistency in case of table modification --- End diff -- I think we can a lot more concise here. It is much easier to construct a nested row per expressions. The schema of each row matches the schema of `BasicColStats`, this makes translation a lot easier (you could also try to use an encoder for this). This would be an example of this: ```scala // Collect statistics per column. // The first element in the result will be the overal row count, the following elements will be structs containing all columnstats. // The layout of each structs follows the layout of the BasicColStats val zero = Literal(0, LongType) val one = Literal(1, LongType) val nullLong = Literal(null, LongType) val nullDouble = Literal(null, DoubleType) val nullBoolean = Literal(null, BooleanType) def countNull(e: Expression): Expression = val expressions = attributes.map { a => val statistics = attr.dataType match { case NumericType | TimestampType | DateType => Seq(Min(a), Max(a), HyperLogLogPlusPlus(a), nullDouble, nullLong, nullLong, nullLong) case StringType | BinaryType => Seq(Min(a), Max(a), HyperLogLogPlusPlus(a), Avg(Length(a)), Max(Length(a)), nullLong, nullLong) case BooleanType => Seq(nullBoolean, nullBoolean, nullLong, nullDouble, nullLong, Sum(If(a, one, zero)), Sum(If(a, zero, one))) case _ => sys.error("not supported?") } CreateStruct(Sum(If(IsNull(a), one, zero)) +: expressions) } val statsRow = Dataset(session, Aggregate(Nil, expressions, relation)).queryExecution.toRdd.collect().head val rowCount = statsRow.getLong(0) val colStats = attributes.zipWithIndex.map { case (a, i) => val colInfo = statsRow.getStruct(i + 1, 8) // .. unwrap the row } ```
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org