You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/02/25 00:16:20 UTC

[GitHub] [spark] sunchao opened a new pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

sunchao opened a new pull request #35657:
URL: https://github.com/apache/spark/pull/35657


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR introduces the initial implementation of Storage-Partitioned Join ([SPIP](https://docs.google.com/document/d/1foTkDSM91VxKgkEcBMsuAvEjNybjja-uHk-r3vtXWFE)).
   
   Changes:
   - `org.apache.spark.sql.connector.read.partitioning.Partitioning` currently is very limited (as mentioned in the SPIP), and cannot be extended to handle join cases. This PR replaces its existing `numPartitions` and `satisfies` with two methods: `distribution` and `ordering`. This allows a V2 data source to report to Spark it's distribution and ordering properties, via `SupportsReportPartitioning` interface.
   - with the above change, `org.apache.spark.sql.connector.read.partitioning.Distribution` and `org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution` now are replaced by classes with the same name in `org.apache.spark.sql.connector.distributions` package. Therefore, this PR marks the former two as deprecated.
   - `DataSourcePartitioning` used to be in `org.apache.spark.sql.execution.datasources.v2`. This moves it into package `org.apache.spark.sql.catalyst.plans.physical` so that it can be extended for more non-V2 use cases, such as Hive bucketing. In addition, it is also changed to accommodate the Storage-Partitioned Join feature.
   - a new expression type: `TransformExpression`, is introduced to bind syntactic partition transforms with their semantic meaning, represented by a V2 function. This expression is un-evaluable and is used later in `EnsureRequirements` to check whether join children are compatible with each other.
   - a new optimizer rule: `V2ScanPartitioning`, is added to recognize `Scan`s implement `SupportsReportPartitioning`. If they do, this rule converts V2 distribution & ordering into their counterparts in catalyst, and annotate `DataSourceV2ScanRelation` with the result. These are later propagated into `DataSourceV2ScanExecBase`.
   - changes are made in `DataSourceV2ScanExecBase` to create `DataSourcePartitioning` for scan if 1) the scan has `ClusteredDistribution`, and 2) if all input splits implement `HasPartitionKey`.
   - A new config: `spark.sql.sources.v2.bucketing.enabled` is introduced to turn on or off the behavior. By default it is false.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   Spark currently support bucketing in DataSource V1, but not in V2. This is the first step to support bucket join, and is general form, storage-partitioned join, for V2 data sources. In addition, the work here can potentially used to support Hive bucketing as well. Please check the SPIP for 
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   With the changes, a user can now:
   - have V2 data sources to report distribution and ordering to Spark on read path
   - Spark will recognize the distribution property and eliminate shuffle in join/aggregate/window, etc, when the source distribution matches the required distribution from these.
   - a new config `spark.sql.sources.v2.bucketing.enabled` is introduced to turn on/off the above behavior
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   - Added a new test suite `DataSourcePartitioningSuite` covers end-to-end tests on the new feature
   - Extended `EnsureRequirementsSuite` to cover `DataSourcePartitioning`
   - Some existing test classes, such as `InMemoryTable` are extended to cover the changes
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1058397619


   Gently ping @cloud-fan @viirya @dongjoon-hyun @c21 @rdblue @aokolnychyi


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829346679



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkKeyGroupedSpec(specs(a)) && checkKeyGroupedSpec(specs(b)) &&

Review comment:
       > ... It looks like we can always eliminate shuffles if all the children are compatible.
   
   When children are compatible, it still doesn't mean each child's partition keys **fully match** the distribution keys. Since in `KeyGroupedPartitioning.satisfies`, we only check if partition keys are subset of distribution keys.
   
   That's why we need an additional check here to make sure that when `spark.sql.requireAllClusterKeysForCoPartition` is true, the child partition keys fully match the distribution keys.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r821744630



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -281,6 +282,56 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class DataSourcePartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case d: ClusteredDistribution =>
+          val attributes = expressions.flatMap(_.collectLeaves())
+          attributes.forall(c => d.clustering.exists(_.semanticEquals(c)))

Review comment:
       This looks very confusing...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826701638



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {
+  override def nullable: Boolean = true
+
+  /**
+   * Whether this [[TransformExpression]] has the same semantics as `other`.
+   * For instance, `bucket(32, c)` is equal to `bucket(32, d)`, but not to `bucket(16, d)` or
+   * `year(c)`.
+   *
+   * This will be used, for instance, by Spark to determine whether storage-partitioned join can
+   * be triggered, by comparing partition transforms from both sides of the join and checking
+   * whether they are compatible.
+   *
+   * @param other the transform expression to compare to
+   * @return true if this and `other` has the same semantics w.r.t to transform, false otherwise.
+   */
+  def equalsTo(other: TransformExpression): Boolean

Review comment:
       It's hard to explain the difference between this new method and `equals`, shall we rename it to `isCompatible`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826693413



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/HashPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link HashPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure
+ * that all rows where {@link HashPartitioning#clustering} evaluate to the same value should be

Review comment:
       ```suggestion
    * that all rows where {@link HashPartitioning#clustering} evaluate to the same hash value should be
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1067928403


   IIUC, the required framework-level change is:
   1. add a new `KeyGroupedPartitioning` (or whatever name) which puts all the records sharing the same key in one partition, and each partitioning only have one key. This new partitioning can satisfy `ClusteredDistribution` and can be compatible with other `KeyGroupedPartitioning` if the key expressions are the same.
   2. add a new `DataSourceTransform` expression which takes some input columns and calculates the result with a v2 function. Two `DataSourceTransform` expressions are semantically equal to each other if the inputs and the v2 functions are the same.
   
   I think the above should be sufficient for supporting data source bucketed join, and more changes are needed to really support storage-partition join. IIUC we don't plan to support storage-partition join in this PR, and it seems to me that all the code about the partition values is dead code for now, right?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829317532



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       It's a function for the bucket transform: `bucket(num_buckets, c)` (it could be `bucket(num_buckets, c1, c2, ..)` in future).
   
   The issue here is `canonicalName` for the bucket `BoundFunction`, for obvious reason, doesn't consider the value of `numBuckets`. However, to check of two bucket transforms are compatible, we need to take that into account. That's why we need the extra `DataSourceBucketTransformExpression`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Hmm I'm thinking whether we can still use `DataSourcePartitioning` for the V2 interface, following similar naming conventions such as `DataSourceRDD`, `DataSourceScanExec`, etc. As you mentioned above, `KeyGroupedPartitioning` is not limited to V2 sources but rather more general and can be applied to other things like Hive bucketing in future.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class KeyGroupedPartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) =>
+          if (requireAllClusterKeys) {
+            // Checks whether this partitioning is partitioned on exactly same clustering keys of
+            // `ClusteredDistribution`.
+            c.areAllClusterKeysMatched(expressions)
+          } else {
+            // We'll need to find leaf attributes from the partition expressions first.
+            val attributes = expressions.flatMap(_.collectLeaves())
+            attributes.forall(x => requiredClustering.exists(_.semanticEquals(x)))

Review comment:
       For simplicity this PR only support transforms with a single argument, so the check here is very similar to `HashPartitioning`. I plan to work on the support of multi-arguments as a separate PR.
   
   > I think the theory is, as long as the transform function is deterministic, if two rows have the same value of [a, b, c], the calculated key values [f1(a, b), f2(b, c)] are also the same.
   
   Yea agree with this. It may be more complicated if you have duplicated keys in distribution or/and partitioning though. We also need to think how to implement `EnsureRequirements.reorderJoinPredicates` for this case.
   

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkKeyGroupedSpec(specs(a)) && checkKeyGroupedSpec(specs(b)) &&

Review comment:
       > ... It looks like we can always eliminate shuffles if all the children are compatible.
   
   When children are compatible, it still doesn't mean each child's partition keys **fully match** the distribution keys. Since in `KeyGroupedPartitioning.satisfies`, we only check if partition keys are subset of distribution keys.
   
   That's why we need an additional check here to make sure that when `spark.sql.requireAllClusterKeysForCoPartition` is true, the child partition keys fully match the distribution keys.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839942784



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -256,6 +285,16 @@ case class EnsureRequirements(
         reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys)
           .orElse(reorderJoinKeysRecursively(
             leftKeys, rightKeys, leftPartitioning, None))
+      case (Some(KeyGroupedPartitioning(clustering, _, _)), _) =>

Review comment:
       Yes, but when `spark.sql.requireAllClusterKeysForCoPartition` is true we still check whether partition keys fully match the cluster keys in `EnsureRequirements.checkKeyGroupedSpec` via a `zip` + `semanticEquals`:
   ```scala
         if (SQLConf.get.getConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION)) {
           attributes.length == clustering.length && attributes.zip(clustering).forall {
             case (l, r) => l.semanticEquals(r)
           }
         } else {
           true // already validated in `KeyGroupedPartitioning.satisfies`
         }
   ```
   
   which is very similar to `HashPartitioning`. 
   
   Given the similarity between reordering and the key position check in `ShuffleSpec`, we might be able to remove the reordering logic all together, but it could worth a separate PR IMO.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839230311



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -574,6 +634,80 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   *
+   * Note that we only allow each partition expression to contain a single partition key.
+   * Therefore the mapping here is very similar to that from `HashShuffleSpec`.
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one child from $e, but found ${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering keys
+    //  2. both partitioning have the same number of partitions
+    //  3. partition expressions from both sides are compatible, which means:
+    //    3.1 both sides have the same number of partition expressions
+    //    3.2 for each pair of partition expressions at the same index, the corresponding
+    //        partition keys must share overlapping positions in their respective clustering keys.
+    //    3.3 each pair of partition expressions at the same index must share compatible
+    //        transform functions.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      val expressions = partitioning.expressions
+      val otherExpressions = otherPartitioning.expressions
+
+      distribution.clustering.length == otherDistribution.clustering.length &&
+          numPartitions == other.numPartitions &&

Review comment:
       nit: 2 spaces indentation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839222052



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +50,85 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)

Review comment:
       ```suggestion
         .getOrElse(Seq.empty)
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839922036



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -68,16 +70,36 @@ case class BatchScanExec(
       val newPartitions = scan.toBatch.planInputPartitions()
 
       originalPartitioning match {
-        case p: DataSourcePartitioning if p.numPartitions != newPartitions.size =>
-          throw new SparkException(
-            "Data source must have preserved the original partitioning during runtime filtering; " +
-            s"reported num partitions: ${p.numPartitions}, " +
-            s"num partitions after runtime filtering: ${newPartitions.size}")
+        case p: KeyGroupedPartitioning =>
+          if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) {
+            throw new SparkException("Data source must have preserved the original partitioning " +
+                "during runtime filtering: not all partitions implement HasPartitionKey after " +
+                "filtering")
+          }
+
+          val newRows = new InternalRowSet(p.expressions.map(_.dataType))
+          newRows ++= newPartitions.map(_.asInstanceOf[HasPartitionKey].partitionKey())
+          val oldRows = p.partitionValuesOpt.get
+
+          if (oldRows.size != newRows.size) {

Review comment:
       I think it just inherits the default implementation from the parent class, which calls the methods we override here such as `contains` underneath.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839888801



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link KeyGroupedPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure that all rows where
+ * {@link KeyGroupedPartitioning#clustering} evaluate to the same value should be in the same
+ * partition.

Review comment:
       Oops you are right. I forgot to update this. Will add.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1068189873


   Thanks @cloud-fan. Yes in terms of framework change what you described is right (plus the changes on deprecating existing code).
   
   > I think the above should be sufficient for supporting data source bucketed join, and more changes are needed to really support storage-partition join. IIUC we don't plan to support storage-partition join in this PR, and it seems to me that all the code about the partition values is dead code for now, right?
   
   This PR does add the initial support for storage-partitioned join. The partition values are used in `DataSourceShuffleSpec.isCompatibleWith` to check if both sides of a join are compatible.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828898202



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {
+  override def nullable: Boolean = true
+
+  /**
+   * Whether this [[TransformExpression]] has the same semantics as `other`.
+   * For instance, `bucket(32, c)` is equal to `bucket(32, d)`, but not to `bucket(16, d)` or
+   * `year(c)`.
+   *
+   * This will be used, for instance, by Spark to determine whether storage-partitioned join can
+   * be triggered, by comparing partition transforms from both sides of the join and checking
+   * whether they are compatible.
+   *
+   * @param other the transform expression to compare to
+   * @return true if this and `other` has the same semantics w.r.t to transform, false otherwise.
+   */
+  def equalsTo(other: TransformExpression): Boolean
+}
+
+/**
+ * A transform expression defined by a V2 data source.
+ */
+case class DataSourceTransformExpression(
+    function: BoundFunction,
+    children: Seq[Expression]) extends TransformExpression {
+
+  override def equalsTo(other: TransformExpression): Boolean = other match {
+    case DataSourceTransformExpression(otherFunction, _) =>
+      function.canonicalName() == otherFunction.canonicalName()

Review comment:
       ah, then `isSameFunction` is probably a better name here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827774224



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/InternalRowSet.scala
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.catalyst.util
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Murmur3HashFunction, RowOrdering}
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+
+/**
+ * A mutable Set with [[InternalRow]] as its element type. It uses Spark's internal murmur hash to
+ * compute hash code from an row, and uses [[RowOrdering]] to perform equality checks.
+ *
+ * @param dataTypes the data types for the row keys this set holds
+ */
+class InternalRowSet(val dataTypes: Seq[DataType]) extends mutable.Set[InternalRow] {

Review comment:
       do we have to extends mutable.Set? I think it's simpler to just have the class `InternalRowSet` with `add` and `contains` APIs. Then we don't need to worry about scala 2.13




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r833502028



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      distribution.clustering.length == otherDistribution.clustering.length &&
+        numPartitions == other.numPartitions &&
+          isClusteringCompatibleWith(otherSpec) &&
+            partitioning.partitionValuesOpt.zip(otherPartitioning.partitionValuesOpt).forall {
+              case (left, right) => left.zip(right).forall { case (l, r) =>
+                ordering.compare(l, r) == 0
+              }
+            }
+    case ShuffleSpecCollection(specs) =>
+      specs.exists(isCompatibleWith)
+    case _ => false
+  }
+
+  /**
+   * Check if the clustering expressions from this spec and `other` are compatible. In
+   * particular, whether each pair of expressions have compatible transform expressions.
+   *
+   * @param other the partitioning from the other side
+   * @return true if the clustering expressions are compatible, false otherwise
+   */
+  private def isClusteringCompatibleWith(other: KeyGroupedShuffleSpec): Boolean = {
+    val expressions = partitioning.expressions
+    val otherPartitioning = other.partitioning
+    val otherExpressions = otherPartitioning.expressions
+
+    expressions.length == otherExpressions.length && {
+      val otherKeyPositions = other.keyPositions
+      keyPositions.zip(otherKeyPositions).forall { case (left, right) =>
+        left.intersect(right).nonEmpty
+      }
+    } && expressions.zip(otherExpressions).forall { case (l, r) =>
+      isExpressionCompatible(l, r)
+    }
+  }
+
+  private def isExpressionCompatible(left: Expression, right: Expression): Boolean =
+    (left, right) match {
+      case (_: LeafExpression, _: LeafExpression) => true
+      case (left: TransformExpression, right: TransformExpression) =>
+        left.isCompatibleWith(right)

Review comment:
       I think the input is checked by validating 1) cluster keys satisfies the distribution keys (via `satisfies`) and 2) checking the key positions from both sides? 
   
   In this PR, the assumption is that each transform expression has only a single input which is a leaf expression, so the compatibility check is similar to that of `HashPartitioning` except we also need to check whether the transform function is compatible.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832359735



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {

Review comment:
       I'm still not convinced that this should be an abstract class. I think this doesn't need any subclasses and we can identify any transform function with a `function: BoundFunction` and `children: Seq[Expression]`, including Spark's own bucket function.
   
   Spark's bucket function can have a fake `BoundFunction` which can't be evaluated, but just a marker to indicate it's spark bucket function with a special `canonicalName`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828227349



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Sure. I can change it to `KeyGroupedPartitioning`. Do you think we should change `DataSourceHashPartitioning` back to `DataSourcePartitioning` or something else as well? since it is not really using hash to partition.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r815017381



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1234,6 +1234,15 @@ object SQLConf {
     .booleanConf
     .createWithDefault(true)
 
+  val V2_BUCKETING_ENABLED = buildConf("spark.sql.sources.v2.bucketing.enabled")
+      .doc(s"Similar to ${BUCKETING_ENABLED.key}, this config is used to enable bucketing for V2 " +
+        "data sources. When turned on, Spark will recognize the specific distribution " +
+        "reported by a V2 data source through SupportsReportPartitioning, and will try to " +
+        "avoid shuffle if necessary.")
+      .version("3.3.0")
+      .booleanConf
+      .createWithDefault(false)

Review comment:
       By default this is false. Previously when V2 data sources report `DataSourcePartitioning`, Spark can potentially eliminate shuffle in aggregation. However, with this config they now have to turn this flag in order to get the same behavior.
   
   My primary goal is to disable storage-partitioned join by default. So perhaps I can introduce another flag to control the join behavior and use this to control the aggregate behavior, and set it to true by default.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829322165



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Hmm I'm thinking whether we can still use `DataSourcePartitioning` for the V2 interface, following similar naming conventions such as `DataSourceRDD`, `DataSourceScanExec`, etc. As you mentioned above, `KeyGroupedPartitioning` is not limited to V2 sources but rather more general and can be applied to other things like Hive bucketing in future.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827771304



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       I like `KeyGroupedPartitioning` more because it's not limited to data source v2. Technically the existing `HashPartitioning` can also be a `KeyGroupedPartitioning` using hash expression as the key expression.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827144086



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,

Review comment:
       Yea this is not too much useful now. It is only used by `DistributionAndOrderingUtils` at the moment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827177268



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {
+  override def nullable: Boolean = true
+
+  /**
+   * Whether this [[TransformExpression]] has the same semantics as `other`.
+   * For instance, `bucket(32, c)` is equal to `bucket(32, d)`, but not to `bucket(16, d)` or
+   * `year(c)`.
+   *
+   * This will be used, for instance, by Spark to determine whether storage-partitioned join can
+   * be triggered, by comparing partition transforms from both sides of the join and checking
+   * whether they are compatible.
+   *
+   * @param other the transform expression to compare to
+   * @return true if this and `other` has the same semantics w.r.t to transform, false otherwise.
+   */
+  def equalsTo(other: TransformExpression): Boolean

Review comment:
       Yea agreed. I can rename it to `isCompatible`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r822095789



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -117,7 +118,9 @@ case class DataSourceV2Relation(
 case class DataSourceV2ScanRelation(
     relation: DataSourceV2Relation,
     scan: Scan,
-    output: Seq[AttributeReference]) extends LeafNode with NamedRelation {
+    output: Seq[AttributeReference],
+    distribution: Distribution = UnspecifiedDistribution,

Review comment:
       Yea I agree with you. What do you think if we pass the clustering expressions instead? something like:
   
   ```scala
   distribution: Option[Seq[Expression]] = None,
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r838905562



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {

Review comment:
       Yea you are right. If necessary, we can also create a special `BoundFunction` sub-class like `SparkBucketFunction` and handle it separately in `TransformExpression.isSameFunction`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839893300



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpression.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Represents a partition transform expression, for instance, `bucket`, `days`, `years`, etc.
+ *
+ * @param function the transform function itself. Spark will use it to decide whether two
+ *                 partition transform expressions are compatible.
+ * @param numBucketsOpt the number of buckets if the transform is `bucket`. Unset otherwise.

Review comment:
       We could, but since we don't check `children` in `isSameFunction` right now, we'll need to add extra logic to decide whether it is bucketing case or non-bucketing case, and check the literal in the children correspondingly. I think this is more complex than the current approach?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840692338



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.connector
+
+import java.util.Collections
+
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder => V1SortOrder, TransformExpression}
+import org.apache.spark.sql.catalyst.plans.{physical => v1}

Review comment:
       it's not v1... it's the internal catalyst representation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839243546



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link KeyGroupedPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure that all rows where
+ * {@link KeyGroupedPartitioning#clustering} evaluate to the same value should be in the same
+ * partition.

Review comment:
       Actually, rows with the same value can also be in different partitions, because each partition can report its value and Spark will group partitions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840684890



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala
##########
@@ -17,23 +17,27 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
-import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils.toCatalyst
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RepartitionByExpression, Sort}
-import org.apache.spark.sql.connector.distributions.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution}
+import org.apache.spark.sql.connector.distributions._
 import org.apache.spark.sql.connector.write.{RequiresDistributionAndOrdering, Write}
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.collection.Utils.sequenceToOption
 
 object DistributionAndOrderingUtils {
 
   def prepareQuery(write: Write, query: LogicalPlan, conf: SQLConf): LogicalPlan = write match {
     case write: RequiresDistributionAndOrdering =>
       val numPartitions = write.requiredNumPartitions()
+
       val distribution = write.requiredDistribution match {
-        case d: OrderedDistribution => d.ordering.map(e => toCatalyst(e, query))
-        case d: ClusteredDistribution => d.clustering.map(e => toCatalyst(e, query))
-        case _: UnspecifiedDistribution => Array.empty[Expression]
+        case d: OrderedDistribution => toCatalystOrdering(d.ordering(), query)
+        case d: ClusteredDistribution =>
+          sequenceToOption(d.clustering.map(e => toCatalyst(e, query)))
+              .getOrElse(Seq.empty[Expression])

Review comment:
       ```suggestion
               .getOrElse(Seq.empty[Expression])
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840688908



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -177,6 +186,26 @@ case class EnsureRequirements(
     children
   }
 
+  private def checkKeyGroupedSpec(shuffleSpec: ShuffleSpec): Boolean = {
+    def check(spec: KeyGroupedShuffleSpec): Boolean = {
+      val attributes = spec.partitioning.expressions.flatMap(_.collectLeaves())
+      val clustering = spec.distribution.clustering
+
+      if (SQLConf.get.getConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION)) {
+        attributes.length == clustering.length && attributes.zip(clustering).forall {
+          case (l, r) => l.semanticEquals(r)
+        }
+      } else {
+        true // already validated in `KeyGroupedPartitioning.satisfies`

Review comment:
       oh wait, doesn't `KeyGroupedPartitioning.satisfies` respect `REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION` already?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840830349



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.connector
+
+import java.util.Collections
+
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder => V1SortOrder, TransformExpression}
+import org.apache.spark.sql.catalyst.plans.{physical => v1}
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, KeyGroupedPartitioning}
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog
+import org.apache.spark.sql.connector.catalog.functions._
+import org.apache.spark.sql.connector.distributions.Distribution
+import org.apache.spark.sql.connector.distributions.Distributions
+import org.apache.spark.sql.connector.expressions._
+import org.apache.spark.sql.connector.expressions.Expressions._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.joins.SortMergeJoinExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf._
+import org.apache.spark.sql.types._
+
+class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase {
+  private var originalV2BucketingEnabled: Boolean = false
+  private var originalAutoBroadcastJoinThreshold: Long = -1
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    originalV2BucketingEnabled = conf.getConf(V2_BUCKETING_ENABLED)
+    conf.setConf(V2_BUCKETING_ENABLED, true)
+    originalAutoBroadcastJoinThreshold = conf.getConf(AUTO_BROADCASTJOIN_THRESHOLD)
+    conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, -1L)
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      super.afterAll()
+    } finally {
+      conf.setConf(V2_BUCKETING_ENABLED, originalV2BucketingEnabled)
+      conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, originalAutoBroadcastJoinThreshold)
+    }
+  }
+
+  before {
+    Seq(UnboundYearsFunction, UnboundDaysFunction, UnboundBucketFunction).foreach { f =>
+      catalog.createFunction(Identifier.of(Array.empty, f.name()), f)
+    }
+  }
+
+  after {
+    catalog.clearTables()
+    catalog.clearFunctions()
+  }
+
+  private val emptyProps: java.util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+  private val table: String = "tbl"
+  private val schema = new StructType()
+      .add("id", IntegerType)
+      .add("data", StringType)
+      .add("ts", TimestampType)
+
+  test("clustered distribution: output partitioning should be KeyGroupedPartitioning") {
+    val partitions: Array[Transform] = Array(Expressions.years("ts"))
+
+    // create a table with 3 partitions, partitioned by `years` transform
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    var df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY ts")
+    val v1Distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(YearsFunction, Seq(attr("ts")))))
+    val partitionValues = Seq(50, 51, 52).map(v => InternalRow.fromSeq(Seq(v)))
+
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+
+    // multiple group keys should work too as long as partition keys are subset of them
+    df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY id, ts")
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+  }
+
+  test("non-clustered distribution: fallback to super.partitioning") {
+    val partitions: Array[Transform] = Array(years("ts"))
+    val ordering: Array[SortOrder] = Array(sort(FieldReference("ts"),
+      SortDirection.ASCENDING, NullOrdering.NULLS_FIRST))
+
+    createTable(table, schema, partitions, Distributions.ordered(ordering), ordering)
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val v1Ordering = Seq(V1SortOrder(attr("ts"), Ascending))
+    val v1Distribution = v1.OrderedDistribution(v1Ordering)
+
+    checkQueryPlan(df, v1Distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: no partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: single partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES (0, 'aaa', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.SinglePartition)
+  }
+
+  test("non-clustered distribution: no V2 catalog") {
+    spark.conf.set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName)
+    val nonFunctionCatalog = spark.sessionState.catalogManager.catalog("testcat2")
+        .asInstanceOf[InMemoryTableCatalog]
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])),
+      catalog = nonFunctionCatalog)
+    sql(s"INSERT INTO testcat2.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat2.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    try {
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    } finally {
+      spark.conf.unset("spark.sql.catalog.testcat2")
+    }
+  }
+
+  test("non-clustered distribution: no V2 function provided") {
+    catalog.clearFunctions()
+
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: V2 bucketing disabled") {
+    withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "false") {
+      val partitions: Array[Transform] = Array(bucket(32, "ts"))
+      createTable(table, schema, partitions,
+        Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+      sql(s"INSERT INTO testcat.ns.$table VALUES " +
+          s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+          s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+          s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+      val df = sql(s"SELECT * FROM testcat.ns.$table")
+      val distribution = v1.ClusteredDistribution(
+        Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    }
+  }
+
+  /**
+   * Check whether the query plan from `df` has the expected `distribution`, `ordering` and
+   * `partitioning`.
+   */
+  private def checkQueryPlan(
+      df: DataFrame,
+      distribution: v1.Distribution,
+      partitioning: v1.Partitioning): Unit = {
+    // check distribution & ordering are correctly populated in logical plan
+    val relation = df.queryExecution.optimizedPlan.collect {
+      case r: DataSourceV2ScanRelation => r
+    }.head
+
+    resolveDistribution(distribution, relation) match {
+      case ClusteredDistribution(clustering, _, _) =>
+        assert(relation.keyGroupedPartitioning.isDefined && relation.keyGroupedPartitioning.get == clustering)
+      case _ =>
+        assert(relation.keyGroupedPartitioning.isEmpty)
+    }
+
+    // check distribution, ordering and output partitioning are correctly populated in physical plan
+    val scan = collect(df.queryExecution.executedPlan) {
+      case s: BatchScanExec => s
+    }.head
+
+    val expectedPartitioning = resolvePartitioning(partitioning, scan)
+    assert(expectedPartitioning == scan.outputPartitioning)
+  }
+
+  private def createTable(
+      table: String,
+      schema: StructType,
+      partitions: Array[Transform],
+      distribution: Distribution = Distributions.unspecified(),
+      ordering: Array[expressions.SortOrder] = Array.empty,
+      catalog: InMemoryTableCatalog = catalog): Unit = {
+    catalog.createTable(Identifier.of(Array("ns"), table),
+      schema, partitions, emptyProps, distribution, ordering, None)
+  }
+
+  private val customers: String = "customers"
+  private val customers_schema = new StructType()
+      .add("customer_name", StringType)
+      .add("customer_age", IntegerType)
+      .add("customer_id", LongType)
+
+  private val orders: String = "orders"
+  private val orders_schema = new StructType()
+      .add("order_amount", DoubleType)
+      .add("customer_id", LongType)
+
+  private def testWithCustomersAndOrders(
+      customers_partitions: Array[Transform],
+      customers_distribution: Distribution,
+      orders_partitions: Array[Transform],
+      orders_distribution: Distribution,
+      expectedNumOfShuffleExecs: Int): Unit = {
+    createTable(customers, customers_schema, customers_partitions, customers_distribution)
+    sql(s"INSERT INTO testcat.ns.$customers VALUES " +
+        s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)")
+
+    createTable(orders, orders_schema, orders_partitions, orders_distribution)
+    sql(s"INSERT INTO testcat.ns.$orders VALUES " +
+        s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)")
+
+    val df = sql("SELECT customer_name, customer_age, order_amount " +
+        s"FROM testcat.ns.$customers c JOIN testcat.ns.$orders o " +
+        "ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount")
+
+    val shuffles = collectShuffles(df.queryExecution.executedPlan)
+    assert(shuffles.length == expectedNumOfShuffleExecs)
+
+    checkAnswer(df,
+      Seq(Row("aaa", 10, 100.0), Row("aaa", 10, 200.0), Row("bbb", 20, 150.0),
+        Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50)))
+  }
+
+  private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeExec] = {
+    collect(plan) {
+      case s: SortMergeJoinExec => s
+    }.flatMap(smj =>
+      collect(smj) {
+        case s: ShuffleExchangeExec => s
+      })
+  }
+
+  test("partitioned join: exact distribution (same number of buckets) from both sides") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(4, "customer_id"))
+
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.clustered(orders_partitions.toArray),
+      0)
+  }
+
+  test("partitioned join: number of buckets mismatch should trigger shuffle") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(2, "customer_id"))
+
+    // should shuffle both sides when number of buckets are not the same
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.clustered(orders_partitions.toArray),
+      2)
+  }
+
+  test("partitioned join: only one side reports partitioning") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(2, "customer_id"))
+
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.unspecified(),
+      2)
+  }
+
+  private val items: String = "items"
+  private val items_schema: StructType = new StructType()
+      .add("id", LongType)
+      .add("name", StringType)
+      .add("price", FloatType)
+      .add("arrive_time", TimestampType)
+
+  private val purchases: String = "purchases"
+  private val purchases_schema: StructType = new StructType()
+      .add("item_id", LongType)
+      .add("price", FloatType)
+      .add("time", TimestampType)
+
+  test("partitioned join: join with two partition keys and matching & sorted partitions") {
+    val items_partitions = Array(bucket(8, "id"), days("arrive_time"))
+    createTable(items, items_schema, items_partitions,
+      Distributions.clustered(items_partitions.toArray))
+    sql(s"INSERT INTO testcat.ns.$items VALUES " +
+        s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " +
+        s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " +
+        s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " +
+        s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " +
+        s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))")
+
+    val purchases_partitions = Array(bucket(8, "item_id"), days("time"))
+    createTable(purchases, purchases_schema, purchases_partitions,
+      Distributions.clustered(purchases_partitions.toArray))
+    sql(s"INSERT INTO testcat.ns.$purchases VALUES " +
+        s"(1, 42.0, cast('2020-01-01' as timestamp)), " +
+        s"(1, 44.0, cast('2020-01-15' as timestamp)), " +
+        s"(1, 45.0, cast('2020-01-15' as timestamp)), " +
+        s"(2, 11.0, cast('2020-01-01' as timestamp)), " +
+        s"(3, 19.5, cast('2020-02-01' as timestamp))")
+
+    val df = sql("SELECT id, name, i.price as purchase_price, p.price as sale_price " +
+        s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " +
+        "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price")
+
+    val shuffles = collectShuffles(df.queryExecution.executedPlan)
+    assert(shuffles.isEmpty, "should not add shuffle for both sides of the join")
+    checkAnswer(df,
+      Seq(Row(1, "aa", 40.0, 42.0), Row(1, "aa", 41.0, 44.0), Row(1, "aa", 41.0, 45.0),
+        Row(2, "bb", 10.0, 11.0), Row(2, "bb", 10.5, 11.0), Row(3, "cc", 15.5, 19.5))
+    )
+  }
+
+  test("partitioned join: join with two partition keys and unsorted partitions") {

Review comment:
       yes this is correct.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840821838



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -177,6 +186,26 @@ case class EnsureRequirements(
     children
   }
 
+  private def checkKeyGroupedSpec(shuffleSpec: ShuffleSpec): Boolean = {
+    def check(spec: KeyGroupedShuffleSpec): Boolean = {
+      val attributes = spec.partitioning.expressions.flatMap(_.collectLeaves())
+      val clustering = spec.distribution.clustering
+
+      if (SQLConf.get.getConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION)) {
+        attributes.length == clustering.length && attributes.zip(clustering).forall {
+          case (l, r) => l.semanticEquals(r)
+        }
+      } else {
+        true // already validated in `KeyGroupedPartitioning.satisfies`

Review comment:
       That is for `spark.sql.requireAllClusterKeysForDistribution` (e.g., aggregate, window), while this is for `spark.sql.requireAllClusterKeysForCoPartition`. I know it's confusing ...




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828231475



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       hmm, aren't we discussing renaming `DataSourceHashPartitioning` to `KeyGroupedPartitioning` here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827171559



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioning.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.v2
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.FunctionCatalog
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning
+import org.apache.spark.sql.connector.read.partitioning.{HashPartitioning, UnknownPartitioning}
+import org.apache.spark.util.collection.Utils.sequenceToOption
+
+/**
+ * Extract [[DataSourceV2ScanRelation]] from the input logical plan, convert any V2 partitioning
+ * reported by data sources to their catalyst counterparts. Then, annotate the plan with the result.
+ */
+object V2ScanPartitioning extends Rule[LogicalPlan] with SQLConfHelper {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case d @ DataSourceV2ScanRelation(relation, scan: SupportsReportPartitioning, _, _) =>
+      val funCatalogOpt = relation.catalog.flatMap {
+        case c: FunctionCatalog => Some(c)
+        case _ => None
+      }
+
+      val catalystClustering = scan.outputPartitioning() match {
+        case hp: HashPartitioning => sequenceToOption(hp.clustering().map(
+          V2ExpressionUtils.toCatalyst(_, relation, funCatalogOpt)))
+        case _: UnknownPartitioning => None
+        case p => throw new IllegalArgumentException("Unsupported data source V2 partitioning " +
+            "type: " + p.getClass.getSimpleName)
+      }
+
+      d.copy(clustering = catalystClustering)

Review comment:
       Since we need a way to pass clustering expressions down to `BatchScanExec`. You can see how it is used in `DataSourceV2Strategy`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828240443



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       Yea, although this PR we only support transform on a single expression to start with. You can see there is a check at line 114. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828271471



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       `DataSourceShuffleSpec` (without enabling it to re-shuffle other children) can't fit nicely into this framework at the moment, I think.
   
    Since its `canCreatePartitioning` is false, we'll treat it the same as `RangeShuffleSpec`. In case both sides are of `DataSourceShuffleSpec`, the `bestSpecOpt` will be `None` in `EnsureRequirements` and Spark will create default partitioning from the required distribution instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826691815



##########
File path: core/src/main/scala/org/apache/spark/util/collection/Utils.scala
##########
@@ -36,4 +36,20 @@ private[spark] object Utils {
     }
     ordering.leastOf(input.asJava, num).iterator.asScala
   }
+
+  /**
+   * Only returns `Some` iff ALL elements in `input` are defined. In this case, it is
+   * equivalent to `Some(input.flatten)`.
+   *
+   * Otherwise, returns `None`.
+   */
+  def sequenceToOption[T](input: Seq[Option[T]]): Option[Seq[T]] = {
+    input.foldLeft(Option(Seq.empty[T])) {

Review comment:
       I think a if-else is easier to read here
   ```
   if (input.forall(_.isDefined)) Some(input.map(_.get)) else None
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829332903



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class KeyGroupedPartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) =>
+          if (requireAllClusterKeys) {
+            // Checks whether this partitioning is partitioned on exactly same clustering keys of
+            // `ClusteredDistribution`.
+            c.areAllClusterKeysMatched(expressions)
+          } else {
+            // We'll need to find leaf attributes from the partition expressions first.
+            val attributes = expressions.flatMap(_.collectLeaves())
+            attributes.forall(x => requiredClustering.exists(_.semanticEquals(x)))

Review comment:
       For simplicity this PR only support transforms with a single argument, so the check here is very similar to `HashPartitioning`. I plan to work on the support of multi-arguments as a separate PR.
   
   > I think the theory is, as long as the transform function is deterministic, if two rows have the same value of [a, b, c], the calculated key values [f1(a, b), f2(b, c)] are also the same.
   
   Yea agree with this. It may be more complicated if you have duplicated keys in distribution or/and partitioning though. We also need to think how to implement `EnsureRequirements.reorderJoinPredicates` for this case.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r831027263



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       The problem is that, v2 `Partitioning` is an interface, which already has 2 implementations in this PR, including UnknownPartitioning. If we want to add more partitioning in the future, the naming must follow its semantic, rather than just a general name `DataSourcePartitioning`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r831520306



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       yes, this is exactly what we do in `DataSourceBucketTransformExpression`. The check is done in `ShuffleSpec.isCompatibleWith` when we have bucket expressions from both sides.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826697525



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,

Review comment:
       do we still have v2 distribution?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839227144



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be

Review comment:
       when this will be none?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840690750



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DistributionAndOrderingSuiteBase.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.connector
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst
+import org.apache.spark.sql.catalyst.analysis.Resolver
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.expressions.SortOrder
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical
+import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.connector.catalog.InMemoryCatalog
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+
+class DistributionAndOrderingSuiteBase

Review comment:
       nit: it should be an `abstract class` otherwise the test runner will run it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839255715



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -256,6 +285,16 @@ case class EnsureRequirements(
         reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys)
           .orElse(reorderJoinKeysRecursively(
             leftKeys, rightKeys, leftPartitioning, None))
+      case (Some(KeyGroupedPartitioning(clustering, _, _)), _) =>

Review comment:
       Is this change really necessary? The ShuffleSpec checks the cluster key positions, so the join keys order doesn't matter. It only matters for bucketed table with sort columns, as reordering the join keys can take advantage of bucketed table output ordering.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao edited a comment on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao edited a comment on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1051105659


   cc @cloud-fan @viirya @dongjoon-hyun @c21 @rdblue @aokolnychyi while I'm still trying to cover more tests, I think this PR is almost ready for review now, and I'd love to get some feedback from you.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827775484



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       `numBuckets` can be part of the transform function inputs as an int literal. Seems we only need a singe class `DataSourceTransform`

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       `numBuckets` can be part of the transform function inputs as an int literal. Seems we only need a single class `DataSourceTransform`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829317532



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       It's a function for the bucket transform: `bucket(num_buckets, c)` (it could be `bucket(num_buckets, c1, c2, ..)` in future).
   
   The issue here is `canonicalName` for the bucket `BoundFunction`, for obvious reason, doesn't consider the value of `numBuckets`. However, to check of two bucket transforms are compatible, we need to take that into account. That's why we need the extra `DataSourceBucketTransformExpression`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828890346



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Yea, these two should match




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r831025584



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       don't we always check if the inputs are the same first, before checking the function identity? num buckets is a input as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832360245



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {

Review comment:
       BTW, the numBucket is also part of `children`, as an int literal.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828232330



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       Do you mean we require the V2 `bucket` function to accept two inputs? I think it'll work too.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832348410



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.

Review comment:
       I'm thinking about the theory of compatibility check here. This is very complicated and I think we must have some restrictions, to disallow cases like partition by `f1(a + b) * f2(c)` or `f1(a + f2(b))`.
   
   Let's say that the partitioning keys must be either a cluster key, or a transform function with one or more cluster keys. Two key grouped partitionings are compatible iff each of their partition key is compatible, which means:
   1. The partition keys of both sides are cluster keys at the same position
   2. The partition keys of both sides are the same transform function and the function inputs point to the cluster keys at the same positions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r831522317



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Make sense. Let me switch to `KeyGroupedPartitioning` then.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828892873



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       sorry I'm confused. What the internal `BoundFunction` means for a bucket function?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829066438



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkKeyGroupedSpec(specs(a)) && checkKeyGroupedSpec(specs(b)) &&

Review comment:
       Do we need `checkKeyGroupedSpec`? It looks like we can always eliminate shuffles if all the children are compatible. We can update `KeyGroupedShuffleSpec.isCompatible` to always return false if the feature flag is off.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       you are right




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826899017



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanPartitioning.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.v2
+
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.FunctionCatalog
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning
+import org.apache.spark.sql.connector.read.partitioning.{HashPartitioning, UnknownPartitioning}
+import org.apache.spark.util.collection.Utils.sequenceToOption
+
+/**
+ * Extract [[DataSourceV2ScanRelation]] from the input logical plan, convert any V2 partitioning
+ * reported by data sources to their catalyst counterparts. Then, annotate the plan with the result.
+ */
+object V2ScanPartitioning extends Rule[LogicalPlan] with SQLConfHelper {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case d @ DataSourceV2ScanRelation(relation, scan: SupportsReportPartitioning, _, _) =>
+      val funCatalogOpt = relation.catalog.flatMap {
+        case c: FunctionCatalog => Some(c)
+        case _ => None
+      }
+
+      val catalystClustering = scan.outputPartitioning() match {
+        case hp: HashPartitioning => sequenceToOption(hp.clustering().map(
+          V2ExpressionUtils.toCatalyst(_, relation, funCatalogOpt)))
+        case _: UnknownPartitioning => None
+        case p => throw new IllegalArgumentException("Unsupported data source V2 partitioning " +
+            "type: " + p.getClass.getSimpleName)
+      }
+
+      d.copy(clustering = catalystClustering)

Review comment:
       why do we need to put the clustering expressions in the logical plan?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r815402315



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala
##########
@@ -30,10 +30,12 @@ object DistributionAndOrderingUtils {
   def prepareQuery(write: Write, query: LogicalPlan, conf: SQLConf): LogicalPlan = write match {
     case write: RequiresDistributionAndOrdering =>
       val numPartitions = write.requiredNumPartitions()
-      val distribution = write.requiredDistribution match {
-        case d: OrderedDistribution => d.ordering.map(e => toCatalyst(e, query))
-        case d: ClusteredDistribution => d.clustering.map(e => toCatalyst(e, query))
-        case _: UnspecifiedDistribution => Array.empty[Expression]
+      val distribution = toCatalystDistribution(write.requiredDistribution(), query) match {

Review comment:
       Looks like it's close to supporting organize data before writing by repartition/sort by functions defined in `FunctionCatalog`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839915221



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -113,11 +113,14 @@ case class DataSourceV2Relation(
  * @param relation a [[DataSourceV2Relation]]
  * @param scan a DSv2 [[Scan]]
  * @param output the output attributes of this relation
+ * @param clustering if set, the clustering expressions that are used to split the rows in the scan

Review comment:
       Sure




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840817063



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -68,16 +70,36 @@ case class BatchScanExec(
       val newPartitions = scan.toBatch.planInputPartitions()
 
       originalPartitioning match {
-        case p: DataSourcePartitioning if p.numPartitions != newPartitions.size =>
-          throw new SparkException(
-            "Data source must have preserved the original partitioning during runtime filtering; " +
-            s"reported num partitions: ${p.numPartitions}, " +
-            s"num partitions after runtime filtering: ${newPartitions.size}")
+        case p: KeyGroupedPartitioning =>
+          if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) {
+            throw new SparkException("Data source must have preserved the original partitioning " +
+                "during runtime filtering: not all partitions implement HasPartitionKey after " +
+                "filtering")
+          }
+
+          val newRows = new InternalRowSet(p.expressions.map(_.dataType))
+          newRows ++= newPartitions.map(_.asInstanceOf[HasPartitionKey].partitionKey())
+          val oldRows = p.partitionValuesOpt.get
+
+          if (oldRows.size != newRows.size) {

Review comment:
       oh actually `oldRows` here is a `Seq` so we can't directly compare them.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840828868



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.connector
+
+import java.util.Collections
+
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder => V1SortOrder, TransformExpression}
+import org.apache.spark.sql.catalyst.plans.{physical => v1}

Review comment:
       oops let me fix these




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840671228



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connector.read.partitioning;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Expression;
+
+/**
+ * Represents a partitioning where rows are split across partitions based on the
+ * partition transform expressions returned by {@link KeyGroupedPartitioning#clustering}.
+ * <p>
+ * Note: Data source implementations should make sure that for a single partition, all of its
+ * rows must be evaluated to the same partition value after being applied by
+ * {@link KeyGroupedPartitioning#clustering} expressions. Different partitions can share the same
+ * partition value: Spark will group these into a single logical partition during planning phase.
+ *
+ * @since 3.3.0
+ */
+@Evolving
+public class KeyGroupedPartitioning implements Partitioning {
+  private final Expression[] clustering;

Review comment:
       nit: maybe `keys` is a better name to match `KeyGroupedPartitioning`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r821708129



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java
##########
@@ -17,17 +17,21 @@
 
 package org.apache.spark.sql.connector.read.partitioning;
 
-import org.apache.spark.annotation.Evolving;
 import org.apache.spark.sql.connector.read.PartitionReader;
 
 /**
  * A concrete implementation of {@link Distribution}. Represents a distribution where records that
  * share the same values for the {@link #clusteredColumns} will be produced by the same
  * {@link PartitionReader}.
+ * <p>
+ * <b>NOTE</b>: this interface is deprecated in favor of
+ * {@link org.apache.spark.sql.connector.distributions.ClusteredDistribution} and is subject to
+ * future removal.
  *
+ * @see org.apache.spark.sql.connector.distributions.ClusteredDistribution
  * @since 3.0.0
  */
-@Evolving
+@Deprecated
 public class ClusteredDistribution implements Distribution {

Review comment:
       If this is already dead code (Spark does not respect it), maybe it's better to remove it so that data source developers can delete the code that will not work.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r821746938



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -117,7 +118,9 @@ case class DataSourceV2Relation(
 case class DataSourceV2ScanRelation(
     relation: DataSourceV2Relation,
     scan: Scan,
-    output: Seq[AttributeReference]) extends LeafNode with NamedRelation {
+    output: Seq[AttributeReference],
+    distribution: Distribution = UnspecifiedDistribution,

Review comment:
       hmm, ideally a scan node should report output partitioning, the parent node such as join node reports required distribution. This looks a bit counterintuitive.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826694112



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/HashPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link HashPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure
+ * that all rows where {@link HashPartitioning#clustering} evaluate to the same value should be
+ * in the same partition.
  *
- * @since 3.0.0
+ * @since 3.3.0
  */
 @Evolving
-public interface Distribution {}
+public class HashPartitioning implements Partitioning {
+  private final Expression[] clustering;
+  private final int numPartitions;
+

Review comment:
       shall we also include the hash function (v2 function) in this class? it can be optional to represent an unknown hash algorithm.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r838908581



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {

Review comment:
       This is handled at the moment via config `spark.sql.shuffle.partitions`: in `EnsureRequirements` if the number of partitions is smaller than this number, Spark will insert shuffle.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827772320



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       I think the current framework already supports eliminating shuffles when the 2 join children are compatible.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r829066438



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkKeyGroupedSpec(specs(a)) && checkKeyGroupedSpec(specs(b)) &&

Review comment:
       Do we need `checkKeyGroupedSpec`? It looks like we can always eliminate shuffles if all the children are compatible. We can update `KeyGroupedShuffleSpec.isCompatible` to always return false if the feature flag is off.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       you are right




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839219572



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link KeyGroupedPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure that all rows where
+ * {@link KeyGroupedPartitioning#clustering} evaluate to the same value should be in the same
+ * partition.

Review comment:
       ```suggestion
    * partition, and in different partitions if the evaluated values are differnet.
   ```

##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link KeyGroupedPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure that all rows where
+ * {@link KeyGroupedPartitioning#clustering} evaluate to the same value should be in the same
+ * partition.

Review comment:
       ```suggestion
    * partition, and in different partitions if the evaluated values are different.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r815013957



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java
##########
@@ -18,33 +18,26 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
 import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
 
 /**
  * An interface to represent the output data partitioning for a data source, which is returned by
- * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work
- * like a snapshot. Once created, it should be deterministic and always report the same number of
- * partitions and the same "satisfy" result for a certain distribution.
+ * {@link SupportsReportPartitioning#outputPartitioning()}.
  *
  * @since 3.0.0
  */
 @Evolving
 public interface Partitioning {
 
   /**
-   * Returns the number of partitions(i.e., {@link InputPartition}s) the data source outputs.
+   * Returns the distribution guarantee that the data source provides.
    */
-  int numPartitions();
+  Distribution distribution();

Review comment:
       This is a breaking change. To make it less disruptive, I can introduce a new interface and mark this as deprecated, although in that way we may need to add a new method in `SupportsReportPartitioning` or create another interface to replace `SupportsReportPartitioning`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r821708404



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java
##########
@@ -29,16 +28,20 @@
  * partition(the output records of a single {@link PartitionReader}).
  * <p>
  * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
+ * {@link Partitioning}. This means data source developers don't need to
  * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
+ * as possible in {@link Partitioning}.
  * <p>
  * Concrete implementations until now:
  * <ul>
  *   <li>{@link ClusteredDistribution}</li>
  * </ul>
  *
+ * <b>NOTE</b>: this interface is deprecated in favor of
+ * {@link org.apache.spark.sql.connector.distributions.Distribution} and is subject to
+ * future removal.
+ *
  * @since 3.0.0
  */
-@Evolving
+@Deprecated

Review comment:
       ditto




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826573598



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -117,7 +118,9 @@ case class DataSourceV2Relation(
 case class DataSourceV2ScanRelation(
     relation: DataSourceV2Relation,
     scan: Scan,
-    output: Seq[AttributeReference]) extends LeafNode with NamedRelation {
+    output: Seq[AttributeReference],
+    distribution: Distribution = UnspecifiedDistribution,

Review comment:
       I'm using 
   ```scala
   clustering: Option[Seq[Expression]] = None
   ```
   
   for now, since `DataSourceHashPartitioning` is the `outputPartitioning` we construct inside `DataSourceV2ScanExecBase`, so it isn't available at this point.
   
   Let me know if you have any other thought.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826704275



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {
+  override def nullable: Boolean = true
+
+  /**
+   * Whether this [[TransformExpression]] has the same semantics as `other`.
+   * For instance, `bucket(32, c)` is equal to `bucket(32, d)`, but not to `bucket(16, d)` or
+   * `year(c)`.
+   *
+   * This will be used, for instance, by Spark to determine whether storage-partitioned join can
+   * be triggered, by comparing partition transforms from both sides of the join and checking
+   * whether they are compatible.
+   *
+   * @param other the transform expression to compare to
+   * @return true if this and `other` has the same semantics w.r.t to transform, false otherwise.
+   */
+  def equalsTo(other: TransformExpression): Boolean
+}
+
+/**
+ * A transform expression defined by a V2 data source.
+ */
+case class DataSourceTransformExpression(
+    function: BoundFunction,
+    children: Seq[Expression]) extends TransformExpression {
+
+  override def equalsTo(other: TransformExpression): Boolean = other match {
+    case DataSourceTransformExpression(otherFunction, _) =>
+      function.canonicalName() == otherFunction.canonicalName()

Review comment:
       shall we check the `children` as well? I don't think `f(a, b)` and `f(x, y)` can be considered as the same transform.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826708563



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       After reading it, I think it's not really a "hash" partitioning, but more like an "identity" partitioning: rows share the same value w.r.t. the clustering expressions(keys) must be in the same partition, and one partition can only have one value.
   
   Shall we adjust the naming accordingly?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r833515150



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {

Review comment:
       > Spark's bucket function can have a fake BoundFunction which can't be evaluated, but just a marker to indicate it's spark bucket function with a special canonicalName.
   
   Hmm how do you make sure the special `canonicalName` is not used by DSv2 implementors? create some reserved name in Spark?
   
   What do you think of keeping a `BucketTransformExpression` with this change? I'm not sure if there is a good way to differentiate the bucket case versus non-bucket case in `TransformExpression` and also check the value of `numBucket` accordingly.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r833512565



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.

Review comment:
       Agreed. I can revise this section of comments to reflect more on what you said. Also realized that I need to add addition checks to make sure the input of the transform functions are "cluster keys" (e.g., `Attribute`), but not some other function calls. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832353073



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      distribution.clustering.length == otherDistribution.clustering.length &&
+        numPartitions == other.numPartitions &&
+          isClusteringCompatibleWith(otherSpec) &&
+            partitioning.partitionValuesOpt.zip(otherPartitioning.partitionValuesOpt).forall {
+              case (left, right) => left.zip(right).forall { case (l, r) =>
+                ordering.compare(l, r) == 0
+              }
+            }
+    case ShuffleSpecCollection(specs) =>
+      specs.exists(isCompatibleWith)
+    case _ => false
+  }
+
+  /**
+   * Check if the clustering expressions from this spec and `other` are compatible. In
+   * particular, whether each pair of expressions have compatible transform expressions.
+   *
+   * @param other the partitioning from the other side
+   * @return true if the clustering expressions are compatible, false otherwise
+   */
+  private def isClusteringCompatibleWith(other: KeyGroupedShuffleSpec): Boolean = {
+    val expressions = partitioning.expressions
+    val otherPartitioning = other.partitioning
+    val otherExpressions = otherPartitioning.expressions
+
+    expressions.length == otherExpressions.length && {
+      val otherKeyPositions = other.keyPositions
+      keyPositions.zip(otherKeyPositions).forall { case (left, right) =>
+        left.intersect(right).nonEmpty
+      }
+    } && expressions.zip(otherExpressions).forall { case (l, r) =>
+      isExpressionCompatible(l, r)
+    }
+  }
+
+  private def isExpressionCompatible(left: Expression, right: Expression): Boolean =
+    (left, right) match {
+      case (_: LeafExpression, _: LeafExpression) => true
+      case (left: TransformExpression, right: TransformExpression) =>
+        left.isCompatibleWith(right)

Review comment:
       shall we look into the input of the transform function and check if they are compatible? Or we simply do it in `TransformExpression. isCompatibleWith`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sigmod commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sigmod commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1076635300


   cc @somani 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827176851



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       The only difference is it has the `numBuckets` in itself which is used in the equality check. Yes, Hive bucketing can also extend `DataSourceTransformExpression` 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839221411



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TransformExpression.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Represents a partition transform expression, for instance, `bucket`, `days`, `years`, etc.
+ *
+ * @param function the transform function itself. Spark will use it to decide whether two
+ *                 partition transform expressions are compatible.
+ * @param numBucketsOpt the number of buckets if the transform is `bucket`. Unset otherwise.

Review comment:
       Can we include it in the `children` using int `Literal`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839247847



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -68,16 +70,36 @@ case class BatchScanExec(
       val newPartitions = scan.toBatch.planInputPartitions()
 
       originalPartitioning match {
-        case p: DataSourcePartitioning if p.numPartitions != newPartitions.size =>
-          throw new SparkException(
-            "Data source must have preserved the original partitioning during runtime filtering; " +
-            s"reported num partitions: ${p.numPartitions}, " +
-            s"num partitions after runtime filtering: ${newPartitions.size}")
+        case p: KeyGroupedPartitioning =>
+          if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) {
+            throw new SparkException("Data source must have preserved the original partitioning " +
+                "during runtime filtering: not all partitions implement HasPartitionKey after " +
+                "filtering")
+          }
+
+          val newRows = new InternalRowSet(p.expressions.map(_.dataType))
+          newRows ++= newPartitions.map(_.asInstanceOf[HasPartitionKey].partitionKey())
+          val oldRows = p.partitionValuesOpt.get
+
+          if (oldRows.size != newRows.size) {

Review comment:
       Does `InternalRowSet` define `equals` method?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839240942



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.

Review comment:
       nvm, Spark groups partitions and sorts the partitions values by itself, in `BatchScanExec`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840679234



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -68,16 +70,36 @@ case class BatchScanExec(
       val newPartitions = scan.toBatch.planInputPartitions()
 
       originalPartitioning match {
-        case p: DataSourcePartitioning if p.numPartitions != newPartitions.size =>
-          throw new SparkException(
-            "Data source must have preserved the original partitioning during runtime filtering; " +
-            s"reported num partitions: ${p.numPartitions}, " +
-            s"num partitions after runtime filtering: ${newPartitions.size}")
+        case p: KeyGroupedPartitioning =>
+          if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) {
+            throw new SparkException("Data source must have preserved the original partitioning " +
+                "during runtime filtering: not all partitions implement HasPartitionKey after " +
+                "filtering")
+          }
+
+          val newRows = new InternalRowSet(p.expressions.map(_.dataType))
+          newRows ++= newPartitions.map(_.asInstanceOf[HasPartitionKey].partitionKey())
+          val oldRows = p.partitionValuesOpt.get
+
+          if (oldRows.size != newRows.size) {

Review comment:
       then can we simply do `oldRows == newRows` here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839226126



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.

Review comment:
       If that's the case, we should document it in the public v2 class `KeyGroupedPartitioning`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839237069



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -35,7 +36,8 @@ import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 case class BatchScanExec(
     output: Seq[AttributeReference],
     @transient scan: Scan,
-    runtimeFilters: Seq[Expression]) extends DataSourceV2ScanExecBase {
+    runtimeFilters: Seq[Expression],
+    clustering: Option[Seq[Expression]] = None) extends DataSourceV2ScanExecBase {

Review comment:
       ditto, partitioning?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839235344



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -113,11 +113,14 @@ case class DataSourceV2Relation(
  * @param relation a [[DataSourceV2Relation]]
  * @param scan a DSv2 [[Scan]]
  * @param output the output attributes of this relation
+ * @param clustering if set, the clustering expressions that are used to split the rows in the scan

Review comment:
       `partitioning`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r838898280



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      distribution.clustering.length == otherDistribution.clustering.length &&
+        numPartitions == other.numPartitions &&
+          isClusteringCompatibleWith(otherSpec) &&
+            partitioning.partitionValuesOpt.zip(otherPartitioning.partitionValuesOpt).forall {
+              case (left, right) => left.zip(right).forall { case (l, r) =>
+                ordering.compare(l, r) == 0
+              }
+            }
+    case ShuffleSpecCollection(specs) =>
+      specs.exists(isCompatibleWith)
+    case _ => false
+  }
+
+  /**
+   * Check if the clustering expressions from this spec and `other` are compatible. In
+   * particular, whether each pair of expressions have compatible transform expressions.
+   *
+   * @param other the partitioning from the other side
+   * @return true if the clustering expressions are compatible, false otherwise
+   */
+  private def isClusteringCompatibleWith(other: KeyGroupedShuffleSpec): Boolean = {

Review comment:
       I'm lack of a good name here ... let me just inline it then since it's just used in a single place.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827164003



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       This is because we don't allow Spark to re-shuffle the other side of the join using `DataSourceShuffleSpec` yet, so its `canCreatePartitioning` is false. However, we do want to skip shuffle the if all sides of a join all have compatible `DataSourceShuffleSpec`, and so there is a special check here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828234971



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       AFAIK the `bucket` transform function support var-length args: `bucket(num_buckets, c1, c2, ...)`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828234259



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Oops sorry I thought you are commenting on `org.apache.spark.sql.connector.read.partitioning.HashPartitioning`. I think we should change there accordingly, right?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r833957187



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {

Review comment:
       > Hmm how do you make sure the special canonicalName is not used by DSv2 implementors?
   
   It's the same problem for other data sources, right? How can they make sure their `canonicialName`s don't conflicts with other data sources.
   
   I think we are fine if the name is special enough: `spark_murmur3_hash_bucket_function` 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832348410



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.

Review comment:
       I'm thinking about the theory of compatibility check here. This is very complicated and I think we must have some restrictions, to disallow cases like partition by `f1(a + b) * f2(c)` or `f1(a + f2(b))`.
   
   Let's say that the partitioning keys must be either a cluster key, or a transform function with one or more cluster keys. Two key grouped partitionings are compatible iff each of their partition key is compatible, which means:
   1. The partition keys of both sides are cluster keys at the same position
   2. The partition keys of both sides are the same transform function and the function inputs point to the cluster keys at the same positions.
   3. The value of each partition at the same index from both sides are the same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r816442688



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java
##########
@@ -18,33 +18,26 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
 import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
 
 /**
  * An interface to represent the output data partitioning for a data source, which is returned by
- * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work
- * like a snapshot. Once created, it should be deterministic and always report the same number of
- * partitions and the same "satisfy" result for a certain distribution.
+ * {@link SupportsReportPartitioning#outputPartitioning()}.
  *
  * @since 3.0.0
  */
 @Evolving
 public interface Partitioning {
 
   /**
-   * Returns the number of partitions(i.e., {@link InputPartition}s) the data source outputs.
+   * Returns the distribution guarantee that the data source provides.
    */
-  int numPartitions();
+  Distribution distribution();

Review comment:
       I'm fine with that, although these two methods will no longer be called after this PR so it could hit users by surprise.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r816442365



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala
##########
@@ -30,10 +30,12 @@ object DistributionAndOrderingUtils {
   def prepareQuery(write: Write, query: LogicalPlan, conf: SQLConf): LogicalPlan = write match {
     case write: RequiresDistributionAndOrdering =>
       val numPartitions = write.requiredNumPartitions()
-      val distribution = write.requiredDistribution match {
-        case d: OrderedDistribution => d.ordering.map(e => toCatalyst(e, query))
-        case d: ClusteredDistribution => d.clustering.map(e => toCatalyst(e, query))
-        case _: UnspecifiedDistribution => Array.empty[Expression]
+      val distribution = toCatalystDistribution(write.requiredDistribution(), query) match {

Review comment:
       Yes, I think this PR also helps on the write path too with required distribution & ordering.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1051105659


   cc @cloud-fan @viirya @dongjoon-hyun @c21 @rdblue while I'm still trying to cover more tests, I think this PR is almost ready for review now, and I'd love to get some feedback from you.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r821743119



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -281,6 +282,56 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class DataSourcePartitioning(

Review comment:
       How about `DataSourceHashPartitioning` as it can satisfy `ClusteredDistribution`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r822093896



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -281,6 +282,56 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class DataSourcePartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case d: ClusteredDistribution =>
+          val attributes = expressions.flatMap(_.collectLeaves())
+          attributes.forall(c => d.clustering.exists(_.semanticEquals(c)))

Review comment:
       You mean the naming? I'll need to update it with `spark.sql.requireAllClusterKeysForDistribution` too. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r822092262



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java
##########
@@ -17,17 +17,21 @@
 
 package org.apache.spark.sql.connector.read.partitioning;
 
-import org.apache.spark.annotation.Evolving;
 import org.apache.spark.sql.connector.read.PartitionReader;
 
 /**
  * A concrete implementation of {@link Distribution}. Represents a distribution where records that
  * share the same values for the {@link #clusteredColumns} will be produced by the same
  * {@link PartitionReader}.
+ * <p>
+ * <b>NOTE</b>: this interface is deprecated in favor of
+ * {@link org.apache.spark.sql.connector.distributions.ClusteredDistribution} and is subject to
+ * future removal.
  *
+ * @see org.apache.spark.sql.connector.distributions.ClusteredDistribution
  * @since 3.0.0
  */
-@Evolving
+@Deprecated
 public class ClusteredDistribution implements Distribution {

Review comment:
       Sure! I can delete it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828276136



##########
File path: sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/InternalRowSet.scala
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.catalyst.util
+
+import scala.collection.mutable
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Murmur3HashFunction, RowOrdering}
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+
+/**
+ * A mutable Set with [[InternalRow]] as its element type. It uses Spark's internal murmur hash to
+ * compute hash code from an row, and uses [[RowOrdering]] to perform equality checks.
+ *
+ * @param dataTypes the data types for the row keys this set holds
+ */
+class InternalRowSet(val dataTypes: Seq[DataType]) extends mutable.Set[InternalRow] {

Review comment:
       I'm following existing classes like `AttributeSet` here. Plus this makes it more useful if Spark needs to use it in somewhere else in future? there are many functions you get it for free from `mutable.Set`, such as `map`, `foreach`, `filter` as well as overloaded operators.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839234362



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -574,6 +634,80 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   *
+   * Note that we only allow each partition expression to contain a single partition key.
+   * Therefore the mapping here is very similar to that from `HashShuffleSpec`.
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one child from $e, but found ${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering keys
+    //  2. both partitioning have the same number of partitions
+    //  3. partition expressions from both sides are compatible, which means:
+    //    3.1 both sides have the same number of partition expressions
+    //    3.2 for each pair of partition expressions at the same index, the corresponding
+    //        partition keys must share overlapping positions in their respective clustering keys.
+    //    3.3 each pair of partition expressions at the same index must share compatible
+    //        transform functions.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      val expressions = partitioning.expressions
+      val otherExpressions = otherPartitioning.expressions
+
+      distribution.clustering.length == otherDistribution.clustering.length &&
+          numPartitions == other.numPartitions &&
+          expressions.length == otherExpressions.length && {
+            val otherKeyPositions = otherSpec.keyPositions
+            keyPositions.zip(otherKeyPositions).forall { case (left, right) =>
+              left.intersect(right).nonEmpty

Review comment:
       What if the transform function inputs are literals? I think in `isExpressionCompatible`, we should not only check the function canonicalized name, but also the literal inputs, then we can remove the bucket hack in `TransformExpression`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828417187



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/transformExpressions.scala
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.catalyst.expressions
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction
+import org.apache.spark.sql.types.DataType
+
+abstract class TransformExpression extends Expression with Unevaluable {
+  override def nullable: Boolean = true
+
+  /**
+   * Whether this [[TransformExpression]] has the same semantics as `other`.
+   * For instance, `bucket(32, c)` is equal to `bucket(32, d)`, but not to `bucket(16, d)` or
+   * `year(c)`.
+   *
+   * This will be used, for instance, by Spark to determine whether storage-partitioned join can
+   * be triggered, by comparing partition transforms from both sides of the join and checking
+   * whether they are compatible.
+   *
+   * @param other the transform expression to compare to
+   * @return true if this and `other` has the same semantics w.r.t to transform, false otherwise.
+   */
+  def equalsTo(other: TransformExpression): Boolean
+}
+
+/**
+ * A transform expression defined by a V2 data source.
+ */
+case class DataSourceTransformExpression(
+    function: BoundFunction,
+    children: Seq[Expression]) extends TransformExpression {
+
+  override def equalsTo(other: TransformExpression): Boolean = other match {
+    case DataSourceTransformExpression(otherFunction, _) =>
+      function.canonicalName() == otherFunction.canonicalName()

Review comment:
       I think we only check whether the two V2 functions are semantically equal here? i.e., given the same input, whether they map to the same output. The comparison on children is done in `ShuffleSpec.isCompatibleWith`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839898162



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be

Review comment:
       I think for built-in data sources with Spark or Hive bucketing, the scan node can already produce sorted partitions according to the bucket ID so the extra check here won't be necessary.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840701425



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.connector
+
+import java.util.Collections
+
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder => V1SortOrder, TransformExpression}
+import org.apache.spark.sql.catalyst.plans.{physical => v1}
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, KeyGroupedPartitioning}
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog
+import org.apache.spark.sql.connector.catalog.functions._
+import org.apache.spark.sql.connector.distributions.Distribution
+import org.apache.spark.sql.connector.distributions.Distributions
+import org.apache.spark.sql.connector.expressions._
+import org.apache.spark.sql.connector.expressions.Expressions._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.joins.SortMergeJoinExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf._
+import org.apache.spark.sql.types._
+
+class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase {
+  private var originalV2BucketingEnabled: Boolean = false
+  private var originalAutoBroadcastJoinThreshold: Long = -1
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    originalV2BucketingEnabled = conf.getConf(V2_BUCKETING_ENABLED)
+    conf.setConf(V2_BUCKETING_ENABLED, true)
+    originalAutoBroadcastJoinThreshold = conf.getConf(AUTO_BROADCASTJOIN_THRESHOLD)
+    conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, -1L)
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      super.afterAll()
+    } finally {
+      conf.setConf(V2_BUCKETING_ENABLED, originalV2BucketingEnabled)
+      conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, originalAutoBroadcastJoinThreshold)
+    }
+  }
+
+  before {
+    Seq(UnboundYearsFunction, UnboundDaysFunction, UnboundBucketFunction).foreach { f =>
+      catalog.createFunction(Identifier.of(Array.empty, f.name()), f)
+    }
+  }
+
+  after {
+    catalog.clearTables()
+    catalog.clearFunctions()
+  }
+
+  private val emptyProps: java.util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+  private val table: String = "tbl"
+  private val schema = new StructType()
+      .add("id", IntegerType)
+      .add("data", StringType)
+      .add("ts", TimestampType)
+
+  test("clustered distribution: output partitioning should be KeyGroupedPartitioning") {
+    val partitions: Array[Transform] = Array(Expressions.years("ts"))
+
+    // create a table with 3 partitions, partitioned by `years` transform
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    var df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY ts")
+    val v1Distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(YearsFunction, Seq(attr("ts")))))
+    val partitionValues = Seq(50, 51, 52).map(v => InternalRow.fromSeq(Seq(v)))
+
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+
+    // multiple group keys should work too as long as partition keys are subset of them
+    df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY id, ts")
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+  }
+
+  test("non-clustered distribution: fallback to super.partitioning") {
+    val partitions: Array[Transform] = Array(years("ts"))
+    val ordering: Array[SortOrder] = Array(sort(FieldReference("ts"),
+      SortDirection.ASCENDING, NullOrdering.NULLS_FIRST))
+
+    createTable(table, schema, partitions, Distributions.ordered(ordering), ordering)
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val v1Ordering = Seq(V1SortOrder(attr("ts"), Ascending))
+    val v1Distribution = v1.OrderedDistribution(v1Ordering)
+
+    checkQueryPlan(df, v1Distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: no partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: single partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES (0, 'aaa', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.SinglePartition)
+  }
+
+  test("non-clustered distribution: no V2 catalog") {
+    spark.conf.set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName)
+    val nonFunctionCatalog = spark.sessionState.catalogManager.catalog("testcat2")
+        .asInstanceOf[InMemoryTableCatalog]
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])),
+      catalog = nonFunctionCatalog)
+    sql(s"INSERT INTO testcat2.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat2.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    try {
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    } finally {
+      spark.conf.unset("spark.sql.catalog.testcat2")
+    }
+  }
+
+  test("non-clustered distribution: no V2 function provided") {
+    catalog.clearFunctions()
+
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: V2 bucketing disabled") {
+    withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "false") {
+      val partitions: Array[Transform] = Array(bucket(32, "ts"))
+      createTable(table, schema, partitions,
+        Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+      sql(s"INSERT INTO testcat.ns.$table VALUES " +
+          s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+          s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+          s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+      val df = sql(s"SELECT * FROM testcat.ns.$table")
+      val distribution = v1.ClusteredDistribution(
+        Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    }
+  }
+
+  /**
+   * Check whether the query plan from `df` has the expected `distribution`, `ordering` and
+   * `partitioning`.
+   */
+  private def checkQueryPlan(
+      df: DataFrame,
+      distribution: v1.Distribution,
+      partitioning: v1.Partitioning): Unit = {
+    // check distribution & ordering are correctly populated in logical plan
+    val relation = df.queryExecution.optimizedPlan.collect {
+      case r: DataSourceV2ScanRelation => r
+    }.head
+
+    resolveDistribution(distribution, relation) match {
+      case ClusteredDistribution(clustering, _, _) =>
+        assert(relation.keyGroupedPartitioning.isDefined && relation.keyGroupedPartitioning.get == clustering)
+      case _ =>
+        assert(relation.keyGroupedPartitioning.isEmpty)
+    }
+
+    // check distribution, ordering and output partitioning are correctly populated in physical plan
+    val scan = collect(df.queryExecution.executedPlan) {
+      case s: BatchScanExec => s
+    }.head
+
+    val expectedPartitioning = resolvePartitioning(partitioning, scan)
+    assert(expectedPartitioning == scan.outputPartitioning)
+  }
+
+  private def createTable(
+      table: String,
+      schema: StructType,
+      partitions: Array[Transform],
+      distribution: Distribution = Distributions.unspecified(),
+      ordering: Array[expressions.SortOrder] = Array.empty,
+      catalog: InMemoryTableCatalog = catalog): Unit = {
+    catalog.createTable(Identifier.of(Array("ns"), table),
+      schema, partitions, emptyProps, distribution, ordering, None)
+  }
+
+  private val customers: String = "customers"
+  private val customers_schema = new StructType()
+      .add("customer_name", StringType)
+      .add("customer_age", IntegerType)
+      .add("customer_id", LongType)
+
+  private val orders: String = "orders"
+  private val orders_schema = new StructType()
+      .add("order_amount", DoubleType)
+      .add("customer_id", LongType)
+
+  private def testWithCustomersAndOrders(
+      customers_partitions: Array[Transform],
+      customers_distribution: Distribution,
+      orders_partitions: Array[Transform],
+      orders_distribution: Distribution,
+      expectedNumOfShuffleExecs: Int): Unit = {
+    createTable(customers, customers_schema, customers_partitions, customers_distribution)
+    sql(s"INSERT INTO testcat.ns.$customers VALUES " +
+        s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)")
+
+    createTable(orders, orders_schema, orders_partitions, orders_distribution)
+    sql(s"INSERT INTO testcat.ns.$orders VALUES " +
+        s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)")
+
+    val df = sql("SELECT customer_name, customer_age, order_amount " +
+        s"FROM testcat.ns.$customers c JOIN testcat.ns.$orders o " +
+        "ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount")
+
+    val shuffles = collectShuffles(df.queryExecution.executedPlan)
+    assert(shuffles.length == expectedNumOfShuffleExecs)
+
+    checkAnswer(df,
+      Seq(Row("aaa", 10, 100.0), Row("aaa", 10, 200.0), Row("bbb", 20, 150.0),
+        Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50)))
+  }
+
+  private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeExec] = {
+    collect(plan) {
+      case s: SortMergeJoinExec => s
+    }.flatMap(smj =>
+      collect(smj) {
+        case s: ShuffleExchangeExec => s
+      })
+  }
+
+  test("partitioned join: exact distribution (same number of buckets) from both sides") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(4, "customer_id"))
+
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.clustered(orders_partitions.toArray),
+      0)
+  }
+
+  test("partitioned join: number of buckets mismatch should trigger shuffle") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(2, "customer_id"))
+
+    // should shuffle both sides when number of buckets are not the same
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.clustered(orders_partitions.toArray),
+      2)
+  }
+
+  test("partitioned join: only one side reports partitioning") {
+    val customers_partitions = Array(bucket(4, "customer_id"))
+    val orders_partitions = Array(bucket(2, "customer_id"))
+
+    testWithCustomersAndOrders(customers_partitions,
+      Distributions.clustered(customers_partitions.toArray),
+      orders_partitions,
+      Distributions.unspecified(),
+      2)
+  }
+
+  private val items: String = "items"
+  private val items_schema: StructType = new StructType()
+      .add("id", LongType)
+      .add("name", StringType)
+      .add("price", FloatType)
+      .add("arrive_time", TimestampType)
+
+  private val purchases: String = "purchases"
+  private val purchases_schema: StructType = new StructType()
+      .add("item_id", LongType)
+      .add("price", FloatType)
+      .add("time", TimestampType)
+
+  test("partitioned join: join with two partition keys and matching & sorted partitions") {
+    val items_partitions = Array(bucket(8, "id"), days("arrive_time"))
+    createTable(items, items_schema, items_partitions,
+      Distributions.clustered(items_partitions.toArray))
+    sql(s"INSERT INTO testcat.ns.$items VALUES " +
+        s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " +
+        s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " +
+        s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " +
+        s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " +
+        s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))")
+
+    val purchases_partitions = Array(bucket(8, "item_id"), days("time"))
+    createTable(purchases, purchases_schema, purchases_partitions,
+      Distributions.clustered(purchases_partitions.toArray))
+    sql(s"INSERT INTO testcat.ns.$purchases VALUES " +
+        s"(1, 42.0, cast('2020-01-01' as timestamp)), " +
+        s"(1, 44.0, cast('2020-01-15' as timestamp)), " +
+        s"(1, 45.0, cast('2020-01-15' as timestamp)), " +
+        s"(2, 11.0, cast('2020-01-01' as timestamp)), " +
+        s"(3, 19.5, cast('2020-02-01' as timestamp))")
+
+    val df = sql("SELECT id, name, i.price as purchase_price, p.price as sale_price " +
+        s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " +
+        "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price")
+
+    val shuffles = collectShuffles(df.queryExecution.executedPlan)
+    assert(shuffles.isEmpty, "should not add shuffle for both sides of the join")
+    checkAnswer(df,
+      Seq(Row(1, "aa", 40.0, 42.0), Row(1, "aa", 41.0, 44.0), Row(1, "aa", 41.0, 45.0),
+        Row(2, "bb", 10.0, 11.0), Row(2, "bb", 10.5, 11.0), Row(3, "cc", 15.5, 19.5))
+    )
+  }
+
+  test("partitioned join: join with two partition keys and unsorted partitions") {

Review comment:
       the only difference from the test case above is, this test inserts data with a random key order?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839911535



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -574,6 +634,80 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   *
+   * Note that we only allow each partition expression to contain a single partition key.
+   * Therefore the mapping here is very similar to that from `HashShuffleSpec`.
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one child from $e, but found ${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering keys
+    //  2. both partitioning have the same number of partitions
+    //  3. partition expressions from both sides are compatible, which means:
+    //    3.1 both sides have the same number of partition expressions
+    //    3.2 for each pair of partition expressions at the same index, the corresponding
+    //        partition keys must share overlapping positions in their respective clustering keys.
+    //    3.3 each pair of partition expressions at the same index must share compatible
+    //        transform functions.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      val expressions = partitioning.expressions
+      val otherExpressions = otherPartitioning.expressions
+
+      distribution.clustering.length == otherDistribution.clustering.length &&
+          numPartitions == other.numPartitions &&
+          expressions.length == otherExpressions.length && {
+            val otherKeyPositions = otherSpec.keyPositions
+            keyPositions.zip(otherKeyPositions).forall { case (left, right) =>
+              left.intersect(right).nonEmpty

Review comment:
       In case of literals, Spark expects the clustering keys also have literals and thus generates the key position mapping accordingly (in fact `EnsureRequirementsSuite` only use literals for testing).
   
   I feel removing the bucket parameter in `TransformExpression` will complicate things a bit and need to be handled as a special case in several places, since now we have a expression with two children while others only have one (we only allow a single child in transform expression yet), and also that the `numBuckets` literal is special and doesn't have any mapping in the key positions, and thus we'd need to change the logic when checking transform expression compatibility.
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832351152



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -569,6 +629,87 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one leaf expression from $e, but found " +
+          s"${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering expressions
+    //  2. both partitioning have the same number of partitions
+    //  3. clustering expressions from both sides are compatible in terms of number, position
+    //      w.r.t the distribution keys, as well as transform.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      distribution.clustering.length == otherDistribution.clustering.length &&
+        numPartitions == other.numPartitions &&
+          isClusteringCompatibleWith(otherSpec) &&
+            partitioning.partitionValuesOpt.zip(otherPartitioning.partitionValuesOpt).forall {
+              case (left, right) => left.zip(right).forall { case (l, r) =>
+                ordering.compare(l, r) == 0
+              }
+            }
+    case ShuffleSpecCollection(specs) =>
+      specs.exists(isCompatibleWith)
+    case _ => false
+  }
+
+  /**
+   * Check if the clustering expressions from this spec and `other` are compatible. In
+   * particular, whether each pair of expressions have compatible transform expressions.
+   *
+   * @param other the partitioning from the other side
+   * @return true if the clustering expressions are compatible, false otherwise
+   */
+  private def isClusteringCompatibleWith(other: KeyGroupedShuffleSpec): Boolean = {

Review comment:
       The name is a bit misleading, it does not only check compatibility of the cluster keys, but also the partitioning keys.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] somani commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
somani commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r834415035



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {

Review comment:
       In the SPIP, it is mentioned that if there are few large partitions, we will still do a shuffle. We are not trying to do that here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832335388



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class KeyGroupedPartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) =>
+          if (requireAllClusterKeys) {
+            // Checks whether this partitioning is partitioned on exactly same clustering keys of
+            // `ClusteredDistribution`.
+            c.areAllClusterKeysMatched(expressions)
+          } else {
+            // We'll need to find leaf attributes from the partition expressions first.
+            val attributes = expressions.flatMap(_.collectLeaves())
+            attributes.forall(x => requiredClustering.exists(_.semanticEquals(x)))

Review comment:
       I don't think we need to consider `EnsureRequirements.reorderJoinPredicates` here. Whatever the distribution keys are, `[a, b, c]` or `[c, b, a]`, `[f1(a, b), f2(b, c)]` can always satisfy it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r832353898



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkKeyGroupedSpec(specs(a)) && checkKeyGroupedSpec(specs(b)) &&

Review comment:
       makes sense




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] pan3793 commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r815401526



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java
##########
@@ -18,33 +18,26 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
 import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
 
 /**
  * An interface to represent the output data partitioning for a data source, which is returned by
- * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work
- * like a snapshot. Once created, it should be deterministic and always report the same number of
- * partitions and the same "satisfy" result for a certain distribution.
+ * {@link SupportsReportPartitioning#outputPartitioning()}.
  *
  * @since 3.0.0
  */
 @Evolving
 public interface Partitioning {
 
   /**
-   * Returns the number of partitions(i.e., {@link InputPartition}s) the data source outputs.
+   * Returns the distribution guarantee that the data source provides.
    */
-  int numPartitions();
+  Distribution distribution();

Review comment:
       How about keep the methods and mark it as `@deprecated`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r822636529



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -117,7 +118,9 @@ case class DataSourceV2Relation(
 case class DataSourceV2ScanRelation(
     relation: DataSourceV2Relation,
     scan: Scan,
-    output: Seq[AttributeReference]) extends LeafNode with NamedRelation {
+    output: Seq[AttributeReference],
+    distribution: Distribution = UnspecifiedDistribution,

Review comment:
       Can we just use `Partitioning` here? Which can be `DataSourceHashPartitioning` under the hood.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on pull request #35657:
URL: https://github.com/apache/spark/pull/35657#issuecomment-1067554812


   @cloud-fan updated according to the discuss we had offline. Please take another look when you get a chance. Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826693036



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/HashPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions

Review comment:
       ... based on the hash value of the "clustering" expressions.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826703650



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       How is it different from `DataSourceTransformExpression`? I think Hive bucketed table can also use `DataSourceTransformExpression` in the future.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826695344



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java
##########
@@ -18,33 +18,26 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.InputPartition;
 import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
 
 /**
  * An interface to represent the output data partitioning for a data source, which is returned by
- * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work
- * like a snapshot. Once created, it should be deterministic and always report the same number of
- * partitions and the same "satisfy" result for a certain distribution.
+ * {@link SupportsReportPartitioning#outputPartitioning()}.
+ * <p>
+ * Note: implementors <b>should NOT</b> directly implement this interface. Instead, they should
+ * implement one of the following subclasses:

Review comment:
       ```suggestion
    * use one of the following subclasses:
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826723088



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       like `KeyGroupedPartitioning`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r826903884



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `DataSourcePartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {
+            case Seq(a, b) =>
+              checkDataSourceSpec(specs(a)) && checkDataSourceSpec(specs(b)) &&
+                  specs(a).isCompatibleWith(specs(b))
+          }.forall(_ == true)
+
       children = children.zip(requiredChildDistributions).zipWithIndex.map {
-        case ((child, _), idx) if !childrenIndexes.contains(idx) =>
+        case ((child, _), idx) if allCompatible || !childrenIndexes.contains(idx) =>

Review comment:
       hmm why do we need this change? I thought the `DataSourceShuffleSpec` can take care all of these.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r827157771



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.

Review comment:
       Yes, I also think `HashPartitioning` is not accurate here. What do you think of `DataSourcePartitioning` (different from the `DataSourceHashPartitioning` in catalyst), or `DataStoragePartitioning`? `KeyGroupedPartitioning` sounds fine too.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828458181



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
##########
@@ -44,20 +52,108 @@ object V2ExpressionUtils extends SQLConfHelper {
     refs.map(ref => resolveRef[T](ref, plan))
   }
 
-  def toCatalyst(expr: V2Expression, query: LogicalPlan): Expression = {
+  /**
+   * Converts the array of input V2 [[V2SortOrder]] into their counterparts in catalyst.
+   */
+  def toCatalystOrdering(ordering: Array[V2SortOrder], query: LogicalPlan): Seq[SortOrder] = {
+    sequenceToOption(ordering.map(toCatalyst(_, query))).asInstanceOf[Option[Seq[SortOrder]]]
+        .getOrElse(Seq.empty)
+  }
+
+  /**
+   * Converts the V2 [[V2Distribution]] into its counterpart in catalyst [[Distribution]].
+   *
+   * If `funCatalogOpt` is set, it will be used to lookup any V2 function referenced
+   * by the input distribution. For instance, a bucket transform in the distribution requires a
+   * corresponding function to exist in the catalog, in order for Spark to leverage bucket join
+   * for the V2 data sources.
+   *
+   * This returns [[UnspecifiedDistribution]] if any non-identity transform is used in the
+   * distribution, AND the `funCatalogOpt` is not set OR the corresponding function is not
+   * defined in the catalog.
+   */
+  def toCatalystDistribution(
+      distribution: V2Distribution,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Distribution = distribution match {
+    case d: V2OrderedDistribution =>
+      val resolvedOrdering = toCatalystOrdering(d.ordering(), query)
+      OrderedDistribution(resolvedOrdering)
+    case d: V2ClusteredDistribution =>
+      sequenceToOption(d.clustering.map(toCatalyst(_, query, funCatalogOpt)))
+          .map(ClusteredDistribution(_))
+          .getOrElse(UnspecifiedDistribution)
+    case _: V2UnspecifiedDistribution =>
+      UnspecifiedDistribution
+  }
+
+  def toCatalyst(
+      expr: V2Expression,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = {
     expr match {
+      case t: Transform =>
+        toCatalystTransform(t, query, funCatalogOpt)
       case SortValue(child, direction, nullOrdering) =>
-        val catalystChild = toCatalyst(child, query)
-        SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
-      case IdentityTransform(ref) =>
-        resolveRef[NamedExpression](ref, query)
+        toCatalyst(child, query, funCatalogOpt).map { catalystChild =>
+          SortOrder(catalystChild, toCatalyst(direction), toCatalyst(nullOrdering), Seq.empty)
+        }
       case ref: FieldReference =>
-        resolveRef[NamedExpression](ref, query)
+        Some(resolveRef[NamedExpression](ref, query))
       case _ =>
         throw new AnalysisException(s"$expr is not currently supported")
     }
   }
 
+  def toCatalystTransform(
+      trans: Transform,
+      query: LogicalPlan,
+      funCatalogOpt: Option[FunctionCatalog] = None): Option[Expression] = trans match {
+    case IdentityTransform(ref) =>
+      Some(resolveRef[NamedExpression](ref, query))
+    case BucketTransform(numBuckets, refs, sorted) if sorted.isEmpty && refs.length == 1 =>
+      val resolvedRefs = refs.map(r => resolveRef[NamedExpression](r, query))
+      funCatalogOpt.flatMap { catalog =>
+        loadV2Function(catalog, "bucket", resolvedRefs).map { bound =>
+          DataSourceBucketTransformExpression(numBuckets, bound, resolvedRefs)

Review comment:
       If we don't want `DataSourceBucketTransformExpression` then we'd have to create a wrapper `BoundFunction` for bucket, such as:
   ```scala
     private case class BucketBoundFunction(
         numBuckets: Int, internal: BoundFunction) extends BoundFunction {
       override def inputTypes(): Array[DataType] = internal.inputTypes()
       override def resultType(): DataType = internal.resultType()
       override def name(): String = internal.name()
       override def canonicalName(): String = ???
     }
   ```
   
   and pass this to `DataSourceTransformExpression `. The problem is we can't easily generate a `canonicalName` here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839225681



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.

Review comment:
       > ascending order
   
   I thought it's ordered by partition id, e.g. `[part1_value, part2_value, ...]`. Are we require the partitioning to sort its partitions by the partition value in ascending order?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839244934



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -18,27 +18,37 @@
 package org.apache.spark.sql.connector.read.partitioning;
 
 import org.apache.spark.annotation.Evolving;
-import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.expressions.Expression;
 
 /**
- * An interface to represent data distribution requirement, which specifies how the records should
- * be distributed among the data partitions (one {@link PartitionReader} outputs data for one
- * partition).
+ * Represents a partitioning where rows are split across partitions based on the expressions
+ * returned by {@link KeyGroupedPartitioning#clustering}.
  * <p>
- * Note that this interface has nothing to do with the data ordering inside one
- * partition(the output records of a single {@link PartitionReader}).
- * <p>
- * The instance of this interface is created and provided by Spark, then consumed by
- * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to
- * implement this interface, but need to catch as more concrete implementations of this interface
- * as possible in {@link Partitioning#satisfy(Distribution)}.
- * <p>
- * Concrete implementations until now:
- * <ul>
- *   <li>{@link ClusteredDistribution}</li>
- * </ul>
+ * Data source implementations should make sure that all rows where
+ * {@link KeyGroupedPartitioning#clustering} evaluate to the same value should be in the same
+ * partition.

Review comment:
       I think we need to spend more time making this doc clear. `KeyGroupedPartitioning` does mean that, logiclally each output partition should only contains rows with the same value, and different partitions have different values. But physically the data source can put rows with the same value in different partitions, and Spark will group them later.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839237069



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -35,7 +36,8 @@ import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 case class BatchScanExec(
     output: Seq[AttributeReference],
     @transient scan: Scan,
-    runtimeFilters: Seq[Expression]) extends DataSourceV2ScanExecBase {
+    runtimeFilters: Seq[Expression],
+    clustering: Option[Seq[Expression]] = None) extends DataSourceV2ScanExecBase {

Review comment:
       ditto, keyGroupedPartitioning?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r839235344



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
##########
@@ -113,11 +113,14 @@ case class DataSourceV2Relation(
  * @param relation a [[DataSourceV2Relation]]
  * @param scan a DSv2 [[Scan]]
  * @param output the output attributes of this relation
+ * @param clustering if set, the clustering expressions that are used to split the rows in the scan

Review comment:
       `keyGroupedPartitioning`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r828921559



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -305,6 +306,63 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
     newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren)
 }
 
+/**
+ * Represents a partitioning where rows are split across partitions based on transforms defined
+ * by `expressions`. `partitionValuesOpt`, if defined, should contain value of partition key(s) in
+ * ascending order, after evaluated by the transforms in `expressions`, for each input partition.
+ * In addition, its length must be the same as the number of input partitions (and thus is a 1-1
+ * mapping), and each row in `partitionValuesOpt` must be unique.
+ *
+ * For example, if `expressions` is `[years(ts_col)]`, then a valid value of `partitionValuesOpt` is
+ * `[0, 1, 2]`, which represents 3 input partitions with distinct partition values. All rows
+ * in each partition have the same value for column `ts_col` (which is of timestamp type), after
+ * being applied by the `years` transform.
+ *
+ * On the other hand, `[0, 0, 1]` is not a valid value for `partitionValuesOpt` since `0` is
+ * duplicated twice.
+ *
+ * @param expressions partition expressions for the partitioning.
+ * @param numPartitions the number of partitions
+ * @param partitionValuesOpt if set, the values for the cluster keys of the distribution, must be
+ *                           in ascending order.
+ */
+case class KeyGroupedPartitioning(
+    expressions: Seq[Expression],
+    numPartitions: Int,
+    partitionValuesOpt: Option[Seq[InternalRow]] = None) extends Partitioning {
+
+  override def satisfies0(required: Distribution): Boolean = {
+    super.satisfies0(required) || {
+      required match {
+        case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) =>
+          if (requireAllClusterKeys) {
+            // Checks whether this partitioning is partitioned on exactly same clustering keys of
+            // `ClusteredDistribution`.
+            c.areAllClusterKeysMatched(expressions)
+          } else {
+            // We'll need to find leaf attributes from the partition expressions first.
+            val attributes = expressions.flatMap(_.collectLeaves())
+            attributes.forall(x => requiredClustering.exists(_.semanticEquals(x)))

Review comment:
       We need to add some comments to explain the theory here. If the required distribution is `[a, b, c]`, and the key-grouped partitioning is `[f1(a, b), f2(b, c)]`, can we satisfy the distribution requirement?
   
   I think the theory is, as long as the transform function is deterministic, if two rows have the same value of `[a, b, c]`, the calculated key values `[f1(a, b), f2(b, c)]` are also the same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840228483



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
##########
@@ -574,6 +634,80 @@ case class HashShuffleSpec(
   override def numPartitions: Int = partitioning.numPartitions
 }
 
+case class KeyGroupedShuffleSpec(
+    partitioning: KeyGroupedPartitioning,
+    distribution: ClusteredDistribution) extends ShuffleSpec {
+
+  /**
+   * A sequence where each element is a set of positions of the partition expression to the cluster
+   * keys. For instance, if cluster keys are [a, b, b] and partition expressions are
+   * [bucket(4, a), years(b)], the result will be [(0), (1, 2)].
+   *
+   * Note that we only allow each partition expression to contain a single partition key.
+   * Therefore the mapping here is very similar to that from `HashShuffleSpec`.
+   */
+  lazy val keyPositions: Seq[mutable.BitSet] = {
+    val distKeyToPos = mutable.Map.empty[Expression, mutable.BitSet]
+    distribution.clustering.zipWithIndex.foreach { case (distKey, distKeyPos) =>
+      distKeyToPos.getOrElseUpdate(distKey.canonicalized, mutable.BitSet.empty).add(distKeyPos)
+    }
+    partitioning.expressions.map { e =>
+      val leaves = e.collectLeaves()
+      assert(leaves.size == 1, s"Expected exactly one child from $e, but found ${leaves.size}")
+      distKeyToPos.getOrElse(leaves.head.canonicalized, mutable.BitSet.empty)
+    }
+  }
+
+  private lazy val ordering: Ordering[InternalRow] =
+    RowOrdering.createNaturalAscendingOrdering(partitioning.expressions.map(_.dataType))
+
+  override def numPartitions: Int = partitioning.numPartitions
+
+  override def isCompatibleWith(other: ShuffleSpec): Boolean = other match {
+    // Here we check:
+    //  1. both distributions have the same number of clustering keys
+    //  2. both partitioning have the same number of partitions
+    //  3. partition expressions from both sides are compatible, which means:
+    //    3.1 both sides have the same number of partition expressions
+    //    3.2 for each pair of partition expressions at the same index, the corresponding
+    //        partition keys must share overlapping positions in their respective clustering keys.
+    //    3.3 each pair of partition expressions at the same index must share compatible
+    //        transform functions.
+    //  4. the partition values, if present on both sides, are following the same order.
+    case otherSpec @ KeyGroupedShuffleSpec(otherPartitioning, otherDistribution) =>
+      val expressions = partitioning.expressions
+      val otherExpressions = otherPartitioning.expressions
+
+      distribution.clustering.length == otherDistribution.clustering.length &&
+          numPartitions == other.numPartitions &&
+          expressions.length == otherExpressions.length && {
+            val otherKeyPositions = otherSpec.keyPositions
+            keyPositions.zip(otherKeyPositions).forall { case (left, right) =>
+              left.intersect(right).nonEmpty

Review comment:
       OK, it's internal class anyway so I'm fine with the current bucket hack to make it simple: transform function should only have one parameter and it's a field reference.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840687078



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
##########
@@ -137,8 +138,16 @@ case class EnsureRequirements(
         Some(finalCandidateSpecs.values.maxBy(_.numPartitions))
       }
 
+      // Check if 1) all children are of `KeyGroupedPartitioning` and 2) they are all compatible
+      // with each other. If both are true, skip shuffle.
+      val allCompatible = childrenIndexes.sliding(2).map {

Review comment:
       nit: instead of `.map(...).forall(_ == true)`, we can do `.forall(...)` directly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840698389



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.connector
+
+import java.util.Collections
+
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder => V1SortOrder, TransformExpression}
+import org.apache.spark.sql.catalyst.plans.{physical => v1}
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, KeyGroupedPartitioning}
+import org.apache.spark.sql.connector.catalog.Identifier
+import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog
+import org.apache.spark.sql.connector.catalog.functions._
+import org.apache.spark.sql.connector.distributions.Distribution
+import org.apache.spark.sql.connector.distributions.Distributions
+import org.apache.spark.sql.connector.expressions._
+import org.apache.spark.sql.connector.expressions.Expressions._
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.joins.SortMergeJoinExec
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf._
+import org.apache.spark.sql.types._
+
+class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase {
+  private var originalV2BucketingEnabled: Boolean = false
+  private var originalAutoBroadcastJoinThreshold: Long = -1
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    originalV2BucketingEnabled = conf.getConf(V2_BUCKETING_ENABLED)
+    conf.setConf(V2_BUCKETING_ENABLED, true)
+    originalAutoBroadcastJoinThreshold = conf.getConf(AUTO_BROADCASTJOIN_THRESHOLD)
+    conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, -1L)
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      super.afterAll()
+    } finally {
+      conf.setConf(V2_BUCKETING_ENABLED, originalV2BucketingEnabled)
+      conf.setConf(AUTO_BROADCASTJOIN_THRESHOLD, originalAutoBroadcastJoinThreshold)
+    }
+  }
+
+  before {
+    Seq(UnboundYearsFunction, UnboundDaysFunction, UnboundBucketFunction).foreach { f =>
+      catalog.createFunction(Identifier.of(Array.empty, f.name()), f)
+    }
+  }
+
+  after {
+    catalog.clearTables()
+    catalog.clearFunctions()
+  }
+
+  private val emptyProps: java.util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+  private val table: String = "tbl"
+  private val schema = new StructType()
+      .add("id", IntegerType)
+      .add("data", StringType)
+      .add("ts", TimestampType)
+
+  test("clustered distribution: output partitioning should be KeyGroupedPartitioning") {
+    val partitions: Array[Transform] = Array(Expressions.years("ts"))
+
+    // create a table with 3 partitions, partitioned by `years` transform
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    var df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY ts")
+    val v1Distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(YearsFunction, Seq(attr("ts")))))
+    val partitionValues = Seq(50, 51, 52).map(v => InternalRow.fromSeq(Seq(v)))
+
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+
+    // multiple group keys should work too as long as partition keys are subset of them
+    df = sql(s"SELECT count(*) FROM testcat.ns.$table GROUP BY id, ts")
+    checkQueryPlan(df, v1Distribution,
+      KeyGroupedPartitioning(v1Distribution.clustering, partitionValues))
+  }
+
+  test("non-clustered distribution: fallback to super.partitioning") {
+    val partitions: Array[Transform] = Array(years("ts"))
+    val ordering: Array[SortOrder] = Array(sort(FieldReference("ts"),
+      SortDirection.ASCENDING, NullOrdering.NULLS_FIRST))
+
+    createTable(table, schema, partitions, Distributions.ordered(ordering), ordering)
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val v1Ordering = Seq(V1SortOrder(attr("ts"), Ascending))
+    val v1Distribution = v1.OrderedDistribution(v1Ordering)
+
+    checkQueryPlan(df, v1Distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: no partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: single partition") {
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES (0, 'aaa', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.ClusteredDistribution(
+      Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+    checkQueryPlan(df, distribution, v1.SinglePartition)
+  }
+
+  test("non-clustered distribution: no V2 catalog") {
+    spark.conf.set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName)
+    val nonFunctionCatalog = spark.sessionState.catalogManager.catalog("testcat2")
+        .asInstanceOf[InMemoryTableCatalog]
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])),
+      catalog = nonFunctionCatalog)
+    sql(s"INSERT INTO testcat2.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat2.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    try {
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    } finally {
+      spark.conf.unset("spark.sql.catalog.testcat2")
+    }
+  }
+
+  test("non-clustered distribution: no V2 function provided") {
+    catalog.clearFunctions()
+
+    val partitions: Array[Transform] = Array(bucket(32, "ts"))
+    createTable(table, schema, partitions,
+      Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+    sql(s"INSERT INTO testcat.ns.$table VALUES " +
+        s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+        s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+        s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+    val df = sql(s"SELECT * FROM testcat.ns.$table")
+    val distribution = v1.UnspecifiedDistribution
+
+    checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+  }
+
+  test("non-clustered distribution: V2 bucketing disabled") {
+    withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "false") {
+      val partitions: Array[Transform] = Array(bucket(32, "ts"))
+      createTable(table, schema, partitions,
+        Distributions.clustered(partitions.map(_.asInstanceOf[Expression])))
+      sql(s"INSERT INTO testcat.ns.$table VALUES " +
+          s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " +
+          s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " +
+          s"(2, 'ccc', CAST('2020-01-01' AS timestamp))")
+
+      val df = sql(s"SELECT * FROM testcat.ns.$table")
+      val distribution = v1.ClusteredDistribution(
+        Seq(TransformExpression(BucketFunction, Seq(attr("ts")), Some(32))))
+
+      checkQueryPlan(df, distribution, v1.UnknownPartitioning(0))
+    }
+  }
+
+  /**
+   * Check whether the query plan from `df` has the expected `distribution`, `ordering` and
+   * `partitioning`.
+   */
+  private def checkQueryPlan(
+      df: DataFrame,
+      distribution: v1.Distribution,
+      partitioning: v1.Partitioning): Unit = {
+    // check distribution & ordering are correctly populated in logical plan
+    val relation = df.queryExecution.optimizedPlan.collect {
+      case r: DataSourceV2ScanRelation => r
+    }.head
+
+    resolveDistribution(distribution, relation) match {
+      case ClusteredDistribution(clustering, _, _) =>
+        assert(relation.keyGroupedPartitioning.isDefined && relation.keyGroupedPartitioning.get == clustering)
+      case _ =>
+        assert(relation.keyGroupedPartitioning.isEmpty)
+    }
+
+    // check distribution, ordering and output partitioning are correctly populated in physical plan
+    val scan = collect(df.queryExecution.executedPlan) {
+      case s: BatchScanExec => s
+    }.head
+
+    val expectedPartitioning = resolvePartitioning(partitioning, scan)
+    assert(expectedPartitioning == scan.outputPartitioning)
+  }
+
+  private def createTable(
+      table: String,
+      schema: StructType,
+      partitions: Array[Transform],
+      distribution: Distribution = Distributions.unspecified(),
+      ordering: Array[expressions.SortOrder] = Array.empty,
+      catalog: InMemoryTableCatalog = catalog): Unit = {
+    catalog.createTable(Identifier.of(Array("ns"), table),
+      schema, partitions, emptyProps, distribution, ordering, None)
+  }
+
+  private val customers: String = "customers"
+  private val customers_schema = new StructType()
+      .add("customer_name", StringType)
+      .add("customer_age", IntegerType)
+      .add("customer_id", LongType)
+
+  private val orders: String = "orders"
+  private val orders_schema = new StructType()
+      .add("order_amount", DoubleType)
+      .add("customer_id", LongType)
+
+  private def testWithCustomersAndOrders(
+      customers_partitions: Array[Transform],
+      customers_distribution: Distribution,
+      orders_partitions: Array[Transform],
+      orders_distribution: Distribution,
+      expectedNumOfShuffleExecs: Int): Unit = {
+    createTable(customers, customers_schema, customers_partitions, customers_distribution)
+    sql(s"INSERT INTO testcat.ns.$customers VALUES " +
+        s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)")
+
+    createTable(orders, orders_schema, orders_partitions, orders_distribution)
+    sql(s"INSERT INTO testcat.ns.$orders VALUES " +
+        s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)")
+
+    val df = sql("SELECT customer_name, customer_age, order_amount " +
+        s"FROM testcat.ns.$customers c JOIN testcat.ns.$orders o " +
+        "ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount")
+
+    val shuffles = collectShuffles(df.queryExecution.executedPlan)
+    assert(shuffles.length == expectedNumOfShuffleExecs)
+
+    checkAnswer(df,
+      Seq(Row("aaa", 10, 100.0), Row("aaa", 10, 200.0), Row("bbb", 20, 150.0),
+        Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50)))
+  }
+
+  private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeExec] = {
+    collect(plan) {
+      case s: SortMergeJoinExec => s

Review comment:
       why don't we collect `ShuffleExchangeExec` directly here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840819827



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connector.read.partitioning;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.connector.expressions.Expression;
+
+/**
+ * Represents a partitioning where rows are split across partitions based on the
+ * partition transform expressions returned by {@link KeyGroupedPartitioning#clustering}.
+ * <p>
+ * Note: Data source implementations should make sure that for a single partition, all of its
+ * rows must be evaluated to the same partition value after being applied by
+ * {@link KeyGroupedPartitioning#clustering} expressions. Different partitions can share the same
+ * partition value: Spark will group these into a single logical partition during planning phase.
+ *
+ * @since 3.3.0
+ */
+@Evolving
+public class KeyGroupedPartitioning implements Partitioning {
+  private final Expression[] clustering;

Review comment:
       👍 I think `keys` sounds better 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840819337



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala
##########
@@ -17,23 +17,27 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
-import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
-import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils.toCatalyst
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.V2ExpressionUtils._
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RepartitionByExpression, Sort}
-import org.apache.spark.sql.connector.distributions.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution}
+import org.apache.spark.sql.connector.distributions._
 import org.apache.spark.sql.connector.write.{RequiresDistributionAndOrdering, Write}
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.collection.Utils.sequenceToOption
 
 object DistributionAndOrderingUtils {
 
   def prepareQuery(write: Write, query: LogicalPlan, conf: SQLConf): LogicalPlan = write match {
     case write: RequiresDistributionAndOrdering =>
       val numPartitions = write.requiredNumPartitions()
+
       val distribution = write.requiredDistribution match {
-        case d: OrderedDistribution => d.ordering.map(e => toCatalyst(e, query))
-        case d: ClusteredDistribution => d.clustering.map(e => toCatalyst(e, query))
-        case _: UnspecifiedDistribution => Array.empty[Expression]
+        case d: OrderedDistribution => toCatalystOrdering(d.ordering(), query)
+        case d: ClusteredDistribution =>
+          sequenceToOption(d.clustering.map(e => toCatalyst(e, query)))
+              .getOrElse(Seq.empty[Expression])

Review comment:
       Hmm I really need to fix my IntelliJ setting ... 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] sunchao commented on a change in pull request #35657: [SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join

Posted by GitBox <gi...@apache.org>.
sunchao commented on a change in pull request #35657:
URL: https://github.com/apache/spark/pull/35657#discussion_r840815754



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -68,16 +70,36 @@ case class BatchScanExec(
       val newPartitions = scan.toBatch.planInputPartitions()
 
       originalPartitioning match {
-        case p: DataSourcePartitioning if p.numPartitions != newPartitions.size =>
-          throw new SparkException(
-            "Data source must have preserved the original partitioning during runtime filtering; " +
-            s"reported num partitions: ${p.numPartitions}, " +
-            s"num partitions after runtime filtering: ${newPartitions.size}")
+        case p: KeyGroupedPartitioning =>
+          if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) {
+            throw new SparkException("Data source must have preserved the original partitioning " +
+                "during runtime filtering: not all partitions implement HasPartitionKey after " +
+                "filtering")
+          }
+
+          val newRows = new InternalRowSet(p.expressions.map(_.dataType))
+          newRows ++= newPartitions.map(_.asInstanceOf[HasPartitionKey].partitionKey())
+          val oldRows = p.partitionValuesOpt.get
+
+          if (oldRows.size != newRows.size) {

Review comment:
       yea we can - I think my original intention here was to provide different error messages based on whether a) the number of partitions changed, or b) the number of partitions are the same but partition values have changed. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org