-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-11982] [SQL] improve performance of cartesian product #9969
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 all commits
2fb7a1c
162268c
0f5d7ba
951fe7a
3a66c89
a94204b
37b3088
074f2a7
99bb8ef
d3edd4f
d88fa69
fbd7dfd
91c7824
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 |
|---|---|---|
|
|
@@ -17,16 +17,75 @@ | |
|
|
||
| package org.apache.spark.sql.execution.joins | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark._ | ||
| import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} | ||
| import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} | ||
| import org.apache.spark.sql.execution.metric.SQLMetrics | ||
| import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} | ||
| import org.apache.spark.util.CompletionIterator | ||
| import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter | ||
|
|
||
|
|
||
| /** | ||
| * An optimized CartesianRDD for UnsafeRow, which will cache the rows from second child RDD, | ||
| * will be much faster than building the right partition for every row in left RDD, it also | ||
| * materialize the right RDD (in case of the right RDD is nondeterministic). | ||
| */ | ||
| private[spark] | ||
| class UnsafeCartesianRDD(left : RDD[UnsafeRow], right : RDD[UnsafeRow], numFieldsOfRight: Int) | ||
| extends CartesianRDD[UnsafeRow, UnsafeRow](left.sparkContext, left, right) { | ||
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[(UnsafeRow, UnsafeRow)] = { | ||
| // We will not sort the rows, so prefixComparator and recordComparator are null. | ||
| val sorter = UnsafeExternalSorter.create( | ||
| context.taskMemoryManager(), | ||
| SparkEnv.get.blockManager, | ||
| context, | ||
| null, | ||
| null, | ||
| 1024, | ||
| SparkEnv.get.memoryManager.pageSizeBytes) | ||
|
|
||
| val partition = split.asInstanceOf[CartesianPartition] | ||
| for (y <- rdd2.iterator(partition.s2, context)) { | ||
| sorter.insertRecord(y.getBaseObject, y.getBaseOffset, y.getSizeInBytes, 0) | ||
| } | ||
|
|
||
| // Create an iterator from sorter and wrapper it as Iterator[UnsafeRow] | ||
| def createIter(): Iterator[UnsafeRow] = { | ||
|
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. Does it make sense to move this into sorter? Seems other calls might want this. Something like UnsafeRowSorter.unsafeRowIterator().
Contributor
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. We use
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. I see. Ok. |
||
| val iter = sorter.getIterator | ||
| val unsafeRow = new UnsafeRow | ||
| new Iterator[UnsafeRow] { | ||
| override def hasNext: Boolean = { | ||
| iter.hasNext | ||
| } | ||
| override def next(): UnsafeRow = { | ||
| iter.loadNext() | ||
| unsafeRow.pointTo(iter.getBaseObject, iter.getBaseOffset, numFieldsOfRight, | ||
| iter.getRecordLength) | ||
| unsafeRow | ||
| } | ||
| } | ||
| } | ||
|
|
||
| val resultIter = | ||
| for (x <- rdd1.iterator(partition.s1, context); | ||
| y <- createIter()) yield (x, y) | ||
|
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. does 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. and we may also need to update
Contributor
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. As we discussed it in #7417, right now it's not clear that which metric could be used as the Even the right table is larger than left, this approach is still much better than current one (building the partition is usually much expensive than loading them from memory or disk), it also fix another problem that the right table could be nondeterministic.
Contributor
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. @cloud-fan For the first question, yes. |
||
| CompletionIterator[(UnsafeRow, UnsafeRow), Iterator[(UnsafeRow, UnsafeRow)]]( | ||
| resultIter, sorter.cleanupResources) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { | ||
| override def output: Seq[Attribute] = left.output ++ right.output | ||
|
|
||
| override def canProcessSafeRows: Boolean = false | ||
| override def canProcessUnsafeRows: Boolean = true | ||
| override def outputsUnsafeRows: Boolean = true | ||
|
|
||
| override private[sql] lazy val metrics = Map( | ||
| "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"), | ||
| "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"), | ||
|
|
@@ -39,18 +98,19 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod | |
|
|
||
| val leftResults = left.execute().map { row => | ||
| numLeftRows += 1 | ||
| row.copy() | ||
| row.asInstanceOf[UnsafeRow] | ||
| } | ||
| val rightResults = right.execute().map { row => | ||
| numRightRows += 1 | ||
| row.copy() | ||
| row.asInstanceOf[UnsafeRow] | ||
| } | ||
|
|
||
| leftResults.cartesian(rightResults).mapPartitionsInternal { iter => | ||
| val joinedRow = new JoinedRow | ||
| val pair = new UnsafeCartesianRDD(leftResults, rightResults, right.output.size) | ||
| pair.mapPartitionsInternal { iter => | ||
| val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema) | ||
| iter.map { r => | ||
| numOutputRows += 1 | ||
| joinedRow(r._1, r._2) | ||
| joiner.join(r._1, r._2) | ||
| } | ||
| } | ||
| } | ||
|
|
||
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.
This doesnt work if iterators contain empty iterators. Fix or assert that can't be.
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.
It checked that the
iteratorsis not emptyThere 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.
That's not what i mean.
If iterators contains an empty one. So iterators is:
(1, 2) : empty : (3, 4)
When you move to the second iterator (current is empty) you will stop and not iterate over the iterator containing (3,4)
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.
Oh, I see, thanks, will fix it.
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.
For UnsafeExternalSorter, it's not possible to have an empty iterator in the middle, they are spilled files. It's still good to be defensive for that.
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.
Yea. I figured it would not be empty but I agree about being defensive. If the implementation of UnsafeExternalSorter changes, we don't want to debug this.