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

WidenSetOperationTypes in subquery attribute missing

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.3.4, 2.4.5, 3.0.0
    • 3.0.2, 3.1.0
    • SQL
    • None

    Description

      I am migrating sql from mysql to spark sql, meet a very strange case. Below is code to reproduce the exception:

       

      val spark = SparkSession.builder()
       .master("local")
       .appName("Word Count")
       .getOrCreate()
      spark.sparkContext.setLogLevel("TRACE")
      val DecimalType = DataTypes.createDecimalType(20, 2)
      val schema = StructType(List(
       StructField("a", DecimalType, true)
      ))
      val dataList = new util.ArrayList[Row]()
      
      
      val df=spark.createDataFrame(dataList,schema)
      df.printSchema()
      df.createTempView("test")
      val sql=
       """
       |SELECT t.kpi_04 FROM
       |(
       | SELECT a as `kpi_04` FROM test
       | UNION ALL
       | SELECT a+a as `kpi_04` FROM test
       |) t
       |
       """.stripMargin
      spark.sql(sql)
      

       

      Exception Message:

       

      Exception in thread "main" org.apache.spark.sql.AnalysisException: Resolved attribute(s) kpi_04#2 missing from kpi_04#4 in operator !Project [kpi_04#2]. Attribute(s) with the same name appear in the operation: kpi_04. Please check if the right attribute(s) are used.;;
      !Project [kpi_04#2]
      +- SubqueryAlias t
       +- Union
       :- Project [cast(kpi_04#2 as decimal(21,2)) AS kpi_04#4]
       : +- Project [a#0 AS kpi_04#2]
       : +- SubqueryAlias test
       : +- LocalRelation <empty>, [a#0]
       +- Project [kpi_04#3]
       +- Project [CheckOverflow((promote_precision(cast(a#0 as decimal(21,2))) + promote_precision(cast(a#0 as decimal(21,2)))), DecimalType(21,2)) AS kpi_04#3]
       +- SubqueryAlias test
       +- LocalRelation <empty>, [a#0]

       

       

      Base the trace log ,seemly the WidenSetOperationTypes add new outer project layer. It caused the parent query lose the reference to subquery. 

       

       

       
      === Applying Rule org.apache.spark.sql.catalyst.analysis.TypeCoercion$WidenSetOperationTypes ===
      !'Project [kpi_04#2] !Project [kpi_04#2]
      !+- 'SubqueryAlias t +- SubqueryAlias t
      ! +- 'Union +- Union
      ! :- Project [a#0 AS kpi_04#2] :- Project [cast(kpi_04#2 as decimal(21,2)) AS kpi_04#4]
      ! : +- SubqueryAlias test : +- Project [a#0 AS kpi_04#2]
      ! : +- LocalRelation <empty>, [a#0] : +- SubqueryAlias test
      ! +- Project [CheckOverflow((promote_precision(cast(a#0 as decimal(21,2))) + promote_precision(cast(a#0 as decimal(21,2)))), DecimalType(21,2)) AS kpi_04#3] : +- LocalRelation <empty>, [a#0]
      ! +- SubqueryAlias test +- Project [kpi_04#3]
      ! +- LocalRelation <empty>, [a#0] +- Project [CheckOverflow((promote_precision(cast(a#0 as decimal(21,2))) + promote_precision(cast(a#0 as decimal(21,2)))), DecimalType(21,2)) AS kpi_04#3]
      ! +- SubqueryAlias test
      ! +- LocalRelation <empty>, [a#0]
      

       

        in the source code ,WidenSetOperationTypes.scala. it is  a intent behavior, but  possibly  miss this edge case. 

      I hope someone can help me out to fix it . 

       

       

      if (targetTypes.nonEmpty) {
       // Add an extra Project if the targetTypes are different from the original types.
       children.map(widenTypes(_, targetTypes))
      } else {
       // Unable to find a target type to widen, then just return the original set.
       children
      }

       

       

       

       

       

       

      Attachments

        Activity

          People

            maropu Takeshi Yamamuro
            kkyong Guojian Li
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: