You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "Hisoka-X (via GitHub)" <gi...@apache.org> on 2023/07/28 02:10:45 UTC

[GitHub] [spark] Hisoka-X opened a new pull request, #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Hisoka-X opened a new pull request, #42194:
URL: https://github.com/apache/spark/pull/42194

   <!--
   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?
   When only one side of a SPJ (Storage-Partitioned Join) is KeyGroupedPartitioning, Spark currently needs to shuffle both sides using HashPartitioning. However, we may just need to shuffle the other side according to the partition transforms defined in KeyGroupedPartitioning. This is especially useful when the other side is relatively small.
   1. Add new config `spark.sql.sources.v2.bucketing.shuffleOneSide.enabled` to control this feature enable or not.
   2. Add `PartitionValueMapShuffleSpec`,`PartitionValueMapPartitioning` and `PartitionValueMapPartitioner` use to partition when we know the tranform value of another side (KeyGroupedPartitioning at now). Spark already know the partition value with partition id of KeyGroupedPartitioning side in `EnsureRequirements`. Then save it in `PartitionValueMapPartitioner` use to shuffle another partition, to make sure the same key data will shuffle into same partition. 
   3. only `identity` transform will work now. Because have another a problem for now, same transform between DS V2 connector implement and catalog function will report different value, before solve this problem, we should only support `identity`. eg: in test package, `YearFunction` https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/connector/catalog/functions/transformFunctions.scala#L47 and https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryBaseTable.scala#L143
   
   <!--
   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.
     4. If you fix some SQL features, you can provide some references of other DBMSes.
     5. If there is design documentation, please add the link.
     6. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   Reduce data shuffle in specific SPJ scenarios
   <!--
   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.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   <!--
   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'.
   -->
   
   
   ### How was this patch tested?
   add new test
   <!--
   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.
   -->
   


-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1303194059


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,21 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map.

Review Comment:
   nit: "partitions all records using partition value map"



-- 
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 #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1675582251

   cc @cloud-fan for another check 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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1288959693


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala:
##########
@@ -299,6 +301,12 @@ object ShuffleExchangeExec {
           ascending = true,
           samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
       case SinglePartition => new ConstantPartitioner
+      case KeyGroupedPartitioning(expressions, n, partitionValues) =>

Review Comment:
   It'd be great if we can also add a few tests in `EnsureRequirementsSuite` to check that Spark picks the right side to shuffle when one side of a join is non-key grouped partitioning and the other side is.



-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1287757003


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala:
##########
@@ -519,6 +519,35 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning {
   }
 }
 
