Description
This is a regression from 1.5.
An example of the failure:
Working with this table...
0: jdbc:hive2://10.1.3.203:10000> DESCRIBE csd_0ae1abc1_a3af_4c63_95b0_9599faca6c3d; +-----------------------+------------+----------+--+ | col_name | data_type | comment | +-----------------------+------------+----------+--+ | c_date | timestamp | NULL | | c_count | int | NULL | | c_location_fips_code | string | NULL | | c_airtemp | float | NULL | | c_dewtemp | float | NULL | | c_pressure | int | NULL | | c_rain | float | NULL | | c_snow | float | NULL | +-----------------------+------------+----------+--+
...and this query (which isn't necessarily all that sensical or useful, but has been adapted from a similarly failing query that uses a custom UDF where the Spark SQL built-in `day` function has been substituted into this query)...
SELECT day ( c_date ) AS c_date, percentile_approx(c_rain, 0.5) AS c_expr_1256887735 FROM csd_0ae1abc1_a3af_4c63_95b0_9599faca6c3d GROUP BY day ( c_date ) ORDER BY c_date;
Spark 1.5 produces the expected results without error.
In Spark 1.6, this plan is produced...
Exchange rangepartitioning(c_date#63009 ASC,16), None +- SortBasedAggregate(key=[dayofmonth(cast(c_date#63011 as date))#63020], functions=[(hiveudaffunction(HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox,org.apache.hadoop.hive.ql.udf.generic.Gene ricUDAFPercentileApprox@6f211801),c_rain#63017,0.5,false,0,0),mode=Complete,isDistinct=false)], output=[c_date#63009,c_expr_1256887735#63010]) +- ConvertToSafe +- !Sort [dayofmonth(cast(c_date#63011 as date))#63020 ASC], false, 0 +- !TungstenExchange hashpartitioning(dayofmonth(cast(c_date#63011 as date))#63020,16), None +- ConvertToUnsafe +- HiveTableScan [c_date#63011,c_rain#63017], MetastoreRelation default, csd_0ae1abc1_a3af_4c63_95b0_9599faca6c3d, None
...which fails with a TreeNodeException and stack traces that include this...
Caused by: ! org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2842.0 failed 4 times, most recent failure: Lost task 0.3 in stage 2842.0 (TID 15007, ip-10-1-1-59.dev.clearstory.com): org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: dayofmonth(cast(c_date#63011 as date))#63020 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:86) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:85) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:258) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:249) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:85) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:62) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:62) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.<init>(Projection.scala:62) at org.apache.spark.sql.execution.SparkPlan$$anonfun$newMutableProjection$1.apply(SparkPlan.scala:254) at org.apache.spark.sql.execution.SparkPlan$$anonfun$newMutableProjection$1.apply(SparkPlan.scala:254) at org.apache.spark.sql.execution.Exchange.org$apache$spark$sql$execution$Exchange$$getPartitionKeyExtractor$1(Exchange.scala:196) at org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:208) at org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:207) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.RuntimeException: Couldn't find dayofmonth(cast(c_date#63011 as date))#63020 in [c_date#63011,c_rain#63017] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:92) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:86) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:48) ... 33 more
It is possible to work around the problem by adding a Project node in case an aggregation is relying on aliases missing in the child plan (https://github.com/mbautin/spark/commit/2e99064b42a6dddf6b94b989c744a1308aacaee2), but it seems there should be a deeper fix that prevents the problem instead of covering for it.
yhuai I think this problem crept in with the changes for SPARK-9830
Attachments
Issue Links
- is related to
-
SPARK-26041 catalyst cuts out some columns from dataframes: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute
- Resolved
- links to