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

    https://github.com/apache/spark/pull/14690#discussion_r77426633
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala
 ---
    @@ -0,0 +1,102 @@
    +/*
    + * 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.datasources
    +
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types.{StructField, StructType}
    +
    +
    +/**
    + * A [[BasicFileCatalog]] for a metastore catalog table.
    + *
    + * @param sparkSession a [[SparkSession]]
    + * @param db the table's database name
    + * @param table the table's (unqualified) name
    + * @param partitionSchema the schema of a partitioned table's partition 
columns
    + * @param sizeInBytes the table's data size in bytes
    + */
    +class TableFileCatalog(
    +    sparkSession: SparkSession,
    +    db: String,
    +    table: String,
    +    partitionSchema: Option[StructType],
    +    override val sizeInBytes: Long)
    +  extends SessionFileCatalog(sparkSession) {
    +
    +  override protected val hadoopConf = 
sparkSession.sessionState.newHadoopConf
    +
    +  private val externalCatalog = sparkSession.sharedState.externalCatalog
    +
    +  private val catalogTable = externalCatalog.getTable(db, table)
    +
    +  private val baseLocation = catalogTable.storage.locationUri
    +
    +  override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq
    +
    +  override def listFiles(filters: Seq[Expression]): Seq[Partition] = 
partitionSchema match {
    +    case Some(partitionSchema) =>
    +      externalCatalog.listPartitionsByFilter(db, table, filters).flatMap {
    +        case CatalogTablePartition(spec, storage, _) =>
    +          storage.locationUri.map(new Path(_)).map { path =>
    +            val files = listDataLeafFiles(path :: Nil).toSeq
    +            val values =
    +              InternalRow.fromSeq(partitionSchema.map { case 
StructField(name, dataType, _, _) =>
    +                Cast(Literal(spec(name)), dataType).eval()
    +              })
    +            Partition(values, files)
    +          }
    +      }
    +    case None =>
    +      Partition(InternalRow.empty, listDataLeafFiles(rootPaths).toSeq) :: 
Nil
    +  }
    +
    +  override def refresh(): Unit = {}
    +
    +
    +  /**
    +   * Returns a [[ListingFileCatalog]] for this table restricted to the 
subset of partitions
    +   * specified by the given partition-pruning filters.
    +   *
    +   * @param filters partition-pruning filters
    +   */
    +  def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = {
    --- End diff --
    
    I see your point about avoiding the overhead of pruning in planning. The 
flip side of that is surfacing information valuable to both the planner and the 
user in the query plan itself. This is why our (VideoAmp's) clone of Spark does 
partition pruning in the optimization phase (as it was in the early days of 
Spark SQL, I think Spark pre-1.3).
    
    We've found that seeing the actual partitions that will be read in the 
query plan (in "explain extended", for example) has been invaluable in 
validating that a query is going to read the files we think it's going to read 
instead of the entire freakin' table.
    
    To be more specific, this is particularly helpful in validating correct 
pruning behavior in left and right outer joins. Sure, an expert will understand 
which sides' partition pruning filters need to go in the "join" clause and 
which in the "where" clause. This is not the case with less experienced users 
and even experienced users can forget which goes where and when.
    
    In practice, performing partition pruning in query planning has not 
presented a problem for us. The operation is either performed in planning or in 
execution. Performing it in planning makes debugging easier. I can see it also 
being helpful in applying the automatic broadcast optimization. (Incidentally, 
this patch does not implement this fix correctly. Oops. We would need to put 
this pruning in the optimizer, not the physical planner.)
    
    Yes, have a broadcast hint. The problem we've had with that is that we 
haven't found a way to apply that in SQL queries. (Is there a syntax in the 
parser for that?)


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