Return-Path: X-Original-To: apmail-spark-issues-archive@minotaur.apache.org Delivered-To: apmail-spark-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3B37E19F98 for ; Tue, 29 Mar 2016 16:49:26 +0000 (UTC) Received: (qmail 84771 invoked by uid 500); 29 Mar 2016 16:49:25 -0000 Delivered-To: apmail-spark-issues-archive@spark.apache.org Received: (qmail 84678 invoked by uid 500); 29 Mar 2016 16:49:25 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 84478 invoked by uid 99); 29 Mar 2016 16:49:25 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Mar 2016 16:49:25 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 9155B2C1F6A for ; Tue, 29 Mar 2016 16:49:25 +0000 (UTC) Date: Tue, 29 Mar 2016 16:49:25 +0000 (UTC) From: "Cheng Lian (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (SPARK-14244) Physical Window operator uses global SizeBasedWindowFunction.n attribute generated on both driver and executor side MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 Cheng Lian created SPARK-14244: ---------------------------------- Summary: Physical Window operator uses global SizeBasedWindowFunction.n attribute generated on both driver and executor side Key: SPARK-14244 URL: https://issues.apache.org/jira/browse/SPARK-14244 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.6.1, 2.0.0 Reporter: Cheng Lian Assignee: Cheng Lian To reproduce this issue, first start a local cluster with at least one worker. Then try the following Spark shell snippet: {code} import org.apache.spark.sql.expressions._ import org.apache.spark.sql.functions._ sqlContext. range(10). select( 'id, cume_dist() over (Window orderBy 'id) as 'cdist ). orderBy('cdist). show() {code} Exception thrown: {noformat} org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in stage 1.0 (TID 11, 192.168.1.101): org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: window__partition__size#4 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:92) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91) 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:67) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:258) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:301) at scala.collection.Iterator$$anon$11.next(Iterator.scala:370) at scala.collection.Iterator$class.foreach(Iterator.scala:742) at scala.collection.AbstractIterator.foreach(Iterator.scala:1194) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:308) at scala.collection.AbstractIterator.to(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:300) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:287) at scala.collection.AbstractIterator.toArray(Iterator.scala:1194) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:350) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:301) at scala.collection.Iterator$$anon$11.next(Iterator.scala:370) at scala.collection.Iterator$class.foreach(Iterator.scala:742) at scala.collection.AbstractIterator.foreach(Iterator.scala:1194) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:308) at scala.collection.AbstractIterator.to(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:300) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:287) at scala.collection.AbstractIterator.toArray(Iterator.scala:1194) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:350) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:248) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:91) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:38) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:38) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at scala.collection.TraversableLike$class.map(TraversableLike.scala:245) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:38) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.generate(GenerateMutableProjection.scala:44) at org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:348) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$org$apache$spark$sql$execution$Window$$anonfun$$processor$1$1.apply(Window.scala:202) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$org$apache$spark$sql$execution$Window$$anonfun$$processor$1$1.apply(Window.scala:201) at org.apache.spark.sql.execution.AggregateProcessor$.apply(Window.scala:927) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2.org$apache$spark$sql$execution$Window$$anonfun$$processor$1(Window.scala:197) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$6.apply(Window.scala:223) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$6.apply(Window.scala:220) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1$$anonfun$16.apply(Window.scala:317) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1$$anonfun$16.apply(Window.scala:317) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at scala.collection.TraversableLike$class.map(TraversableLike.scala:245) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1.(Window.scala:317) at org.apache.spark.sql.execution.Window$$anonfun$14.apply(Window.scala:289) at org.apache.spark.sql.execution.Window$$anonfun$14.apply(Window.scala:288) at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$22.apply(RDD.scala:754) at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$22.apply(RDD.scala:754) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:69) at org.apache.spark.scheduler.Task.run(Task.scala:82) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:231) 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 window__partition__size#4 in [window__partition__size#0,rowNumber#3] 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:98) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:92) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:46) ... 87 more Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1457) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1445) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1444) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1444) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:809) at scala.Option.foreach(Option.scala:257) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:809) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1666) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1625) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1614) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:630) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1765) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1828) at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:971) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) at org.apache.spark.rdd.RDD.reduce(RDD.scala:953) at org.apache.spark.rdd.RDD$$anonfun$takeOrdered$1.apply(RDD.scala:1352) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:357) at org.apache.spark.rdd.RDD.takeOrdered(RDD.scala:1339) at org.apache.spark.sql.execution.TakeOrderedAndProject.executeCollect(limit.scala:126) at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2049) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:53) at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2312) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2048) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2055) at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1822) at org.apache.spark.sql.Dataset$$anonfun$head$1.apply(Dataset.scala:1821) at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2342) at org.apache.spark.sql.Dataset.head(Dataset.scala:1821) at org.apache.spark.sql.Dataset.take(Dataset.scala:2005) at org.apache.spark.sql.Dataset.showString(Dataset.scala:231) at org.apache.spark.sql.Dataset.show(Dataset.scala:508) at org.apache.spark.sql.Dataset.show(Dataset.scala:468) at org.apache.spark.sql.Dataset.show(Dataset.scala:477) ... 53 elided Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: window__partition__size#4 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:92) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91) 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:67) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:258) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:301) at scala.collection.Iterator$$anon$11.next(Iterator.scala:370) at scala.collection.Iterator$class.foreach(Iterator.scala:742) at scala.collection.AbstractIterator.foreach(Iterator.scala:1194) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:308) at scala.collection.AbstractIterator.to(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:300) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:287) at scala.collection.AbstractIterator.toArray(Iterator.scala:1194) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:350) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:301) at scala.collection.Iterator$$anon$11.next(Iterator.scala:370) at scala.collection.Iterator$class.foreach(Iterator.scala:742) at scala.collection.AbstractIterator.foreach(Iterator.scala:1194) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:308) at scala.collection.AbstractIterator.to(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:300) at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1194) at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:287) at scala.collection.AbstractIterator.toArray(Iterator.scala:1194) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:350) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:264) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:248) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:91) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:38) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:38) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at scala.collection.TraversableLike$class.map(TraversableLike.scala:245) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:38) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.generate(GenerateMutableProjection.scala:44) at org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:348) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$org$apache$spark$sql$execution$Window$$anonfun$$processor$1$1.apply(Window.scala:202) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$org$apache$spark$sql$execution$Window$$anonfun$$processor$1$1.apply(Window.scala:201) at org.apache.spark.sql.execution.AggregateProcessor$.apply(Window.scala:927) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2.org$apache$spark$sql$execution$Window$$anonfun$$processor$1(Window.scala:197) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$6.apply(Window.scala:223) at org.apache.spark.sql.execution.Window$$anonfun$windowFrameExpressionFactoryPairs$2$$anonfun$6.apply(Window.scala:220) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1$$anonfun$16.apply(Window.scala:317) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1$$anonfun$16.apply(Window.scala:317) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:245) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186) at scala.collection.TraversableLike$class.map(TraversableLike.scala:245) at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186) at org.apache.spark.sql.execution.Window$$anonfun$14$$anon$1.(Window.scala:317) at org.apache.spark.sql.execution.Window$$anonfun$14.apply(Window.scala:289) at org.apache.spark.sql.execution.Window$$anonfun$14.apply(Window.scala:288) at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$22.apply(RDD.scala:754) at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$22.apply(RDD.scala:754) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:69) at org.apache.spark.scheduler.Task.run(Task.scala:82) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:231) 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 window__partition__size#4 in [window__partition__size#0,rowNumber#3] 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:98) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:92) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:46) ... 87 more {noformat} The reason is that we are trying to bind global attribute {{SizeBasedWindowFunction.n}} created on driver side to a schema containing {{SizeBasedWindowFunction.n}} created on executor side. It fails because these two attributes have different expression IDs. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org