The fromRow method is removed in spark3.0. And the new API is :
val encoder = RowEncoder(schema)
val row = encoder.createDeserializer().apply(internalRow)

Thanks,
Jia Ke

From: Wenchen Fan <cloud0...@gmail.com>
Sent: Friday, November 27, 2020 9:32 PM
To: Jason Jun <jaes...@gmail.com>
Cc: Spark dev list <dev@spark.apache.org>
Subject: Re: How to convert InternalRow to Row.

InternalRow is an internal/developer API that might change overtime. Right now, 
the way to convert it to Row is to use  `RowEncoder`, but you need to know the 
data schema:
val encoder = RowEncoder(schema)
val row = encoder.fromRow(internalRow)

On Fri, Nov 27, 2020 at 6:16 AM Jason Jun 
<jaes...@gmail.com<mailto:jaes...@gmail.com>> wrote:
Hi dev,

i'm working on generating custom pipeline on the fly, which means I generate 
SparkPlan along with each node in my pipeline.

So, my pipeline end up with PipeLineRelation extending BaseRelation like:

case class PipeLineRelation(schema: StructType, pipeLinePlan: 
LogicalPlan)(@transient override val sqlContext: SQLContext) extends 
BaseRelation with PrunedFilteredScan {
  override def needConversion: Boolean = true
  override def unhandledFilters(filters: Array[Filter]): Array[Filter] = filters

  override def buildScan(requiredColumns: Array[String], filters: 
Array[Filter]): RDD[Row] = {
    ...
    val sparkPlan = 
sqlContext.sparkSession.sessionState.planner.plan(pipeLinePlan).next()
    sparkPlan.execute().mapPartitions { itr =>
      itr.map { internalRow =>
        val values = prunedColumnWithIndex.map { case (index, columnType) =>
          internalRow.get(index, columnType)
        }
        Row.fromSeq(values) // Line 46
      }
    }
  }
}

I'm getting InternalRow by executing subsequent Spark Plan, and converting it 
into Row using Row.fromSeq(). i saw values at Line 46 are what i exactly want :
------
values = {Object[5]@14277}
 0 = {UTF8String@14280} "Thin"
 1 = {UTF8String@14281} "Cell phone"
 2 = {Integer@14282} 6000
 3 = {Integer@14283} 2
 4 = {Integer@14284} 12000

but execution of Line 46 ended up with this error :
------
Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most 
recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost, executor driver): 
java.lang.IllegalArgumentException: The value (2) of the type 
(java.lang.Integer) cannot be converted to the string type
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:290)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toCatalystImpl(CatalystTypeConverters.scala:285)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toCatalyst(CatalystTypeConverters.scala:103)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToCatalystConverter$2.apply(CatalystTypeConverters.scala:396)
at 
org.apache.spark.sql.execution.RDDConversions$$anonfun$rowToRowRdd$1$$anonfun$apply$3.apply(ExistingRDD.scala:60)
at 
org.apache.spark.sql.execution.RDDConversions$$anonfun$rowToRowRdd$1$$anonfun$apply$3.apply(ExistingRDD.scala:57)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:370)
at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)
at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:619)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:836)
at 
org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:836)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:121)
at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:402)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:408)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
----

Is it existing bug? otherwise how do I convert InternalRow to Row?

Thanks in advance.
Jason

Reply via email to