Description
ReorderJoin collects all predicates and try to put them into join condition when creating ordered join. If a predicate with an IN subquery is in a join condition instead of a filter condition, `RewritePredicateSubquery.rewriteExistentialExpr` would fail to convert the subquery to an ExistenceJoin, and thus result in error.
For example, tpcds q45 fails due to the above reason:
spark-sql> explain codegen > SELECT > ca_zip, > ca_city, > sum(ws_sales_price) > FROM web_sales, customer, customer_address, date_dim, item > WHERE ws_bill_customer_sk = c_customer_sk > AND c_current_addr_sk = ca_address_sk > AND ws_item_sk = i_item_sk > AND (substr(ca_zip, 1, 5) IN > ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') > OR > i_item_id IN (SELECT i_item_id > FROM item > WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) > ) > ) > AND ws_sold_date_sk = d_date_sk > AND d_qoy = 2 AND d_year = 2001 > GROUP BY ca_zip, ca_city > ORDER BY ca_zip, ca_city > LIMIT 100; 17/03/25 15:27:02 ERROR SparkSQLDriver: Failed in [explain codegen SELECT ca_zip, ca_city, sum(ws_sales_price) FROM web_sales, customer, customer_address, date_dim, item WHERE ws_bill_customer_sk = c_customer_sk AND c_current_addr_sk = ca_address_sk AND ws_item_sk = i_item_sk AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR i_item_id IN (SELECT i_item_id FROM item WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) ) ) AND ws_sold_date_sk = d_date_sk AND d_qoy = 2 AND d_year = 2001 GROUP BY ca_zip, ca_city ORDER BY ca_zip, ca_city LIMIT 100] java.lang.UnsupportedOperationException: Cannot evaluate expression: list#1 [] at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:224) at org.apache.spark.sql.catalyst.expressions.ListQuery.doGenCode(subquery.scala:262) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.catalyst.expressions.In$$anonfun$3.apply(predicates.scala:199) at org.apache.spark.sql.catalyst.expressions.In$$anonfun$3.apply(predicates.scala:199) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.catalyst.expressions.In.doGenCode(predicates.scala:199) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.catalyst.expressions.Or.doGenCode(predicates.scala:379) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.getJoinCondition(BroadcastHashJoinExec.scala:174) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:199) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.consume(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:215) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.consume(SortMergeJoinExec.scala:36) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.doProduce(SortMergeJoinExec.scala:601) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.produce(SortMergeJoinExec.scala:36) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduceWithKeys(HashAggregateExec.scala:600) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduce(HashAggregateExec.scala:148) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.produce(HashAggregateExec.scala:38) at org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:329) at org.apache.spark.sql.execution.debug.package$$anonfun$codegenString$3.apply(package.scala:66) at org.apache.spark.sql.execution.debug.package$$anonfun$codegenString$3.apply(package.scala:62) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732) at org.apache.spark.sql.execution.debug.package$.codegenString(package.scala:62) at org.apache.spark.sql.execution.command.ExplainCommand.run(commands.scala:116) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:67) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:183) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:68) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:617) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:688) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:62) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:335) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:739) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:178) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:117) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) java.lang.UnsupportedOperationException: Cannot evaluate expression: list#1 [] at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:224) at org.apache.spark.sql.catalyst.expressions.ListQuery.doGenCode(subquery.scala:262) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.catalyst.expressions.In$$anonfun$3.apply(predicates.scala:199) at org.apache.spark.sql.catalyst.expressions.In$$anonfun$3.apply(predicates.scala:199) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.immutable.List.map(List.scala:285) at org.apache.spark.sql.catalyst.expressions.In.doGenCode(predicates.scala:199) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.catalyst.expressions.Or.doGenCode(predicates.scala:379) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:104) at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:101) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:101) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.getJoinCondition(BroadcastHashJoinExec.scala:174) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:199) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.consume(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:215) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.consume(SortMergeJoinExec.scala:36) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.doProduce(SortMergeJoinExec.scala:601) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.SortMergeJoinExec.produce(SortMergeJoinExec.scala:36) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) at org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduceWithKeys(HashAggregateExec.scala:600) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.doProduce(HashAggregateExec.scala:148) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) at org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) at org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.produce(HashAggregateExec.scala:38) at org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:329) at org.apache.spark.sql.execution.debug.package$$anonfun$codegenString$3.apply(package.scala:66) at org.apache.spark.sql.execution.debug.package$$anonfun$codegenString$3.apply(package.scala:62) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732) at org.apache.spark.sql.execution.debug.package$.codegenString(package.scala:62) at org.apache.spark.sql.execution.command.ExplainCommand.run(commands.scala:116) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:67) at org.apache.spark.sql.Dataset.<init>(Dataset.scala:183) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:68) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:617) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:688) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:62) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:335) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:739) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:178) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:117) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Attachments
Issue Links
- links to