Brennan Stein created SPARK-40212:
-------------------------------------

             Summary: SparkSQL castPartValue does not properly handle byte & 
short
                 Key: SPARK-40212
                 URL: https://issues.apache.org/jira/browse/SPARK-40212
             Project: Spark
          Issue Type: Bug
          Components: SQL
    Affects Versions: 3.3.0
            Reporter: Brennan Stein


Reading in a parquet file partitioned on disk by a `Byte`-type column fails 
with the following exception:

 
{code:java}
[info]   Cause: java.lang.ClassCastException: java.lang.Integer cannot be cast 
to java.lang.Byte
[info]   at scala.runtime.BoxesRunTime.unboxToByte(BoxesRunTime.java:95)
[info]   at 
org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getByte(rows.scala:39)
[info]   at 
org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getByte$(rows.scala:39)
[info]   at 
org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getByte(rows.scala:195)
[info]   at 
org.apache.spark.sql.catalyst.expressions.JoinedRow.getByte(JoinedRow.scala:86)
[info]   at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.writeFields_0_6$(Unknown
 Source)
[info]   at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
 Source)
[info]   at 
org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.$anonfun$buildReaderWithPartitionValues$8(ParquetFileFormat.scala:385)
[info]   at 
org.apache.spark.sql.execution.datasources.RecordReaderIterator$$anon$1.next(RecordReaderIterator.scala:62)
[info]   at 
org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.next(FileScanRDD.scala:189)
[info]   at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
[info]   at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown
 Source)
[info]   at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
[info]   at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
[info]   at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364)
[info]   at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)
[info]   at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)
[info]   at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
[info]   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
[info]   at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
[info]   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
[info]   at org.apache.spark.scheduler.Task.run(Task.scala:136)
[info]   at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
[info]   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
[info]   at 
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
[info]   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:748) {code}
I believe the issue to stem from 
[PartitioningUtils::castPartValueToDesiredType|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala#L533]
 returning an Integer for ByteType and ShortType (which then fails to unbox to 
the expected type):

 
{code:java}
case ByteType | ShortType | IntegerType => Integer.parseInt(value) {code}
 

The issue appears to have been introduced in [this 
commit|https://github.com/apache/spark/commit/fc29c91f27d866502f5b6cc4261d4943b5cccc7e]
 so likely affects Spark 3.2 as well, though I've only tested on 3.3.0.

 

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to