[ 
https://issues.apache.org/jira/browse/SPARK-13087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15126980#comment-15126980
 ] 

Trystan Leftwich commented on SPARK-13087:
------------------------------------------

I re-built 1.6.1 with commit ddb9633043e82fb2a34c7e0e29b487f635c3c744 this 
morning and I'm seeing a similar error to above. Similarly we use a custom UDF 
and the above commit fixes the issue.

{code:sql}
SELECT
   concat(t_4.firstname," ",t_4.lastname) customer_name,
  agg_cust(t_3.customercountestimate1_c2) ctd_customercountestimate1_ok
FROM
  as.sales t_3
JOIN
   as.customer t_4
ON
   t_3.key_c1 = t_4.customerkey
GROUP BY
   concat(t_4.firstname," ",t_4.lastname)
{code}

{code}
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding 
attribute, tree: concat(firstname#329, ,lastname#330)#339
        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.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:39)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:39)
        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.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:39)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:33)
        at 
org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:585)
        at 
org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:227)
        at 
org.apache.spark.sql.execution.Exchange.org$apache$spark$sql$execution$Exchange$$getPartitionKeyExtractor$1(Exchange.scala:197)
        at 
org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:209)
        at 
org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:208)
        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 concat(firstname#329, 
,lastname#330)#339 in [firstname#329,lastname#330,customercountestimate1_c2#326]
        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)
        ... 34 more
{code}

> Grouping by a complex expression may lead to incorrect AttributeReferences in 
> aggregations
> ------------------------------------------------------------------------------------------
>
>                 Key: SPARK-13087
>                 URL: https://issues.apache.org/jira/browse/SPARK-13087
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.0
>            Reporter: Mark Hamstra
>
> This is a regression from 1.5.
> An example of the failure:
> Working with this table...
> {code}
> 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     |
> +-----------------------+------------+----------+--+
> {code}
> ...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)...
> {code}
> 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;
> {code}
> Spark 1.5 produces the expected results without error.
> In Spark 1.6, this plan is produced...
> {code}
> 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
> {code}
> ...which fails with a TreeNodeException and stack traces that include this...
> {code}
> 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
> {code}
> 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



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to