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

    https://github.com/apache/spark/pull/15090#discussion_r79329382
  
    --- 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)
    --- End diff --
    
    I think I'll check the column by hand here based on the caseSensitive 
configuration, so that I can use the name in relation's output. Using 
`resolve()` has a problem, for example:
    `create table (key int)`
    `analyze ... for columns KEY`
    then if using `resolve()`,the name of expression is `KEY`(case 
insensitive), so will be the name in `colStats` in Statistics class.
    As a result, when we load the stats from metastore, the name in schema is 
`key`, while the name in table parameter is `KEY`, they don't match, so we lost 
the column 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