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

Catalyst error on on certain struct operation (Couldn't find _gen_alias_)

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.1.1, 3.2.0
    • 2.4.8, 3.0.3, 3.1.2, 3.2.0
    • SQL
    • None
    • Spark 3.1.1

      Scala 2.12.10

    Description

      When I run the following:

      import org.apache.spark.sql.{functions => f}
      import org.apache.spark.sql.expressions.Window
      
      val df = Seq(
          ("t1", "123", "bob"),
          ("t1", "456", "bob"),
          ("t2", "123", "sam")
      ).toDF("type", "value", "name")
      
      val test = df.select(
          $"*",
          f.struct(f.count($"*").over(Window.partitionBy($"type", $"value", $"name")).as("count"), $"name").as("name_count")
      ).select(
        $"*",
        f.max($"name_count").over(Window.partitionBy($"type", $"value")).as("best_name")
      )
      
      test.printSchema
      

      I get the following schema, which is fine:

      root
       |-- type: string (nullable = true)
       |-- value: string (nullable = true)
       |-- name: string (nullable = true)
       |-- name_count: struct (nullable = false)
       |    |-- count: long (nullable = false)
       |    |-- name: string (nullable = true)
       |-- best_name: struct (nullable = true)
       |    |-- count: long (nullable = false)
       |    |-- name: string (nullable = true)
      

      However when I get a subfield of the best_name struct, I get an error:

      test.select($"best_name.name").show(10, false)
      org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _gen_alias_3458#3458
        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:75)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:74)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:317)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:73)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:317)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:407)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:243)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:405)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:358)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:306)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:74)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.$anonfun$bindReferences$1(BoundAttribute.scala:96)
        at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
        at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at scala.collection.TraversableLike.map(TraversableLike.scala:238)
        at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
        at scala.collection.AbstractTraversable.map(Traversable.scala:108)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReferences(BoundAttribute.scala:96)
        at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.<init>(InterpretedMutableProjection.scala:35)
        at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$19.applyOrElse(Optimizer.scala:1589)
        at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$$anonfun$apply$19.applyOrElse(Optimizer.scala:1586)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:317)
        at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:73)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:317)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:407)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:243)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:405)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:358)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:407)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:243)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:405)
        at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:358)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:322)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown(AnalysisHelper.scala:171)
        at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDown$(AnalysisHelper.scala:169)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDown(LogicalPlan.scala:29)
        at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:306)
        at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$.apply(Optimizer.scala:1586)
        at org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation$.apply(Optimizer.scala:1585)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:216)
        at scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60)
        at scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68)
        at scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:38)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:213)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:205)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:205)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:183)
        at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
        at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:183)
        at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:87)
        at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
        at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:143)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
        at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:143)
        at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:84)
        at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:84)
        at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:95)
        at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:113)
        at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:110)
        at org.apache.spark.sql.execution.QueryExecution.$anonfun$simpleString$2(QueryExecution.scala:161)
        at org.apache.spark.sql.execution.ExplainUtils$.processPlan(ExplainUtils.scala:115)
        at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:161)
        at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:206)
        at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:175)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:98)
        at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
        at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:772)
        at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3685)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2722)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2929)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:301)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:338)
        at org.apache.spark.sql.Dataset.show(Dataset.scala:827)
        ... 49 elided
      Caused by: java.lang.RuntimeException: Couldn't find _gen_alias_3458#3458 in [type#3427,value#3428,name#3429]
        at scala.sys.package$.error(package.scala:30)
        at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.$anonfun$applyOrElse$1(BoundAttribute.scala:81)
        at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
        ... 144 more
      

      Note that when I cast the struct type it works:

      import org.apache.spark.sql.types._
      test.select(
          $"best_name".cast(StructType(Seq(StructField("count", LongType), StructField("name", StringType))))("name")
      ).show(10, false)
      
      +--------------+
      |best_name.name|
      +--------------+
      |bob           |
      |sam           |
      |bob           |
      +--------------+
      

      Attachments

        Activity

          People

            viirya L. C. Hsieh
            dsolow1 Daniel Solow
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: