Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9969#discussion_r45937773
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
 ---
    @@ -17,16 +17,69 @@
     
     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
    +
    +
    +private[spark]
    +class UnsafeCartesianRDD(rdd1 : RDD[UnsafeRow], rdd2 : RDD[UnsafeRow])
    +  extends CartesianRDD[UnsafeRow, UnsafeRow](rdd1.sparkContext, rdd1, 
rdd2) {
    +
    +  override def compute(split: Partition, context: TaskContext): 
Iterator[(UnsafeRow, UnsafeRow)] = {
    +    val sorter = UnsafeExternalSorter.create(
    +      context.taskMemoryManager(),
    +      SparkEnv.get.blockManager,
    +      context,
    +      null,
    +      null,
    +      1024,
    +      SparkEnv.get.memoryManager.pageSizeBytes)
    +
    +    val currSplit = split.asInstanceOf[CartesianPartition]
    +    var numFields = 0
    +    for (y <- rdd2.iterator(currSplit.s2, context)) {
    +      numFields = y.numFields()
    +      sorter.insertRecord(y.getBaseObject, y.getBaseOffset, 
y.getSizeInBytes, 0)
    +    }
    +
    +    def createIter(): Iterator[UnsafeRow] = {
    +      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, 
numFields, iter.getRecordLength)
    +          unsafeRow
    +        }
    +      }
    +    }
    +
    +    val resultIter =
    +      for (x <- rdd1.iterator(currSplit.s1, context);
    +           y <- createIter()) yield (x, y)
    --- End diff --
    
    does the `UnsafeExternalSorter` preserve records order if it spills? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to