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

    https://github.com/apache/spark/pull/13494#discussion_r70247856
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala
 ---
    @@ -0,0 +1,143 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
SessionCatalog}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.aggregate._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +
    +/**
    + * This rule optimizes the execution of queries that can be answered by 
looking only at
    + * partition-level metadata. This applies when all the columns scanned are 
partition columns, and
    + * the query has an aggregate operator that satisfies the following 
conditions:
    + * 1. aggregate expression is partition columns.
    + *  e.g. SELECT col FROM tbl GROUP BY col.
    + * 2. aggregate function on partition columns with DISTINCT.
    + *  e.g. SELECT col1, count(DISTINCT col2) FROM tbl GROUP BY col1.
    + * 3. aggregate function on partition columns which have same result w or 
w/o DISTINCT keyword.
    + *  e.g. SELECT col1, Max(col2) FROM tbl GROUP BY col1.
    + */
    +case class OptimizeMetadataOnlyQuery(
    +    catalog: SessionCatalog,
    +    conf: SQLConf) extends Rule[LogicalPlan] {
    +
    +  def apply(plan: LogicalPlan): LogicalPlan = {
    +    if (!conf.optimizerMetadataOnly) {
    +      return plan
    +    }
    +
    +    plan.transform {
    +      case a @ Aggregate(_, aggExprs, child @ 
PartitionedRelation(partAttrs, relation)) =>
    +        // We only apply this optimization when only partitioned 
attributes are scanned.
    +        if (a.references.subsetOf(partAttrs)) {
    +          val aggFunctions = aggExprs.flatMap(_.collect {
    +            case agg: AggregateExpression => agg
    +          })
    +          val isAllDistinctAgg = aggFunctions.forall { agg =>
    +            agg.isDistinct || (agg.aggregateFunction match {
    +              // `Max` and `Min` are always distinct aggregate functions 
no matter they have
    +              // DISTINCT keyword or not, as the result will be same.
    +              case _: Max => true
    --- End diff --
    
    Yes, we need to handle them.


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