Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13494#discussion_r68536010
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala
 ---
    @@ -0,0 +1,190 @@
    +/*
    + * 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.{AnalysisException, SparkSession}
    +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}
    +
    +/**
    + * When scanning only partition columns, get results based on metadata 
without scanning files.
    + * It is used for distinct, distinct aggregations or distinct-like 
aggregations(example: Max/Min).
    + * Example: select Max(partition) from table.
    + */
    +case class MetadataOnlyOptimizer(
    +    sparkSession: SparkSession,
    +    catalog: SessionCatalog) extends Rule[LogicalPlan] {
    +
    +  private def canSupportMetadataOnly(a: Aggregate): Boolean = {
    --- End diff --
    
    here is my thoughts about the optimizable cases:
    
    First of all, only parition colums are required(which means we need to 
traverse down the plan tree and find table relation here)
    
    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 
count(DISTINCT a) FROM tbl GROUP BY b`
    3. aggregate function on partition columns which have same result with or 
without DISTINCT keyword, e.g. `SELECT sum(a) FROM tbl GROUP BY b`


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