Github user wzhfy commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15090#discussion_r79272492
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
    @@ -0,0 +1,159 @@
    +/*
    + * 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, 
BasicColStats, 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,
    + * 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 relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
    +
    +    // check correctness of column names
    +    val validColumns = mutable.MutableList[NamedExpression]()
    +    val resolver = sessionState.conf.resolver
    +    columnNames.foreach { col =>
    +      val exprOption = relation.resolve(col.split("\\."), resolver)
    +      if (exprOption.isEmpty) {
    +        throw new AnalysisException(s"Invalid column name: $col")
    +      }
    +      if (validColumns.map(_.exprId).contains(exprOption.get.exprId)) {
    +        throw new AnalysisException(s"Duplicate column name: $col")
    +      }
    +      validColumns += exprOption.get
    +    }
    +
    +    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("ANALYZE TABLE is not supported for " +
    +          s"${otherRelation.nodeName}.")
    +    }
    +
    +    def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit 
= {
    +      // 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 BasicColStats.
    +      val ndvMaxErr = sessionState.conf.ndvMaxError
    +      val expressions = Count(Literal(1)).toAggregateExpression() +:
    +        validColumns.map(ColumnStatsStruct(_, ndvMaxErr))
    +      val namedExpressions = expressions.map(e => Alias(e, e.toString)())
    +      val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, 
namedExpressions, relation))
    +        .queryExecution.toRdd.collect().head
    +
    +      // unwrap the result
    +      val rowCount = statsRow.getLong(0)
    +      val colStats = validColumns.zipWithIndex.map { case (expr, i) =>
    +        val colInfo = statsRow.getStruct(i + 1, 
ColumnStatsStruct.statsNumber)
    +        val colStats = ColumnStatsStruct.unwrapRow(expr, colInfo)
    +        (expr.name, colStats)
    +      }.toMap
    +
    +      val statistics =
    +        Statistics(sizeInBytes = newTotalSize, rowCount = Some(rowCount), 
basicColStats = colStats)
    --- End diff --
    
    I think this is an open question. IMO, as a computing framework like Spark, 
it is difficult to decide when to recollect the stats, because that depends on 
how often the data is changed and how it is changed, i.e. how customers use 
Spark. For different applications, this can be quite different. So I think 
maybe we should leave the decision to users or someone like DBAs who know their 
applications well, let them decide when to run the "ANALYZE" commands and 
refresh the stats.


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

Reply via email to