-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26312][SQL]Replace RDDConversions.rowToRowRdd with RowEncoder to improve its conversion performance #23262
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 3 commits
ddb2528
89f3191
907d3f3
56cf4e5
41937f5
e8817e3
9758534
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,51 +17,39 @@ | |||
|
|
||||
| package org.apache.spark.sql.execution | ||||
|
|
||||
| import scala.reflect.runtime.universe.TypeTag | ||||
|
|
||||
| import org.apache.spark.rdd.RDD | ||||
| import org.apache.spark.sql.{Encoder, Row, SparkSession} | ||||
| import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | ||||
| import org.apache.spark.sql.catalyst.InternalRow | ||||
| import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation | ||||
| import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} | ||||
| import org.apache.spark.sql.catalyst.expressions._ | ||||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||||
| import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} | ||||
| import org.apache.spark.sql.catalyst.util.truncatedString | ||||
| import org.apache.spark.sql.execution.metric.SQLMetrics | ||||
| import org.apache.spark.sql.types.DataType | ||||
| import org.apache.spark.sql.types.StructType | ||||
|
|
||||
| object RDDConversions { | ||||
| def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { | ||||
| def productToRowRdd[A <: Product : TypeTag](data: RDD[A], | ||||
| outputSchema: StructType): RDD[InternalRow] = { | ||||
| val converters = ExpressionEncoder[A].resolveAndBind(outputSchema.toAttributes) | ||||
| data.mapPartitions { iterator => | ||||
| val numColumns = outputTypes.length | ||||
| val mutableRow = new GenericInternalRow(numColumns) | ||||
| val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) | ||||
| iterator.map { r => | ||||
| var i = 0 | ||||
| while (i < numColumns) { | ||||
| mutableRow(i) = converters(i)(r.productElement(i)) | ||||
| i += 1 | ||||
| } | ||||
|
|
||||
| mutableRow | ||||
| converters.toRow(r) | ||||
| } | ||||
| } | ||||
| } | ||||
|
|
||||
| /** | ||||
| * Convert the objects inside Row into the types Catalyst expected. | ||||
| */ | ||||
| def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = { | ||||
| def rowToRowRdd(data: RDD[Row], outputSchema: StructType): RDD[InternalRow] = { | ||||
|
||||
| val converters = RowEncoder(outputSchema) | ||||
|
||||
| case chr: Char => UTF8String.fromString(chr.toString) |
Looks we're going to drop Char as StringType. I think it's trivial and rather a mistake that we supported this. I don't feel strongly about documenting it in migration guide but if anyone feels so, we better do 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.
nit: indent
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.
well, seems like this is never used actually... shall we remove it instead if this is the case?