-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-56903][SQL] Spread NULL outer join keys across shuffle partitions #55927
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
aa7138a
7fe801c
6c9aa3b
7c760ec
400cfbe
cbca318
9eb4ae4
4a021b6
59819f4
ad70bf1
c0dc4d6
f551dab
76bbb03
e778497
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -86,7 +86,8 @@ case class ClusteredDistribution( | |
| clustering: Seq[Expression], | ||
| requireAllClusterKeys: Boolean = SQLConf.get.getConf( | ||
| SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION), | ||
| requiredNumPartitions: Option[Int] = None) extends Distribution { | ||
| requiredNumPartitions: Option[Int] = None, | ||
| allowNullKeySpreading: Boolean = false) extends Distribution { | ||
| require( | ||
| clustering != Nil, | ||
| "The clustering expressions of a ClusteredDistribution should not be Nil. " + | ||
|
|
@@ -97,7 +98,11 @@ case class ClusteredDistribution( | |
| assert(requiredNumPartitions.isEmpty || requiredNumPartitions.get == numPartitions, | ||
| s"This ClusteredDistribution requires ${requiredNumPartitions.get} partitions, but " + | ||
| s"the actual number of partitions is $numPartitions.") | ||
| HashPartitioning(clustering, numPartitions) | ||
| if (allowNullKeySpreading) { | ||
| NullAwareHashPartitioning(clustering, numPartitions) | ||
| } else { | ||
| HashPartitioning(clustering, numPartitions) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -282,7 +287,7 @@ trait HashPartitioningLike extends Expression with Partitioning with Unevaluable | |
| expressions.length == h.expressions.length && expressions.zip(h.expressions).forall { | ||
| case (l, r) => l.semanticEquals(r) | ||
| } | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _, _) => | ||
| if (requireAllClusterKeys) { | ||
| // Checks `HashPartitioning` is partitioned on exactly same clustering keys of | ||
| // `ClusteredDistribution`. | ||
|
|
@@ -324,6 +329,46 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) | |
| newChildren: IndexedSeq[Expression]): HashPartitioning = copy(expressions = newChildren) | ||
| } | ||
|
|
||
| /** | ||
| * Represents a hash partitioning for equi-join inputs where rows with a NULL join key do not need | ||
| * to be co-located. Non-NULL join keys preserve the same partitioning contract as | ||
| * [[HashPartitioning]], while rows with any NULL join key may be spread across partitions. | ||
| */ | ||
| case class NullAwareHashPartitioning(expressions: Seq[Expression], numPartitions: Int) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Design alternative worth considering: a The marker this carries is one bit ("NULL keys may be spread, so I don't deliver strict same-key co-location"). Encoding it as a parallel type means duplicating With a flag:
The one argument for distinct types is EXPLAIN-string visibility — a one-line Separately on this class's Scaladoc: worth calling out that
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea this is an alternative design. The pros and cons are: Pros:
Cons:
I'm a bit concerned about the cons since |
||
| extends HashPartitioningLike { | ||
|
|
||
| override def satisfies0(required: Distribution): Boolean = { | ||
| (required match { | ||
| case UnspecifiedDistribution => true | ||
| case AllTuples => numPartitions == 1 | ||
| case _ => false | ||
| }) || { | ||
| required match { | ||
| case c @ ClusteredDistribution( | ||
| requiredClustering, requireAllClusterKeys, _, allowNullKeySpreading) | ||
| if allowNullKeySpreading => | ||
| if (requireAllClusterKeys) { | ||
| c.areAllClusterKeysMatched(expressions) | ||
| } else { | ||
| expressions.forall(x => requiredClustering.exists(_.semanticEquals(x))) | ||
| } | ||
| case _ => false | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = | ||
| NullAwareHashShuffleSpec(this, distribution) | ||
|
|
||
| def partitionIdExpression: Expression = Pmod( | ||
| new CollationAwareMurmur3Hash(expressions), Literal(numPartitions) | ||
| ) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. After the single-eval refactor in
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch. Removed |
||
|
|
||
| override protected def withNewChildrenInternal( | ||
| newChildren: IndexedSeq[Expression]): NullAwareHashPartitioning = | ||
| copy(expressions = newChildren) | ||
| } | ||
|
|
||
| case class CoalescedBoundary(startReducerIndex: Int, endReducerIndex: Int) | ||
|
|
||
| /** | ||
|
|
@@ -345,6 +390,42 @@ case class CoalescedHashPartitioning(from: HashPartitioning, partitions: Seq[Coa | |
| copy(from = from.copy(expressions = newChildren)) | ||
| } | ||
|
|
||
| case class CoalescedNullAwareHashPartitioning( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing Scaladoc here (and on
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added |
||
| from: NullAwareHashPartitioning, | ||
| partitions: Seq[CoalescedBoundary]) extends HashPartitioningLike { | ||
|
|
||
| override def expressions: Seq[Expression] = from.expressions | ||
|
|
||
| override def satisfies0(required: Distribution): Boolean = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This body is identical to Two cleaner shapes:
Side note: both overrides skip the |
||
| (required match { | ||
| case UnspecifiedDistribution => true | ||
| case AllTuples => numPartitions == 1 | ||
| case _ => false | ||
| }) || { | ||
| required match { | ||
| case c @ ClusteredDistribution( | ||
| requiredClustering, requireAllClusterKeys, _, allowNullKeySpreading) | ||
| if allowNullKeySpreading => | ||
| if (requireAllClusterKeys) { | ||
| c.areAllClusterKeysMatched(expressions) | ||
| } else { | ||
| expressions.forall(x => requiredClustering.exists(_.semanticEquals(x))) | ||
| } | ||
| case _ => false | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def createShuffleSpec(distribution: ClusteredDistribution): ShuffleSpec = | ||
| CoalescedHashShuffleSpec(from.createShuffleSpec(distribution), partitions) | ||
|
|
||
| override val numPartitions: Int = partitions.length | ||
|
|
||
| override protected def withNewChildrenInternal( | ||
| newChildren: IndexedSeq[Expression]): CoalescedNullAwareHashPartitioning = | ||
| copy(from = from.copy(expressions = newChildren)) | ||
| } | ||
|
|
||
| /** | ||
| * Represents a partitioning where rows are split across partitions based on transforms defined by | ||
| * `expressions`. | ||
|
|
@@ -482,7 +563,7 @@ case class KeyedPartitioning( | |
|
|
||
| def groupedSatisfies(required: Distribution): Boolean = { | ||
| required match { | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _, _) => | ||
| if (requireAllClusterKeys) { | ||
| // Checks whether this partitioning is partitioned on exactly same clustering keys of | ||
| // `ClusteredDistribution`. | ||
|
|
@@ -657,7 +738,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) | |
| // `RangePartitioning(a, b, c)` satisfies `OrderedDistribution(a, b)`. | ||
| val minSize = Seq(requiredOrdering.size, ordering.size).min | ||
| requiredOrdering.take(minSize) == ordering.take(minSize) | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _, _) => | ||
| val expressions = ordering.map(_.child) | ||
| if (requireAllClusterKeys) { | ||
| // Checks `RangePartitioning` is partitioned on exactly same clustering keys of | ||
|
|
@@ -782,7 +863,7 @@ case class ShufflePartitionIdPassThrough( | |
| super.satisfies0(required) || { | ||
| required match { | ||
| // TODO(SPARK-53428): Support Direct Passthrough Partitioning in the Streaming Joins | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _) => | ||
| case c @ ClusteredDistribution(requiredClustering, requireAllClusterKeys, _, _) => | ||
| val partitioningExpressions = expr.child :: Nil | ||
| if (requireAllClusterKeys) { | ||
| c.areAllClusterKeysMatched(partitioningExpressions) | ||
|
|
@@ -903,6 +984,16 @@ case class HashShuffleSpec( | |
| left.intersect(right).nonEmpty | ||
| } | ||
| } | ||
| case otherNullAwareSpec @ NullAwareHashShuffleSpec(otherPartitioning, otherDistribution) | ||
| if distribution.allowNullKeySpreading && otherDistribution.allowNullKeySpreading => | ||
| distribution.clustering.length == otherDistribution.clustering.length && | ||
| partitioning.numPartitions == otherPartitioning.numPartitions && | ||
| partitioning.expressions.length == otherPartitioning.expressions.length && { | ||
| val otherHashKeyPositions = otherNullAwareSpec.hashKeyPositions | ||
| hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, right) => | ||
| left.intersect(right).nonEmpty | ||
| } | ||
| } | ||
| case ShuffleSpecCollection(specs) => | ||
| specs.exists(isCompatibleWith) | ||
| case _ => | ||
|
|
@@ -923,7 +1014,67 @@ case class HashShuffleSpec( | |
|
|
||
| override def createPartitioning(clustering: Seq[Expression]): Partitioning = { | ||
| val exprs = hashKeyPositions.map(v => clustering(v.head)) | ||
| HashPartitioning(exprs, partitioning.numPartitions) | ||
| if (distribution.allowNullKeySpreading) { | ||
| NullAwareHashPartitioning(exprs, partitioning.numPartitions) | ||
| } else { | ||
| HashPartitioning(exprs, partitioning.numPartitions) | ||
| } | ||
| } | ||
|
|
||
| override def numPartitions: Int = partitioning.numPartitions | ||
| } | ||
|
|
||
| case class NullAwareHashShuffleSpec( | ||
| partitioning: NullAwareHashPartitioning, | ||
| distribution: ClusteredDistribution) extends ShuffleSpec { | ||
|
|
||
| lazy val hashKeyPositions: 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(k => distKeyToPos.getOrElse(k.canonicalized, mutable.BitSet.empty)) | ||
| } | ||
|
|
||
| override def isCompatibleWith(other: ShuffleSpec): Boolean = other match { | ||
|
peter-toth marked this conversation as resolved.
|
||
| case SinglePartitionShuffleSpec => | ||
| partitioning.numPartitions == 1 | ||
| case otherSpec @ NullAwareHashShuffleSpec(otherPartitioning, otherDistribution) => | ||
| distribution.clustering.length == otherDistribution.clustering.length && | ||
| partitioning.numPartitions == otherPartitioning.numPartitions && | ||
| partitioning.expressions.length == otherPartitioning.expressions.length && { | ||
| val otherHashKeyPositions = otherSpec.hashKeyPositions | ||
| hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, right) => | ||
| left.intersect(right).nonEmpty | ||
| } | ||
| } | ||
| case otherHashSpec @ HashShuffleSpec(otherPartitioning, otherDistribution) | ||
| if distribution.allowNullKeySpreading && otherDistribution.allowNullKeySpreading => | ||
| distribution.clustering.length == otherDistribution.clustering.length && | ||
| partitioning.numPartitions == otherPartitioning.numPartitions && | ||
| partitioning.expressions.length == otherPartitioning.expressions.length && { | ||
| val otherHashKeyPositions = otherHashSpec.hashKeyPositions | ||
| hashKeyPositions.zip(otherHashKeyPositions).forall { case (left, right) => | ||
| left.intersect(right).nonEmpty | ||
| } | ||
| } | ||
| case ShuffleSpecCollection(specs) => | ||
| specs.exists(isCompatibleWith) | ||
| case _ => | ||
| false | ||
| } | ||
|
|
||
| override def canCreatePartitioning: Boolean = { | ||
| if (SQLConf.get.getConf(SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION)) { | ||
| distribution.areAllClusterKeysMatched(partitioning.expressions) | ||
| } else { | ||
| true | ||
| } | ||
| } | ||
|
|
||
| override def createPartitioning(clustering: Seq[Expression]): Partitioning = { | ||
| val exprs = hashKeyPositions.map(v => clustering(v.head)) | ||
| NullAwareHashPartitioning(exprs, partitioning.numPartitions) | ||
| } | ||
|
|
||
| override def numPartitions: Int = partitioning.numPartitions | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -241,6 +241,11 @@ case class ShuffleExchangeExec( | |
| */ | ||
| @transient | ||
| lazy val shuffleDependency : ShuffleDependency[Int, InternalRow, InternalRow] = { | ||
| outputPartitioning match { | ||
|
peter-toth marked this conversation as resolved.
Outdated
|
||
| case h: NullAwareHashPartitioning => | ||
| logWarning(s"Materializing null-aware hash shuffle with ${h.numPartitions} partitions.") | ||
| case _ => | ||
| } | ||
| // Wrap in the exchange's RDD scope so that any wrapper RDDs created during shuffle dependency | ||
| // preparation (e.g. by prepareShuffleDependency's mapPartitionsInternal calls) get this | ||
| // exchange's scope ID. | ||
|
|
@@ -349,6 +354,8 @@ object ShuffleExchangeExec { | |
| // For HashPartitioning, the partitioning key is already a valid partition ID, as we use | ||
| // `HashPartitioning.partitionIdExpression` to produce partitioning key. | ||
| new PartitionIdPassthrough(n) | ||
| case NullAwareHashPartitioning(_, n) => | ||
| new PartitionIdPassthrough(n) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: the parallel
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point. Added the comment above in the code |
||
| case ShufflePartitionIdPassThrough(_, n) => | ||
| // For ShufflePartitionIdPassThrough, the DirectShufflePartitionID expression directly | ||
| // produces partition IDs, so we use PartitionIdPassthrough to pass them through directly. | ||
|
|
@@ -403,6 +410,24 @@ object ShuffleExchangeExec { | |
| case h: HashPartitioning => | ||
| val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) | ||
| row => projection(row).getInt(0) | ||
| case h: NullAwareHashPartitioning => | ||
| val partitionIdProjection = | ||
| UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) | ||
| val joinKeyProjection = UnsafeProjection.create(h.expressions, outputAttributes) | ||
| var nullKeyPartition = | ||
| new XORShiftRandom(TaskContext.get().partitionId()).nextInt(h.numPartitions) | ||
|
peter-toth marked this conversation as resolved.
|
||
| row => { | ||
| val joinKeys = joinKeyProjection(row) | ||
| if (joinKeys.anyNull()) { | ||
| // NULL join keys cannot match under ordinary equi-join semantics. Spread them | ||
| // round-robin within each map task so identical rows do not collapse to one reducer. | ||
| val partition = nullKeyPartition | ||
| nullKeyPartition = (nullKeyPartition + 1) % h.numPartitions | ||
| partition | ||
| } else { | ||
| partitionIdProjection(row).getInt(0) | ||
| } | ||
| } | ||
|
Comment on lines
+412
to
+437
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Join keys are evaluated twice for non-NULL rows on this path: once via Could evaluate the keys once, check Combined with the static-nullability gate at
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I updated it to only evaluate the join keys once but the logic becomes more complicated. Please take another look! |
||
| case RangePartitioning(sortingExpressions, _) => | ||
| val projection = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) | ||
| row => projection(row) | ||
|
|
@@ -419,17 +444,22 @@ object ShuffleExchangeExec { | |
|
|
||
| val isRoundRobin = newPartitioning.isInstanceOf[RoundRobinPartitioning] && | ||
| newPartitioning.numPartitions > 1 | ||
| val isNullAwareRoundRobin = | ||
|
peter-toth marked this conversation as resolved.
Outdated
|
||
| newPartitioning.isInstanceOf[NullAwareHashPartitioning] && | ||
| newPartitioning.numPartitions > 1 | ||
| val needsDeterministicLocalSort = | ||
| (isRoundRobin || isNullAwareRoundRobin) && SQLConf.get.sortBeforeRepartition | ||
|
|
||
| val rddWithPartitionIds: RDD[Product2[Int, InternalRow]] = { | ||
| // [SPARK-23207] Have to make sure the generated RoundRobinPartitioning is deterministic, | ||
| // [SPARK-23207] Have to make sure stateful row-to-partition assignment is deterministic, | ||
| // otherwise a retry task may output different rows and thus lead to data loss. | ||
| // | ||
| // Currently we following the most straight-forward way that perform a local sort before | ||
| // partitioning. | ||
| // | ||
| // Note that we don't perform local sort if the new partitioning has only 1 partition, under | ||
| // that case all output rows go to the same partition. | ||
| val newRdd = if (isRoundRobin && SQLConf.get.sortBeforeRepartition) { | ||
| val newRdd = if (needsDeterministicLocalSort) { | ||
| rdd.mapPartitionsInternal { iter => | ||
| val recordComparatorSupplier = new Supplier[RecordComparator] { | ||
| override def get: RecordComparator = new RecordBinaryComparator() | ||
|
|
@@ -468,7 +498,9 @@ object ShuffleExchangeExec { | |
| } | ||
|
|
||
| // round-robin function is order sensitive if we don't sort the input. | ||
| val isOrderSensitive = isRoundRobin && !SQLConf.get.sortBeforeRepartition | ||
| // Stateful partition assignment is order-sensitive when it depends on row visitation order. | ||
| val isOrderSensitive = | ||
| (isRoundRobin || isNullAwareRoundRobin) && !SQLConf.get.sortBeforeRepartition | ||
| if (needToCopyObjectsBeforeShuffle(part)) { | ||
| newRdd.mapPartitionsWithIndexInternal((_, iter) => { | ||
| val getPartitionKey = getPartitionKeyExtractor() | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| package org.apache.spark.sql.execution.joins | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, IsNull} | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, LeftExistence, LeftOuter, LeftSingle, RightOuter} | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, Partitioning, PartitioningCollection, UnknownPartitioning, UnspecifiedDistribution} | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -28,6 +28,21 @@ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Dist | |||||||||||||||||||||||||||||||||||||||||||||||||||
| trait ShuffledJoin extends JoinCodegenSupport { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| def isSkewJoin: Boolean | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| private def containsNullSafeJoinMarker(keys: Seq[Expression]): Boolean = { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| keys.exists(_.exists(_.isInstanceOf[IsNull])) | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| private lazy val canSpreadNullJoinKeys: Boolean = { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Actually, why this is needed at all and when can't we spread nulls?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For most types the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Without spreading, NullType <=> keys all hash to the same value (Murmur3Hash(null) is deterministic) → all NULL rows collocate on one reducer. The executor then runs:
So even with NULL rows colocated, the executor's anyNull guard prevents NULL=NULL from matching. The <=> semantics the user wanted (NULL matches NULL) is never delivered for NullType — the rewrite was supposed to convert NULLs With spreading, NULL rows scatter across reducers. Each reducer's executor sees some NULL rows from both sides. The anyNull guard fires the same way. Same padding emission, same lack of matching. Output is identical with or without spreading — both produce the broken-but-self-consistent "NULL=NULL doesn't match" behavior for NullType.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm that is a good point. It seems the check is indeed unnecessary then, let me remove it.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please don't forget to update the PR description and let's leave some comments here why spreading nulls is safe in <=> outer joins.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updated the PR description. As for left anti join, yes, ordinary shuffled left anti equi-joins with
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks. Sure, handling outer joins in this PR is a nice improvement.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The gate opts in based on join type alone, ignoring whether the shuffle keys are actually nullable. For an outer join on non-nullable keys (e.g.
Two options worth considering:
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point. Updated |
||||||||||||||||||||||||||||||||||||||||||||||||||||
| val isOuterJoin = joinType == LeftOuter || joinType == RightOuter || joinType == FullOuter | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| val canSpread = isOuterJoin && | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| !containsNullSafeJoinMarker(leftKeys) && | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| !containsNullSafeJoinMarker(rightKeys) | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| if (canSpread) { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| logWarning(s"Using null-aware shuffle distribution for $joinType equi-join keys.") | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| canSpread | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
Comment on lines
+32
to
+48
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Two improvements on this gate: (1) Static nullability check. Outer joins on non-nullable keys (PK/FK / NOT-NULL columns / post- (2) Reframe the comment around the structural reason. The current comment only addresses the
Suggested change
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updated |
||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| override def nodeName: String = { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| if (isSkewJoin) super.nodeName + "(skew=true)" else super.nodeName | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -39,6 +54,9 @@ trait ShuffledJoin extends JoinCodegenSupport { | |||||||||||||||||||||||||||||||||||||||||||||||||||
| // We re-arrange the shuffle partitions to deal with skew join, and the new children | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| // partitioning doesn't satisfy `ClusteredDistribution`. | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| UnspecifiedDistribution :: UnspecifiedDistribution :: Nil | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } else if (canSpreadNullJoinKeys) { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| ClusteredDistribution(leftKeys, allowNullKeySpreading = true) :: | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
peter-toth marked this conversation as resolved.
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
| ClusteredDistribution(rightKeys, allowNullKeySpreading = true) :: Nil | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } else { | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil | ||||||||||||||||||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Worth a Scaladoc on this field describing the contract: it's a permission, not a requirement (an ordinary
HashPartitioningstill satisfies this distribution when the flag istrue; the flag only weakens what the default partitioning produced bycreatePartitioninglooks like). And it's the consumer-side knob — the partitioning-side marker (NullAwareHashPartitioningtoday, or a flag onHashPartitioningper the comment below) is what tells downstream operators they need to re-shuffle for strictClusteredDistribution.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added