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

    https://github.com/apache/spark/pull/14864#discussion_r78129784
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala 
---
    @@ -156,24 +155,57 @@ case class FileSourceScanExec(
         false
       }
     
    -  override val outputPartitioning: Partitioning = {
    +  @transient private lazy val selectedPartitions = 
relation.location.listFiles(partitionFilters)
    +
    +  override val (outputPartitioning, outputOrdering): (Partitioning, 
Seq[SortOrder]) = {
         val bucketSpec = if 
(relation.sparkSession.sessionState.conf.bucketingEnabled) {
           relation.bucketSpec
         } else {
           None
         }
    -    bucketSpec.map { spec =>
    -      val numBuckets = spec.numBuckets
    -      val bucketColumns = spec.bucketColumnNames.flatMap { n =>
    -        output.find(_.name == n)
    -      }
    -      if (bucketColumns.size == spec.bucketColumnNames.size) {
    -        HashPartitioning(bucketColumns, numBuckets)
    -      } else {
    -        UnknownPartitioning(0)
    -      }
    -    }.getOrElse {
    -      UnknownPartitioning(0)
    +    bucketSpec match {
    +      case Some(spec) =>
    +        val numBuckets = spec.numBuckets
    +
    +        def toAttribute(colName: String, columnType: String): Attribute =
    +          output.find(_.name == colName).getOrElse {
    +            throw new AnalysisException(s"Could not find $columnType 
column $colName for " +
    --- End diff --
    
    I see what you meant earlier. I have made changes to the PR to follow this:
    
    For bucketed columns:
    `HashPartitioning` would be used only when:
    - ALL the bucketing columns are being read from the table
    
    For sorted columns:
    Sort ordering should be used when ALL these criteria's match:
    1. `HashPartitioning` is being used
    2. A prefix (or all) of the sort columns are being read from the table.
    
    Sort ordering would be over the prefix subset of `sort columns` being read
    from the table. eg.
    Assume (col0, col2, col3) are the columns read from the table
    - If sort columns are (col0, col1), then sort ordering would be considered 
as (col0)
    - If sort columns are (col1, col0), then sort ordering would be empty as 
per rule #2 above


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