Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-26041

catalyst cuts out some columns from dataframes: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Cannot Reproduce
    • 2.3.0, 2.3.1, 2.3.2, 2.4.0
    • None
    • Optimizer, Spark Core
    • Spark 2.3.2 

      Hadoop 2.6

      When we materialize one of intermediate dataframes as a parquet table, and read it back in, this error doesn't happen (exact same downflow queries ). 

       

    Description

      There is a workflow with a number of group-by's, joins, `exists` and `in`s between a set of dataframes.

      We are getting following exception and the reason that the Catalyst cuts some columns out of dataframes: 

      Unhandled error: <class 'py4j.protocol.Py4JJavaError'>, An error occurred while calling o1187.cache.
      : org.apache.spark.SparkException: Job aborted due to stage failure: Task 153 in stage 2011.0 failed 4 times, most recent failure: Lost task 153.3 in stage 2011.0 (TID 832340, pc1udatahad23, execut
      or 153): org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: part_code#56012
       at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
       at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
       at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
       at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
       at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
       at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
       at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
       at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
       at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
       at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
       at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
       at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
       at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
       at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
       at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
       at org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate$.bind(GeneratePredicate.scala:45)
       at org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate$.bind(GeneratePredicate.scala:40)
       at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1318)
       at org.apache.spark.sql.execution.SparkPlan.newPredicate(SparkPlan.scala:401)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec.org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$lzycompute(BroadcastNestedLoopJoinExec.scala:87)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec.org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition(BroadcastNestedLoopJoinExec.scala:85)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4$$anonfun$6.apply(BroadcastNestedLoopJoinExec.scala:210)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4$$anonfun$6.apply(BroadcastNestedLoopJoinExec.scala:210)
       at scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38)
       at scala.collection.IndexedSeqOptimized$class.exists(IndexedSeqOptimized.scala:46)
       at scala.collection.mutable.ArrayOps$ofRef.exists(ArrayOps.scala:186)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4.apply(BroadcastNestedLoopJoinExec.scala:210)
       at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4.apply(BroadcastNestedLoopJoinExec.scala:209)
       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
       at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
       at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage39.processNext(Unknown Source)
       at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
       at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
       at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
       at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:187)
       at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
       at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
       at org.apache.spark.scheduler.Task.run(Task.scala:109)
       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
       at java.lang.Thread.run(Thread.java:748)
      

      Caused by

      Caused by: java.lang.RuntimeException: Couldn't find part_code#56012 in [ind_seq#51113L,do_not_rent_flag#50818,title_cd#1576]
       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:97)
       at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:91)
       at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
       ... 43 more
       
      

      Driver stack trace:

      Driver stacktrace:
              at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1609)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1597)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1596)
              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:1596)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831)
              at scala.Option.foreach(Option.scala:257)
              at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:831)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1830)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1779)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1768)
              at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
              at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:642)
              at org.apache.spark.SparkContext.runJob(SparkContext.scala:2034)
              at org.apache.spark.SparkContext.runJob(SparkContext.scala:2055)
              at org.apache.spark.SparkContext.runJob(SparkContext.scala:2074)
              at org.apache.spark.SparkContext.runJob(SparkContext.scala:2099)
              at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:939)
              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:363)
              at org.apache.spark.rdd.RDD.collect(RDD.scala:938)
              at org.apache.spark.RangePartitioner$.sketch(Partitioner.scala:306)
              at org.apache.spark.RangePartitioner.<init>(Partitioner.scala:168)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$.prepareShuffleDependency(ShuffleExchangeExec.scala:224)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:91)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.doExecute(SortMergeJoinExec.scala:150)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.FilterExec.inputRDDs(basicPhysicalOperators.scala:121)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegenExec.scala:363)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.inputRDDs(SortMergeJoinExec.scala:386)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegenExec.scala:363)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.inputRDDs(SortMergeJoinExec.scala:386)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegenExec.scala:363)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.inputRDDs(SortMergeJoinExec.scala:386)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.inputRDDs(BroadcastHashJoinExec.scala:76)
              at org.apache.spark.sql.execution.FilterExec.inputRDDs(basicPhysicalOperators.scala:121)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.inputRDDs(BroadcastHashJoinExec.scala:76)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.inputRDDs(BroadcastHashJoinExec.scala:76)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegenExec.scala:363)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.inputRDDs(SortMergeJoinExec.scala:386)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.doExecute(WholeStageCodegenExec.scala:363)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.joins.SortMergeJoinExec.inputRDDs(SortMergeJoinExec.scala:386)
              at org.apache.spark.sql.execution.ProjectExec.inputRDDs(basicPhysicalOperators.scala:41)
              at org.apache.spark.sql.execution.ExpandExec.inputRDDs(ExpandExec.scala:89)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:150)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:92)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:128)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:119)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:371)
              at org.apache.spark.sql.execution.SortExec.inputRDDs(SortExec.scala:121)
              at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:605)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
              at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
              at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
              at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
              at org.apache.spark.sql.execution.columnar.InMemoryRelation.buildBuffers(InMemoryRelation.scala:107)
              at org.apache.spark.sql.execution.columnar.InMemoryRelation.<init>(InMemoryRelation.scala:102)
              at org.apache.spark.sql.execution.columnar.InMemoryRelation$.apply(InMemoryRelation.scala:43)
              at org.apache.spark.sql.execution.CacheManager$$anonfun$cacheQuery$1.apply(CacheManager.scala:97)
              at org.apache.spark.sql.execution.CacheManager.writeLock(CacheManager.scala:67)
              at org.apache.spark.sql.execution.CacheManager.cacheQuery(CacheManager.scala:91)
              at org.apache.spark.sql.Dataset.persist(Dataset.scala:2902)
              at org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)
              at sun.reflect.GeneratedMethodAccessor89.invoke(Unknown Source)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:498)
              at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
              at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
              at py4j.Gateway.invoke(Gateway.java:282)
              at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
              at py4j.commands.CallCommand.execute(CallCommand.java:79)
              at py4j.GatewayConnection.run(GatewayConnection.java:238)
              at java.lang.Thread.run(Thread.java:748)
      
      

      Caused by:

              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: null
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:56)
              at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:91)
              at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:90)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
              at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
              at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:90)
              at org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate$.bind(GeneratePredicate.scala:45)
              at org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate$.bind(GeneratePredicate.scala:40)
              at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:1318)
              at org.apache.spark.sql.execution.SparkPlan.newPredicate(SparkPlan.scala:401)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec.org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition$lzycompute(BroadcastNestedLoopJoinExec.scala:87)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec.org$apache$spark$sql$execution$joins$BroadcastNestedLoopJoinExec$$boundCondition(BroadcastNestedLoopJoinExec.scala:85)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4$$anonfun$6.apply(BroadcastNestedLoopJoinExec.scala:210)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4$$anonfun$6.apply(BroadcastNestedLoopJoinExec.scala:210)
              at scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38)
              at scala.collection.IndexedSeqOptimized$class.exists(IndexedSeqOptimized.scala:46)
              at scala.collection.mutable.ArrayOps$ofRef.exists(ArrayOps.scala:186)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4.apply(BroadcastNestedLoopJoinExec.scala:210)
              at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec$$anonfun$5$$anonfun$apply$4.apply(BroadcastNestedLoopJoinExec.scala:209)
              at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
              at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
              at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
              at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
              at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage39.processNext(Unknown Source)
              at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
              at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$10$$anon$1.hasNext(WholeStageCodegenExec.scala:614)
              at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
              at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:187)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
              at org.apache.spark.scheduler.Task.run(Task.scala:109)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              ... 1 more
      

      Caused by one more time :

      Caused by: java.lang.RuntimeException: Couldn't find part_code#56012 in [ind_seq#51113L,do_not_rent_flag#50818,title_cd#1576]
              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:97)
              at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:91)
              at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
              ... 43 more
      
      

      Attachments

        1. SPARK-26041.txt
          51 kB
          Ruslan Dautkhanov

        Issue Links

          Activity

            People

              Unassigned Unassigned
              Tagar Ruslan Dautkhanov
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: