Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-11012 Canonicalize view definitions
  3. SPARK-12725

SQL generation suffers from name conficts introduced by some analysis rules

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.0.0
    • SQL
    • None

    Description

      Some analysis rules generate auxiliary attribute references with the same name but different expression IDs. For example, ResolveAggregateFunctions introduces havingCondition and aggOrder, and DistinctAggregationRewriter introduces gid.

      This is OK for normal query execution since these attribute references get expression IDs. However, it's troublesome when converting resolved query plans back to SQL query strings since expression IDs are erased.

      Here's an example Spark 1.6.0 snippet for illustration:

      sqlContext.range(10).select('id as 'a, 'id as 'b).registerTempTable("t")
      sqlContext.sql("SELECT SUM(a) FROM t GROUP BY a, b ORDER BY COUNT(a), COUNT(b)").explain(true)
      

      The above code produces the following resolved plan:

      == Analyzed Logical Plan ==
      _c0: bigint
      Project [_c0#101L]
      +- Sort [aggOrder#102L ASC,aggOrder#103L ASC], true
         +- Aggregate [a#47L,b#48L], [(sum(a#47L),mode=Complete,isDistinct=false) AS _c0#101L,(count(a#47L),mode=Complete,isDistinct=false) AS aggOrder#102L,(count(b#48L),mode=Complete,isDistinct=false) AS aggOrder#103L]
            +- Subquery t
               +- Project [id#46L AS a#47L,id#46L AS b#48L]
                  +- LogicalRDD [id#46L], MapPartitionsRDD[44] at range at <console>:26
      

      Here we can see that both aggregate expressions in ORDER BY are extracted into an Aggregate operator, and both of them are named aggOrder with different expression IDs.

      Attachments

        Activity

          People

            smilegator Xiao Li
            lian cheng Cheng Lian
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: