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

Query produces results in incorrect order when a composite order by clause refers to both original columns and aliases

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.3.0
    • 2.2.2, 2.3.0
    • SQL
    • None

    Description

      Here is the test snippet.

      scala> Seq[(Integer, Integer)](
           |         (1, 1),
           |         (1, 3),
           |         (2, 3),
           |         (3, 3),
           |         (4, null),
           |         (5, null)
           |       ).toDF("key", "value").createOrReplaceTempView("src")
      
      scala> sql(
           |         """
           |           |SELECT MAX(value) as value, key as col2
           |           |FROM src
           |           |GROUP BY key
           |           |ORDER BY value desc, key
           |         """.stripMargin).show
      +-----+----+
      |value|col2|
      +-----+----+
      |    3|   3|
      |    3|   2|
      |    3|   1|
      | null|   5|
      | null|   4|
      +-----+----+
      

      Here is the explain output :

      == Parsed Logical Plan ==
      'Sort ['value DESC NULLS LAST, 'key ASC NULLS FIRST], true
      +- 'Aggregate ['key], ['MAX('value) AS value#9, 'key AS col2#10]
         +- 'UnresolvedRelation `src`
      
      == Analyzed Logical Plan ==
      value: int, col2: int
      Project [value#9, col2#10]
      +- Sort [value#9 DESC NULLS LAST, col2#10 DESC NULLS LAST], true
         +- Aggregate [key#5], [max(value#6) AS value#9, key#5 AS col2#10]
            +- SubqueryAlias src
               +- Project [_1#2 AS key#5, _2#3 AS value#6]
                  +- LocalRelation [_1#2, _2#3]
      

      The sort direction should be ascending for the 2nd column. Instead its being changed
      to descending in Analyzer.resolveAggregateFunctions.

      The above testcase models TPCDS-Q71 and thus we have the same issue in Q71 as well.

      Attachments

        Activity

          People

            dkbiswal Dilip Biswal
            dkbiswal Dilip Biswal
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: