You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Michael Armbrust (JIRA)" <ji...@apache.org> on 2014/06/26 23:21:25 UTC

[jira] [Updated] (SPARK-2066) Better error message for non-aggregated attributes with aggregates

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

Michael Armbrust updated SPARK-2066:
------------------------------------

    Summary: Better error message for non-aggregated attributes with aggregates  (was: Better Error message for unresolved attributes)

> Better error message for non-aggregated attributes with aggregates
> ------------------------------------------------------------------
>
>                 Key: SPARK-2066
>                 URL: https://issues.apache.org/jira/browse/SPARK-2066
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.0.0
>            Reporter: Reynold Xin
>            Assignee: Cheng Lian
>
> [~marmbrus]
> Run the following query
> {code}
> scala> c.hql("select key, count(*) from src").collect()
> {code}
> Got the following exception at runtime
> {code}
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: No function to evaluate expression. type: AttributeReference, tree: key#61
> 	at org.apache.spark.sql.catalyst.expressions.AttributeReference.eval(namedExpressions.scala:157)
> 	at org.apache.spark.sql.catalyst.expressions.Projection.apply(Projection.scala:35)
> 	at org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$1.apply(Aggregate.scala:154)
> 	at org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$1.apply(Aggregate.scala:134)
> 	at org.apache.spark.rdd.RDD$$anonfun$12.apply(RDD.scala:558)
> 	at org.apache.spark.rdd.RDD$$anonfun$12.apply(RDD.scala:558)
> 	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
> 	at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
> 	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:261)
> 	at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:111)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:51)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:187)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:744)
> {code}
> This should either fail in analysis time, or pass at runtime. Definitely shouldn't fail at runtime.



--
This message was sent by Atlassian JIRA
(v6.2#6252)