[ https://issues.apache.org/jira/browse/SPARK-3336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14119400#comment-14119400 ]
Davies Liu commented on SPARK-3336: ----------------------------------- [~marmbrus], If we reverse the order of count() and MYUDF(), then it can run successfully: {code} SELECT MYUDF(foo), COUNT(1) FROM bar GROUP BY MYUDF(foo); {code} If we put COUNT(1) before MYUDF(foo), it will also raise the same exception in Scala: scala> sqlContext.sql("SELECT count(1), MYUDF(name) from people GROUP BY MYUDF(name) ").collect() 14/09/02 22:45:50 ERROR Executor: Exception in task 2.0 in stage 3.0 (TID 214) org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: name#0 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:47) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:43) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:42) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:144) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4$$anonfun$apply$2.apply(TreeNode.scala:180) 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.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:178) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273) at scala.collection.AbstractIterator.to(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) at scala.collection.AbstractIterator.toArray(Iterator.scala:1157) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildrenDown(TreeNode.scala:191) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:147) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:162) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273) at scala.collection.AbstractIterator.to(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) at scala.collection.AbstractIterator.toArray(Iterator.scala:1157) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildrenDown(TreeNode.scala:191) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:147) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:135) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:42) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:52) at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:52) 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:52) at org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7$$anon$1.<init>(Aggregate.scala:176) at org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7.apply(Aggregate.scala:172) at org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7.apply(Aggregate.scala:151) at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596) at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) at org.apache.spark.rdd.RDD.iterator(RDD.scala:229) at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) at org.apache.spark.rdd.RDD.iterator(RDD.scala:229) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:62) at org.apache.spark.scheduler.Task.run(Task.scala:54) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:178) 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:745) Caused by: java.lang.RuntimeException: Couldn't find name#0 in [aggResult:SUM(PartialCount#35L)#38L,c0#31] 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:46) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:43) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:46) ... 66 more So it seems that it's not a bug related to Python UDF, it's a bug in sql parser or query plan. > [Spark SQL] In pyspark, cannot group by field on UDF > ---------------------------------------------------- > > Key: SPARK-3336 > URL: https://issues.apache.org/jira/browse/SPARK-3336 > Project: Spark > Issue Type: Bug > Components: PySpark, SQL > Affects Versions: 1.1.0 > Reporter: kay feng > Assignee: Davies Liu > > Running pyspark on a spark cluster with standalone master. > Cannot group by field on a UDF. But we can group by UDF in Scala. > For example: > q = sqlContext.sql('SELECT COUNT(*), MYUDF(foo) FROM bar GROUP BY > MYUDF(foo)') > out = q.collect() > I got this exception: > Py4JJavaError: An error occurred while calling o183.collect. > : org.apache.spark.SparkException: Job aborted due to stage failure: Task 26 > in stage 56.0 failed 4 times, most recent failure: Lost task 26.3 in stage > 56.0 (TID 14038, ip-10-33-9-144.us-west-2.compute.internal): > org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding > attribute, tree: pythonUDF#1278 > > org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:47) > > org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:43) > > org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:42) > > org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:165) > > org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:183) > scala.collection.Iterator$$anon$11.next(Iterator.scala:328) > scala.collection.Iterator$class.foreach(Iterator.scala:727) > scala.collection.AbstractIterator.foreach(Iterator.scala:1157) > > scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48) > > scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103) > > scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47) > scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273) > scala.collection.AbstractIterator.to(Iterator.scala:1157) > > scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265) > scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157) > > scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252) > scala.collection.AbstractIterator.toArray(Iterator.scala:1157) > > org.apache.spark.sql.catalyst.trees.TreeNode.transformChildrenDown(TreeNode.scala:212) > > org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:168) > > org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:156) > > org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:42) > > org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:52) > > org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection$$anonfun$$init$$2.apply(Projection.scala:52) > > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) > scala.collection.immutable.List.foreach(List.scala:318) > scala.collection.TraversableLike$class.map(TraversableLike.scala:244) > scala.collection.AbstractTraversable.map(Traversable.scala:105) > > org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.<init>(Projection.scala:52) > > org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7$$anon$1.<init>(Aggregate.scala:176) > > org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7.apply(Aggregate.scala:172) > > org.apache.spark.sql.execution.Aggregate$$anonfun$execute$1$$anonfun$7.apply(Aggregate.scala:151) > org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596) > org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596) > > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > org.apache.spark.sql.SchemaRDD.compute(SchemaRDD.scala:115) > org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) > org.apache.spark.rdd.RDD.iterator(RDD.scala:229) > > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35) > org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262) > org.apache.spark.rdd.RDD.iterator(RDD.scala:229) > org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:62) > org.apache.spark.scheduler.Task.run(Task.scala:54) > org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177) > > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > java.lang.Thread.run(Thread.java:745) > Driver stacktrace: > at > org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1185) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1174) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1173) > at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) > at > org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1173) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688) > at > org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:688) > at scala.Option.foreach(Option.scala:236) > at > org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:688) > at > org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1391) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498) > at akka.actor.ActorCell.invoke(ActorCell.scala:456) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237) > at akka.dispatch.Mailbox.run(Mailbox.scala:219) > at > akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386) > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > at > scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) -- 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