You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Alessandro Bacchini (Jira)" <ji...@apache.org> on 2022/02/22 09:14:00 UTC

[jira] [Updated] (SPARK-38285) ClassCastException: GenericArrayData cannot be cast to InternalRow

     [ https://issues.apache.org/jira/browse/SPARK-38285?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alessandro Bacchini updated SPARK-38285:
----------------------------------------
    Description: 
The following code with Spark 3.2.1 raises an exception:

{code:python}
import pyspark.sql.functions as F
from pyspark.sql.types import StructType, StructField, ArrayType, StringType

t = StructType([
    StructField('o', 
        ArrayType(
            StructType([
                StructField('s', StringType(), False),
                StructField('b', ArrayType(
                    StructType([
                        StructField('e', StringType(), False)
                    ]),
                    True),
                False)
            ]), 
        True),
    False)])

value = {
    "o": [
        {
            "s": "string1",
            "b": [
                {
                    "e": "string2"
                },
                {
                    "e": "string3"
                }
            ]
        },
        {
            "s": "string4",
            "b": [
                {
                    "e": "string5"
                },
                {
                    "e": "string6"
                },
                {
                    "e": "string7"
                }
            ]
        }
    ]
}

df = (
    spark.createDataFrame([value], schema=t)
    .select(F.explode("o").alias("eo"))
    .select("eo.b.e")
)


df.show()
{code}

The exception message is:
{code}
java.lang.ClassCastException: org.apache.spark.sql.catalyst.util.GenericArrayData cannot be cast to org.apache.spark.sql.catalyst.InternalRow
	at org.apache.spark.sql.catalyst.util.GenericArrayData.getStruct(GenericArrayData.scala:76)
	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$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at org.apache.spark.sql.execution.collect.UnsafeRowBatchUtils$.encodeUnsafeRows(UnsafeRowBatchUtils.scala:80)
	at org.apache.spark.sql.execution.collect.Collector.$anonfun$processFunc$1(Collector.scala:155)
	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$3(ResultTask.scala:75)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$1(ResultTask.scala:75)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:55)
	at org.apache.spark.scheduler.Task.doRunTask(Task.scala:153)
	at org.apache.spark.scheduler.Task.$anonfun$run$1(Task.scala:122)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.Task.run(Task.scala:93)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$13(Executor.scala:824)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1641)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:827)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:683)
	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)
{code}

I am using Spark 3.2.1, but I don't know if even Spark 3.3.0 is affected.

Please note that the issue seems to be related to SPARK-37577: I am using the same DataFrame schema, but this time I have populated it with non empty value.

I think that this is bug because with the following configuration it works as expected:
{code:python}
spark.conf.set("spark.sql.optimizer.expression.nestedPruning.enabled", False)
spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", False)
{code}

  was:
The following code with Spark 3.2.1 raises an exception:

{code:python}
import pyspark.sql.functions as F
from pyspark.sql.types import StructType, StructField, ArrayType, StringType

t = StructType([
    StructField('o', 
        ArrayType(
            StructType([
                StructField('s', StringType(), False),
                StructField('b', ArrayType(
                    StructType([
                        StructField('e', StringType(), False)
                    ]),
                    True),
                False)
            ]), 
        True),
    False)])

value = {
    "o": [
        {
            "s": "string1",
            "b": [
                {
                    "e": "string2"
                },
                {
                    "e": "string3"
                }
            ]
        },
        {
            "s": "string4",
            "b": [
                {
                    "e": "string5"
                },
                {
                    "e": "string6"
                },
                {
                    "e": "string7"
                }
            ]
        }
    ]
}

df = (
    spark.createDataFrame([value], schema=t)
    .select(F.explode("o").alias("eo"))
    .select("eo.b.e")
)


df.show()
{code}

The exception message is:
{code}
java.lang.ClassCastException: org.apache.spark.sql.catalyst.util.GenericArrayData cannot be cast to org.apache.spark.sql.catalyst.InternalRow
	at org.apache.spark.sql.catalyst.util.GenericArrayData.getStruct(GenericArrayData.scala:76)
	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$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
	at org.apache.spark.sql.execution.collect.UnsafeRowBatchUtils$.encodeUnsafeRows(UnsafeRowBatchUtils.scala:80)
	at org.apache.spark.sql.execution.collect.Collector.$anonfun$processFunc$1(Collector.scala:155)
	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$3(ResultTask.scala:75)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$1(ResultTask.scala:75)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:55)
	at org.apache.spark.scheduler.Task.doRunTask(Task.scala:153)
	at org.apache.spark.scheduler.Task.$anonfun$run$1(Task.scala:122)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.scheduler.Task.run(Task.scala:93)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$13(Executor.scala:824)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1641)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:827)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:683)
	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)
{code}

I am using Spark 3.2.1.

Please note that the issue seems to be related to SPARK-37577: I am using the same DataFrame schema, but this time I have populated it with non empty value.

I think that this is bug because with the following configuration it works as expected:
{code:python}
spark.conf.set("spark.sql.optimizer.expression.nestedPruning.enabled", False)
spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", False)
{code}


> ClassCastException: GenericArrayData cannot be cast to InternalRow
> ------------------------------------------------------------------
>
>                 Key: SPARK-38285
>                 URL: https://issues.apache.org/jira/browse/SPARK-38285
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 3.2.1
>            Reporter: Alessandro Bacchini
>            Priority: Major
>
> The following code with Spark 3.2.1 raises an exception:
> {code:python}
> import pyspark.sql.functions as F
> from pyspark.sql.types import StructType, StructField, ArrayType, StringType
> t = StructType([
>     StructField('o', 
>         ArrayType(
>             StructType([
>                 StructField('s', StringType(), False),
>                 StructField('b', ArrayType(
>                     StructType([
>                         StructField('e', StringType(), False)
>                     ]),
>                     True),
>                 False)
>             ]), 
>         True),
>     False)])
> value = {
>     "o": [
>         {
>             "s": "string1",
>             "b": [
>                 {
>                     "e": "string2"
>                 },
>                 {
>                     "e": "string3"
>                 }
>             ]
>         },
>         {
>             "s": "string4",
>             "b": [
>                 {
>                     "e": "string5"
>                 },
>                 {
>                     "e": "string6"
>                 },
>                 {
>                     "e": "string7"
>                 }
>             ]
>         }
>     ]
> }
> df = (
>     spark.createDataFrame([value], schema=t)
>     .select(F.explode("o").alias("eo"))
>     .select("eo.b.e")
> )
> df.show()
> {code}
> The exception message is:
> {code}
> java.lang.ClassCastException: org.apache.spark.sql.catalyst.util.GenericArrayData cannot be cast to org.apache.spark.sql.catalyst.InternalRow
> 	at org.apache.spark.sql.catalyst.util.GenericArrayData.getStruct(GenericArrayData.scala:76)
> 	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$$anon$1.hasNext(WholeStageCodegenExec.scala:759)
> 	at org.apache.spark.sql.execution.collect.UnsafeRowBatchUtils$.encodeUnsafeRows(UnsafeRowBatchUtils.scala:80)
> 	at org.apache.spark.sql.execution.collect.Collector.$anonfun$processFunc$1(Collector.scala:155)
> 	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$3(ResultTask.scala:75)
> 	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
> 	at org.apache.spark.scheduler.ResultTask.$anonfun$runTask$1(ResultTask.scala:75)
> 	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:55)
> 	at org.apache.spark.scheduler.Task.doRunTask(Task.scala:153)
> 	at org.apache.spark.scheduler.Task.$anonfun$run$1(Task.scala:122)
> 	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:93)
> 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$13(Executor.scala:824)
> 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1641)
> 	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:827)
> 	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
> 	at com.databricks.spark.util.ExecutorFrameProfiler$.record(ExecutorFrameProfiler.scala:110)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:683)
> 	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)
> {code}
> I am using Spark 3.2.1, but I don't know if even Spark 3.3.0 is affected.
> Please note that the issue seems to be related to SPARK-37577: I am using the same DataFrame schema, but this time I have populated it with non empty value.
> I think that this is bug because with the following configuration it works as expected:
> {code:python}
> spark.conf.set("spark.sql.optimizer.expression.nestedPruning.enabled", False)
> spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", False)
> {code}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org