[ https://issues.apache.org/jira/browse/SPARK-37577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
L. C. Hsieh updated SPARK-37577: -------------------------------- Component/s: SQL (was: PySpark) > ClassCastException: ArrayType cannot be cast to StructType > ---------------------------------------------------------- > > Key: SPARK-37577 > URL: https://issues.apache.org/jira/browse/SPARK-37577 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 3.2.0 > Environment: Py: 3.9 > Reporter: Rafal Wojdyla > Priority: Major > > Reproduction: > {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)]) > ( > spark.createDataFrame([], schema=t) > .select(F.explode("o").alias("eo")) > .select("eo.*") > .select(F.explode("b")) > .count() > ) > {code} > Code above works fine in 3.1.2, fails in 3.2.0. See stacktrace below. Note > that if you remove, field {{s}}, the code works fine, which is a bit > unexpected and likely a clue. > {noformat} > Py4JJavaError: An error occurred while calling o156.count. > : java.lang.ClassCastException: class org.apache.spark.sql.types.ArrayType > cannot be cast to class org.apache.spark.sql.types.StructType > (org.apache.spark.sql.types.ArrayType and > org.apache.spark.sql.types.StructType are in unnamed module of loader 'app') > at > org.apache.spark.sql.catalyst.expressions.GetStructField.childSchema$lzycompute(complexTypeExtractors.scala:107) > at > org.apache.spark.sql.catalyst.expressions.GetStructField.childSchema(complexTypeExtractors.scala:107) > at > org.apache.spark.sql.catalyst.expressions.GetStructField.$anonfun$extractFieldName$1(complexTypeExtractors.scala:117) > at scala.Option.getOrElse(Option.scala:189) > at > org.apache.spark.sql.catalyst.expressions.GetStructField.extractFieldName(complexTypeExtractors.scala:117) > at > org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing$$anonfun$1$$anonfun$2.applyOrElse(NestedColumnAliasing.scala:372) > at > org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing$$anonfun$1$$anonfun$2.applyOrElse(NestedColumnAliasing.scala:368) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformUpWithPruning$4(TreeNode.scala:539) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUpWithPruning(TreeNode.scala:539) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:508) > at > org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing$$anonfun$1.applyOrElse(NestedColumnAliasing.scala:368) > at > org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing$$anonfun$1.applyOrElse(NestedColumnAliasing.scala:366) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$transformExpressionsDownWithPruning$1(QueryPlan.scala:152) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:193) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:193) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:204) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:214) > at > org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:323) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:214) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDownWithPruning(QueryPlan.scala:152) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsWithPruning(QueryPlan.scala:123) > at > org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:101) > at > org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing$.unapply(NestedColumnAliasing.scala:366) > at > org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$14.applyOrElse(Optimizer.scala:826) > at > org.apache.spark.sql.catalyst.optimizer.ColumnPruning$$anonfun$apply$14.applyOrElse(Optimizer.scala:783) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:481) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:82) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:481) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1128) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1127) > at > org.apache.spark.sql.catalyst.plans.logical.Generate.mapChildren(basicLogicalOperators.scala:121) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1128) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1127) > at > org.apache.spark.sql.catalyst.plans.logical.OrderPreservingUnaryNode.mapChildren(LogicalPlan.scala:206) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1128) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1127) > at > org.apache.spark.sql.catalyst.plans.logical.Aggregate.mapChildren(basicLogicalOperators.scala:940) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:486) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:30) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:447) > at > org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:783) > at > org.apache.spark.sql.catalyst.optimizer.ColumnPruning$.apply(Optimizer.scala:780) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:211) > at > scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) > at > scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) > at scala.collection.immutable.List.foldLeft(List.scala:91) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:208) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:200) > at scala.collection.immutable.List.foreach(List.scala:431) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:200) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:179) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:179) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:138) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:196) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) > at > org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:196) > at > org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:134) > at > org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:130) > at > org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:148) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executedPlan$1(QueryExecution.scala:166) > at > org.apache.spark.sql.execution.QueryExecution.withCteMap(QueryExecution.scala:73) > at > org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:163) > at > org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:163) > at > org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:214) > at > org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:259) > at > org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:228) > at > org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98) > at > org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163) > at > org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64) > at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3704) > at org.apache.spark.sql.Dataset.count(Dataset.scala:3011) > at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.base/java.lang.reflect.Method.invoke(Method.java:566) > at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244) > at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) > at py4j.Gateway.invoke(Gateway.java:282) > at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) > at py4j.commands.CallCommand.execute(CallCommand.java:79) > at > py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182) > at py4j.ClientServerConnection.run(ClientServerConnection.java:106) > at java.base/java.lang.Thread.run(Thread.java:834) > {noformat} -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org