Skip to content

[SPARK-16189][SQL] Add ExternalRDD logical plan for input with RDD to have a chance to eliminate serialize/deserialize. #13890

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

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 3 additions & 9 deletions sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Original file line number Diff line number Diff line change
Expand Up @@ -237,10 +237,8 @@ class SparkSession private(
@Experimental
def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = {
SparkSession.setActiveSession(this)
val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
val attributeSeq = schema.toAttributes
val rowRDD = RDDConversions.productToRowRdd(rdd, schema.map(_.dataType))
Dataset.ofRows(self, LogicalRDD(attributeSeq, rowRDD)(self))
val encoder = Encoders.product[A]
Dataset.ofRows(self, ExternalRDD(rdd, self)(encoder))
}

/**
Expand Down Expand Up @@ -425,11 +423,7 @@ class SparkSession private(
*/
@Experimental
def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = {
val enc = encoderFor[T]
val attributes = enc.schema.toAttributes
val encoded = data.map(d => enc.toRow(d))
val plan = LogicalRDD(attributes, encoded)(self)
Dataset[T](self, plan)
Dataset[T](self, ExternalRDD(data, self))
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution
import org.apache.commons.lang.StringUtils

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{AnalysisException, Row, SparkSession, SQLContext}
import org.apache.spark.sql.{AnalysisException, Encoder, Row, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource}
Expand Down Expand Up @@ -74,13 +74,71 @@ object RDDConversions {
}
}

private[sql] object ExternalRDD {

def apply[T: Encoder](rdd: RDD[T], session: SparkSession): LogicalPlan = {
val externalRdd = ExternalRDD(CatalystSerde.generateObjAttr[T], rdd)(session)
CatalystSerde.serialize[T](externalRdd)
}
}

/** Logical plan node for scanning data from an RDD. */
private[sql] case class ExternalRDD[T](
outputObjAttr: Attribute,
rdd: RDD[T])(session: SparkSession)
extends LeafNode with ObjectProducer with MultiInstanceRelation {

override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil

override def newInstance(): ExternalRDD.this.type =
ExternalRDD(outputObjAttr.newInstance(), rdd)(session).asInstanceOf[this.type]

override def sameResult(plan: LogicalPlan): Boolean = {
plan.canonicalized match {
case ExternalRDD(_, otherRDD) => rdd.id == otherRDD.id
case _ => false
}
}

override protected def stringArgs: Iterator[Any] = Iterator(output)

@transient override lazy val statistics: Statistics = Statistics(
// TODO: Instead of returning a default value here, find a way to return a meaningful size
// estimate for RDDs. See PR 1238 for more discussions.
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
)
}

/** Physical plan node for scanning data from an RDD. */
private[sql] case class ExternalRDDScanExec[T](
outputObjAttr: Attribute,
rdd: RDD[T]) extends LeafExecNode with ObjectProducerExec {

private[sql] override lazy val metrics = Map(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))

protected override def doExecute(): RDD[InternalRow] = {
val numOutputRows = longMetric("numOutputRows")
val outputDataType = outputObjAttr.dataType
rdd.mapPartitionsInternal { iter =>
val outputObject = ObjectOperator.wrapObjectToRow(outputDataType)
iter.map { value =>
numOutputRows += 1
outputObject(value)
}
}
}

override def simpleString: String = {
s"Scan $nodeName${output.mkString("[", ",", "]")}"
}
}

/** Logical plan node for scanning data from an RDD of InternalRow. */
private[sql] case class LogicalRDD(
output: Seq[Attribute],
rdd: RDD[InternalRow])(session: SparkSession)
extends LogicalPlan with MultiInstanceRelation {

override def children: Seq[LogicalPlan] = Nil
extends LeafNode with MultiInstanceRelation {

override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil

Expand All @@ -96,16 +154,14 @@ private[sql] case class LogicalRDD(

override protected def stringArgs: Iterator[Any] = Iterator(output)

override def producedAttributes: AttributeSet = outputSet

@transient override lazy val statistics: Statistics = Statistics(
// TODO: Instead of returning a default value here, find a way to return a meaningful size
// estimate for RDDs. See PR 1238 for more discussions.
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
)
}

/** Physical plan node for scanning data from an RDD. */
/** Physical plan node for scanning data from an RDD of InternalRow. */
private[sql] case class RDDScanExec(
output: Seq[Attribute],
rdd: RDD[InternalRow],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,12 @@ private[sql] case class LocalTableScanExec(
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))

private val unsafeRows: Array[InternalRow] = {
val proj = UnsafeProjection.create(output, output)
rows.map(r => proj(r).copy()).toArray
if (rows.isEmpty) {
Array.empty
} else {
val proj = UnsafeProjection.create(output, output)
rows.map(r => proj(r).copy()).toArray
}
}

private lazy val rdd = sqlContext.sparkContext.parallelize(unsafeRows)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -409,6 +409,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case logical.RepartitionByExpression(expressions, child, nPartitions) =>
exchange.ShuffleExchange(HashPartitioning(
expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil
case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil
case LogicalRDD(output, rdd) => RDDScanExec(output, rdd, "ExistingRDD") :: Nil
case BroadcastHint(child) => planLater(child) :: Nil
case _ => Nil
Expand Down
8 changes: 7 additions & 1 deletion sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.LogicalRDD
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression
import org.apache.spark.sql.execution.columnar.InMemoryRelation
import org.apache.spark.sql.execution.datasources.LogicalRelation
Expand Down Expand Up @@ -241,6 +241,12 @@ abstract class QueryTest extends PlanTest {
case _: LogicalRelation => return
case p if p.getClass.getSimpleName == "MetastoreRelation" => return
case _: MemoryPlan => return
case p: InMemoryRelation =>
p.child.transform {
case _: ObjectConsumerExec => return
case _: ObjectProducerExec => return
}
p
}.transformAllExpressions {
case a: ImperativeAggregate => return
case _: TypedAggregateExpression => return
Expand Down