Hi Nasrulla, Without details of your code / configuration, it's a bit hard to tell what exactly went wrong, since there can be a lot of places that could go wrong...
But one thing for sure is that, the interpreted code path (non-WSCG) and the WSCG path are two separate things and it wouldn't surprise me that one works and the other doesn't, because you can have different features/bugs in them. Depending on which version/branch of Spark you're working with, you might need to implement columnar support slightly differently. c.f. https://github.com/apache/spark/commit/c341de8b3e1f1d3327bd4ae3b0d2ec048f64d306 Best regards, Kris -- Kris Mok Software Engineer Databricks Inc. kris....@databricks.com databricks.com <http://databricks.com/> On Fri, Jun 12, 2020 at 11:09 AM Nasrulla Khan Haris < nasrulla.k...@microsoft.com> wrote: > > > Thanks Kris for your inputs. Yes I have a new data source which wraps > around built-in parquet data source. What I do not understand is with WSCG > disabled, output is not columnar batch, if my changes do not handle > columnar support, shouldn’t the behavior remain same with or without WSCG. > > > > > > > > *From:* Kris Mo <kris....@databricks.com> > *Sent:* Friday, June 12, 2020 2:20 AM > *To:* Nasrulla Khan Haris <nasrulla.k...@microsoft.com.invalid> > *Cc:* dev@spark.apache.org > *Subject:* [EXTERNAL] Re: ColumnnarBatch to InternalRow Cast exception > with codegen enabled. > > > > Hi Nasrulla, > > > > Not sure what your new code is doing, but the symptom looks like you're > creating a new data source that wraps around the builtin Parquet data > source? > > > > The problem here is, whole-stage codegen generated code for row-based > input, but the actual input is columnar. > > In other words, in your setup, the vectorized Parquet reader is enabled > (which produces columnar output), and you probably wrote a new operator > that didn't properly interact with the columnar support, so that WSCG > thought it should generate row-based code instead of columnar code. > > > > Hope it helps, > > Kris > > -- > > > > Kris Mok > > Software Engineer Databricks Inc. > > kris....@databricks.com > > databricks.com > <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Fdatabricks.com%2F&data=02%7C01%7CNasrulla.Khan%40microsoft.com%7C4755c6eb23a245f62f8c08d80eb1da53%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637275504329646919&sdata=PkkwqVJgkcR92uhEujWmCpuMFg9gLXNzXLHwZOr%2B1bA%3D&reserved=0> > > [image: Image removed by sender.] > > <https://nam06.safelinks.protection.outlook.com/?url=http%3A%2F%2Fdatabricks.com%2F&data=02%7C01%7CNasrulla.Khan%40microsoft.com%7C4755c6eb23a245f62f8c08d80eb1da53%7C72f988bf86f141af91ab2d7cd011db47%7C1%7C0%7C637275504329656882&sdata=h4GnaCoU6Dc8DAx2boaXLKOW089%2BCZtqYsSYid0%2F22g%3D&reserved=0> > > > > > > On Thu, Jun 11, 2020 at 5:41 PM Nasrulla Khan Haris > <nasrulla.k...@microsoft.com.invalid> wrote: > > HI Spark developer, > > > > I have a new baseRelation which Initializes ParquetFileFormat object and > when reading the data I am encountering Cast Exception below, however when > I disable codegen support with config “spark.sql.codegen.wholeStage"= > false, I do not encounter this exception. > > > > > > 20/06/11 17:35:39 INFO FileScanRDD: Reading File path: file:///D:/ > jvm/src/test/scala/resources/pems_sorted/station=402260/part-r-00245-ddaee723-f3f6-4f25-a34b-3312172aa6d7.snappy.parquet, > range: 0-50936, partition values: [402260] > > 20/06/11 17:35:39 INFO CodecPool: Got brand-new decompressor [.snappy] > > 20/06/11 17:35:40 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID > 0) > > java.lang.ClassCastException: org.apache.spark.sql.vectorized.ColumnarBatch > cannot be cast to org.apache.spark.sql.catalyst.InternalRow > > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.agg_doAggregateWithKeys_0$(Unknown > Source) > > 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$13$$anon$1.hasNext(WholeStageCodegenExec.scala:636) > > at > scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409) > > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99) > > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55) > > at org.apache.spark.scheduler.Task.run(Task.scala:123) > > at > org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408) > > at > org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) > > at > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) > > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > > at java.lang.Thread.run(Thread.java:748) > > > > > > Appreciate your inputs. > > > > Thanks, > > NKH > >