+/**
+ * Represents a partitioning use partition value map to partition.
+ */
+case class PartitionValueMapPartitioning(

Review Comment:
   Could we reuse `KeyGroupedPartitioning` instead of creating a new `Partitioning`? they look mostly similar, except `valueMap` which I think can be computed using `partitionValues`? The `PartitionValueMapShuffleSpec` also look similar to `KeyGroupedShuffleSpec`
   



##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],
+    override val numPartitions: Int) extends Partitioner {
+  override def getPartition(key: Any): Int = {
+    val keys = key.asInstanceOf[Seq[Any]]
+    valueMap.getOrElseUpdate(keys, Utils.nonNegativeMod(keys.hashCode, numPartitions))

Review Comment:
   When do we need to update this map? will this collide with one of the existing mappings?



##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   Instead of using `Seq[Any]`, can we use `InternalRowComparableWrapper` here?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala:
##########
@@ -368,150 +368,197 @@ case class EnsureRequirements(
     var newLeft = left
     var newRight = right
 
-    val specs = Seq(left, right).zip(requiredChildDistribution).map { case (p, d) =>
+    val specsOpts = Seq(left, right).zip(requiredChildDistribution).map { case (p, d) =>
       if (!d.isInstanceOf[ClusteredDistribution]) return None
       val cd = d.asInstanceOf[ClusteredDistribution]
       val specOpt = createKeyGroupedShuffleSpec(p.outputPartitioning, cd)
-      if (specOpt.isEmpty) return None
-      specOpt.get
+      specOpt
+    }
+    val specsAllExist = specsOpts.forall(_.nonEmpty)
+    if ((!conf.v2BucketingShuffleOneSideEnabled && !specsAllExist)
+      || specsOpts.count(_.isEmpty) > 1) {
+      return None
     }
-
-    val leftSpec = specs.head
-    val rightSpec = specs(1)
 
     var isCompatible = false
-    if (!conf.v2BucketingPushPartValuesEnabled) {
-      isCompatible = leftSpec.isCompatibleWith(rightSpec)
-    } else {
-      logInfo("Pushing common partition values for storage-partitioned join")
-      isCompatible = leftSpec.areKeysCompatible(rightSpec)
-
-      // Partition expressions are compatible. Regardless of whether partition values
-      // match from both sides of children, we can calculate a superset of partition values and
-      // push-down to respective data sources so they can adjust their output partitioning by
-      // filling missing partition keys with empty partitions. As result, we can still avoid
-      // shuffle.
-      //
-      // For instance, if two sides of a join have partition expressions
-      // `day(a)` and `day(b)` respectively
-      // (the join query could be `SELECT ... FROM t1 JOIN t2 on t1.a = t2.b`), but
-      // with different partition values:
-      //   `day(a)`: [0, 1]
-      //   `day(b)`: [1, 2, 3]
-      // Following the case 2 above, we don't have to shuffle both sides, but instead can
-      // just push the common set of partition values: `[0, 1, 2, 3]` down to the two data
-      // sources.
-      if (isCompatible) {
-        val leftPartValues = leftSpec.partitioning.partitionValues
-        val rightPartValues = rightSpec.partitioning.partitionValues
-
-        logInfo(
-          s"""
-             |Left side # of partitions: ${leftPartValues.size}
-             |Right side # of partitions: ${rightPartValues.size}
-             |""".stripMargin)
-
-        // As partition keys are compatible, we can pick either left or right as partition
-        // expressions
-        val partitionExprs = leftSpec.partitioning.expressions
-
-        var mergedPartValues = InternalRowComparableWrapper
+
+    if (specsAllExist) {

Review Comment:
   Instead of always shuffle the other non-key grouped partitioning side according to the key grouped side, I wonder whether we can leverage the existing logic in `EnsureRequirements` which considers the trade-off between shuffle parallelism and cost.
   
   For instance, if one side of the join is using hash partitioning with 200 partitions, while the other side is using key grouped partitioning with 5 partitions, it's probably better to shuffle the key grouped partitioning side into 200 partitions, to ensure the job has good parallelism.
   
   Maybe we can check if it's possible to make `KeyGroupedPartitioning.canCreatePartitioning` to return true if `spark.sql.sources.v2.bucketing.shuffleOneSide.enabled` is enabled. 



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291967573


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   ~~Oh, I missed this, let me move it into `spark-catalyst`~~



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1290223195


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala:
##########
@@ -299,6 +301,12 @@ object ShuffleExchangeExec {
           ascending = true,
           samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
       case SinglePartition => new ConstantPartitioner
+      case KeyGroupedPartitioning(expressions, n, partitionValues) =>

Review Comment:
   I updated one case for this, please check is that ok?



-- 
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] szehon-ho commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1294086708


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala:
##########
@@ -299,6 +301,12 @@ object ShuffleExchangeExec {
           ascending = true,
           samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
       case SinglePartition => new ConstantPartitioner
+      case KeyGroupedPartitioning(expressions, n, partitionValues) =>
+        val partitionValueMap = mutable.Map[Seq[Any], Int]()
+        partitionValues.zipWithIndex.foreach(partAndIndex => {

Review Comment:
   Suggestion: would this be slightly easier to read ?
   
   ```
         case KeyGroupedPartitioning(expressions, n, partitionValues) =>
           val valueMap = partitionValues.zipWithIndex.map{
             case (partition, index) => (partition.toSeq(expressions.map(_.dataType)), index)
           }.toMap
           new KeyGroupedPartitioner(mutable.Map(valueMap.toSeq: _*), n)
   ```



-- 
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 #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1691918522

   Merged to master, thanks @Hisoka-X !


-- 
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 #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1679649561

   Yes good point @szehon-ho . If `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` and `spark.sql.sources.v2.bucketing.pushPartValues.enabled` are both turned on, this may not work since the data on the hash partitioning side is shuffled according to the partition values before the grouping, which contain duplicates.
   
   Perhaps we should use `KeyGroupedPartitioning.uniquePartitionValues` when computing `valueMap`.


-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291826067


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1500,6 +1500,16 @@ object SQLConf {
       .booleanConf
       .createWithDefault(false)
 
+ val V2_BUCKETING_SHUFFLE_ONE_SIDE_ENABLED =
+    buildConf("spark.sql.sources.v2.bucketing.shuffleOneSide.enabled")

Review Comment:
   nit: maybe just `spark.sql.sources.v2.bucketing.shuffle.enabled`? `shuffleOneSide` is a bit hard to understand.



-- 
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] Hisoka-X commented on pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1692550630

   Thanks @sunchao and @szehon-ho !


-- 
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] szehon-ho commented on pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1679310263

   >  maybe we should use partitionValue which after group partition splits.
   
   Thanks, I think that would be great if we can somehow !
   


-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1305073226


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1500,6 +1500,16 @@ object SQLConf {
       .booleanConf
       .createWithDefault(false)
 
+ val V2_BUCKETING_SHUFFLE_ENABLED =
+    buildConf("spark.sql.sources.v2.bucketing.shuffle.enabled")
+      .doc("During a storage-partitioned join, whether to allow to shuffle only one side." +
+        "When only one side is KeyGroupedPartitioning, if the conditions are met, spark will " +
+        "only shuffle the other side. This optimization will reduce the amount of data that " +

Review Comment:
   shall we make the algorithm smarter? If the other side is large, doing a `KeyGroupedPartitioning` may lead to skew and it's still better to shuffle both sides with hash partitioning.
   
   Let's think of an extreme case: one side reports `KeyGroupedPartitioning` with only one partition, with this optimization, we end up with doing the join using a single thread.



-- 
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] Hisoka-X commented on pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1654872993

   cc @sunchao @cloud-fan 


-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1289574749


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class KeyGroupedPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   done



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291988730


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   At now `InternalRowComparableWrapper` not supported `Serializable` , so can't use it as `KeyGroupedPartitioner` fields. This is the only way until there is a better solution.
   
   ```scala
   Serialization stack:
   	- object not serializable (class: org.apache.spark.sql.catalyst.expressions.GeneratedClass, value: org.apache.spark.sql.catalyst.expressions.GeneratedClass@560b0bb7)
   	- field (class: org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering, name: this$0, type: class org.apache.spark.sql.catalyst.expressions.GeneratedClass)
   	- object (class org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering, org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificOrdering@1eff706)
   	- field (class: org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper, name: ordering, type: class org.apache.spark.sql.catalyst.expressions.BaseOrdering)
   ```



-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1292353870


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   Ah OK, let's potentially do it as a follow-up 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] sunchao commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1303195440


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,21 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map.
+ * The valueMap is a map that contains tuples of (partition value, partition id). It generated
+ * by [[org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning]], used to partition
+ * another side to make sure records with same partition value are in same partition.

Review Comment:
   nit: 
   - "another side" -> "the other side of a join"?
   - "in same partition" -> "in the same 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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1302282211


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala:
##########
@@ -182,7 +182,16 @@ case class BatchScanExec(
 
               // Now fill missing partition keys with empty partitions
               val partitionMapping = nestGroupedPartitions.toMap
-              finalPartitions = spjParams.commonPartitionValues.get.flatMap {
+
+              // SPARK-41471: We keep to order of partition keys in `commonPartitionValues` to

Review Comment:
   I think partition values should be sorted by themselves but just `mergePartitions` in `EnsureRequirements` doesn't maintain the ordering. 
   
   Instead of doing it here, can we update `InternalRowComparableWrapper.mergePartitions` to make sure the result is ordered?



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291964718


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1500,6 +1500,16 @@ object SQLConf {
       .booleanConf
       .createWithDefault(false)
 
+ val V2_BUCKETING_SHUFFLE_ONE_SIDE_ENABLED =
+    buildConf("spark.sql.sources.v2.bucketing.shuffleOneSide.enabled")

Review Comment:
   Done



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291967573


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   Oh, I missed this, let me move it into `spark-catalyst`



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1288037738


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala:
##########
@@ -368,150 +368,197 @@ case class EnsureRequirements(
     var newLeft = left
     var newRight = right
 
-    val specs = Seq(left, right).zip(requiredChildDistribution).map { case (p, d) =>
+    val specsOpts = Seq(left, right).zip(requiredChildDistribution).map { case (p, d) =>
       if (!d.isInstanceOf[ClusteredDistribution]) return None
       val cd = d.asInstanceOf[ClusteredDistribution]
       val specOpt = createKeyGroupedShuffleSpec(p.outputPartitioning, cd)
-      if (specOpt.isEmpty) return None
-      specOpt.get
+      specOpt
+    }
+    val specsAllExist = specsOpts.forall(_.nonEmpty)
+    if ((!conf.v2BucketingShuffleOneSideEnabled && !specsAllExist)
+      || specsOpts.count(_.isEmpty) > 1) {
+      return None
     }
-
-    val leftSpec = specs.head
-    val rightSpec = specs(1)
 
     var isCompatible = false
-    if (!conf.v2BucketingPushPartValuesEnabled) {
-      isCompatible = leftSpec.isCompatibleWith(rightSpec)
-    } else {
-      logInfo("Pushing common partition values for storage-partitioned join")
-      isCompatible = leftSpec.areKeysCompatible(rightSpec)
-
-      // Partition expressions are compatible. Regardless of whether partition values
-      // match from both sides of children, we can calculate a superset of partition values and
-      // push-down to respective data sources so they can adjust their output partitioning by
-      // filling missing partition keys with empty partitions. As result, we can still avoid
-      // shuffle.
-      //
-      // For instance, if two sides of a join have partition expressions
-      // `day(a)` and `day(b)` respectively
-      // (the join query could be `SELECT ... FROM t1 JOIN t2 on t1.a = t2.b`), but
-      // with different partition values:
-      //   `day(a)`: [0, 1]
-      //   `day(b)`: [1, 2, 3]
-      // Following the case 2 above, we don't have to shuffle both sides, but instead can
-      // just push the common set of partition values: `[0, 1, 2, 3]` down to the two data
-      // sources.
-      if (isCompatible) {
-        val leftPartValues = leftSpec.partitioning.partitionValues
-        val rightPartValues = rightSpec.partitioning.partitionValues
-
-        logInfo(
-          s"""
-             |Left side # of partitions: ${leftPartValues.size}
-             |Right side # of partitions: ${rightPartValues.size}
-             |""".stripMargin)
-
-        // As partition keys are compatible, we can pick either left or right as partition
-        // expressions
-        val partitionExprs = leftSpec.partitioning.expressions
-
-        var mergedPartValues = InternalRowComparableWrapper
+
+    if (specsAllExist) {

Review Comment:
   Ohhhhh, Cool. The code more clear now.



-- 
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 #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1664609291

   Sorry for the delay. I'll take a look at this in the next 1-2 days.


-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1294230219


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala:
##########
@@ -299,6 +301,12 @@ object ShuffleExchangeExec {
           ascending = true,
           samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
       case SinglePartition => new ConstantPartitioner
+      case KeyGroupedPartitioning(expressions, n, partitionValues) =>
+        val partitionValueMap = mutable.Map[Seq[Any], Int]()
+        partitionValues.zipWithIndex.foreach(partAndIndex => {

Review Comment:
   Done. Thanks for advise.



-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1305077969


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1500,6 +1500,16 @@ object SQLConf {
       .booleanConf
       .createWithDefault(false)
 
+ val V2_BUCKETING_SHUFFLE_ENABLED =
+    buildConf("spark.sql.sources.v2.bucketing.shuffle.enabled")
+      .doc("During a storage-partitioned join, whether to allow to shuffle only one side." +
+        "When only one side is KeyGroupedPartitioning, if the conditions are met, spark will " +
+        "only shuffle the other side. This optimization will reduce the amount of data that " +

Review Comment:
   I think the `ShuffleSpec` "framework" in `EnsureRequirements` already takes this into consideration. This PR mainly makes `KeyGroupedShuffleSpec` behaves similar to `HashShuffleSpec` and be able to shuffle the other side (via making `canCreatePartitioning` return true).



-- 
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 diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1288949568


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class KeyGroupedPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   nit: can we add some comments on what `valueMap` is? 



##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],
+    override val numPartitions: Int) extends Partitioner {
+  override def getPartition(key: Any): Int = {
+    val keys = key.asInstanceOf[Seq[Any]]
+    valueMap.getOrElseUpdate(keys, Utils.nonNegativeMod(keys.hashCode, numPartitions))

Review Comment:
   I wonder what will happen if we have the following in the `valueMap`:
   ```
   key1 -> 0
   key2 -> 1
   key3 -> 2
   ```
   
   Now we have a incoming `key4` that is not the same as any of `key[1-3]`. This means we have a row on one side that doesn't belong to any partition on the other side. In this case, the current logic is we just take the hash and randomly pick an existing partition between 1 and 3 for the row.
   
   This should work for inner joins, and maybe OK for outer joins too (although I think we should check closely on this). Could you add some unit tests to cover this scenario?
   



##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   I see. I guess it also makes sense to keep `KeyGroupedPartitioner` in `spark-catalyst` or `spark-sql`, since the concept of `KeyGroupedPartitioning` only applies to Spark SQL.



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1287859433


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],
+    override val numPartitions: Int) extends Partitioner {
+  override def getPartition(key: Any): Int = {
+    val keys = key.asInstanceOf[Seq[Any]]
+    valueMap.getOrElseUpdate(keys, Utils.nonNegativeMod(keys.hashCode, numPartitions))

Review Comment:
   > When do we need to update this map?
   
   When the shuffle side (without KeyGroupedPartitioning side) have some partition value not in KeyGroupedPartitioning. This is a fallback operation to ensure that all data will be shuffled. In fact we can add Filter to avoid this kind data to be shuffled. We can do it later.
   
   > will this collide with one of the existing mappings?
   
   no, just update not existed key.



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1287899742


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   not easy, `InternalRowComparableWrapper` in spark-catalyst, but partitioner in spark-core. If do this, we should do some code move.



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1291967573


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,18 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map
+ */
+private[spark] class PartitionValueMapPartitioner(
+    valueMap: mutable.Map[Seq[Any], Int],

Review Comment:
   Oh, I missed this, let me move it into `spark-sql`



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1303647273


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,21 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map.

Review Comment:
   Thanks @sunchao ! Adressed all.



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1302357936


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala:
##########
@@ -182,7 +182,16 @@ case class BatchScanExec(
 
               // Now fill missing partition keys with empty partitions
               val partitionMapping = nestGroupedPartitions.toMap
-              finalPartitions = spjParams.commonPartitionValues.get.flatMap {
+
+              // SPARK-41471: We keep to order of partition keys in `commonPartitionValues` to

Review Comment:
   Thanks for advise! Adressed.



-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1288036864


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala:
##########
@@ -519,6 +519,35 @@ case class BroadcastPartitioning(mode: BroadcastMode) extends Partitioning {
   }
 }
 
+/**
+ * Represents a partitioning use partition value map to partition.
+ */
+case class PartitionValueMapPartitioning(

Review Comment:
   good point. Done



-- 
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] szehon-ho commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1294086708


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala:
##########
@@ -299,6 +301,12 @@ object ShuffleExchangeExec {
           ascending = true,
           samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition)
       case SinglePartition => new ConstantPartitioner
+      case KeyGroupedPartitioning(expressions, n, partitionValues) =>
+        val partitionValueMap = mutable.Map[Seq[Any], Int]()
+        partitionValues.zipWithIndex.foreach(partAndIndex => {

Review Comment:
   Suggestion: would this be slightly easier to read ?
   
   ```
         case KeyGroupedPartitioning(expressions, n, partitionValues) =>
           val newMap = partitionValues.zipWithIndex.map{
             case (partition, index) => (partition.toSeq(expressions.map(_.dataType)), index)
           }.toMap
           new KeyGroupedPartitioner(mutable.Map(newMap.toSeq: _*), n)
   ```



-- 
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] Hisoka-X commented on pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on PR #42194:
URL: https://github.com/apache/spark/pull/42194#issuecomment-1678670922

   > I'm wondering, will this work if other SPJ flags: spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled and spark.sql.sources.v2.bucketing.pushPartValues.enabled are set to true, and we have case of multiple split for a partition?
   > 
   > In that case, it seems the BatchScanExec of the side with KeyGroupedPartitioning will group partition splits, will it make it out-of-sync with the other side using KeyGroupedPartitioner?
   > 
   > It may also be hard to work with #42306 which uses a similar mechanism to group partition splits?
   
   This is a problem, let me add a test case for this, maybe we should use partitionValue which after group partition splits. Thanks for point that, cc @sunchao 


-- 
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] Hisoka-X commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1298176014


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala:
##########
@@ -182,7 +182,16 @@ case class BatchScanExec(
 
               // Now fill missing partition keys with empty partitions
               val partitionMapping = nestGroupedPartitions.toMap
-              finalPartitions = spjParams.commonPartitionValues.get.flatMap {
+
+              // SPARK-41471: We keep to order of partition keys in `commonPartitionValues` to

Review Comment:
   @sunchao @szehon-ho I let `BatchScanExec` finalPartitions with ordered when `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` and `spark.sql.sources.v2.bucketing.pushPartValues.enabled `are both turned on, to make sure to `finalPartitions` match with `KeyGroupedPartitioning.uniquePartitionValues` when use `spark.sql.sources.v2.bucketing.shuffle.enabled`. And we use `uniquePartitionValues` to generate `valueMap`, I think this change will solve problem. PTAL. 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] sunchao commented on a diff in pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #42194:
URL: https://github.com/apache/spark/pull/42194#discussion_r1303195471


##########
core/src/main/scala/org/apache/spark/Partitioner.scala:
##########
@@ -137,6 +137,21 @@ private[spark] class PartitionIdPassthrough(override val numPartitions: Int) ext
   override def getPartition(key: Any): Int = key.asInstanceOf[Int]
 }
 
+/**
+ * A [[org.apache.spark.Partitioner]] that partitions all records use partition value map.
+ * The valueMap is a map that contains tuples of (partition value, partition id). It generated

Review Comment:
   nit:
   - The `valueMap` (back-quoted to make it monospaced)
   - "It generated" -> "It is generated"



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/exchange/EnsureRequirementsSuite.scala:
##########
@@ -1109,6 +1111,32 @@ class EnsureRequirementsSuite extends SharedSparkSession {
     }
   }
 
+  test(s"SPARK-41471: shuffle right side when" +

Review Comment:
   nit: there is no need to use `s` here, same for the line below.



-- 
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 closed pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao closed pull request #42194: [SPARK-41471][SQL] Reduce Spark shuffle when only one side of a join is KeyGroupedPartitioning
URL: https://github.com/apache/spark/pull/42194


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