You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by yu...@apache.org on 2023/06/03 05:19:34 UTC

[spark] branch master updated: [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc]

This is an automated email from the ASF dual-hosted git repository.

yumwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 898ad77900d [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc]
898ad77900d is described below

commit 898ad77900d887ac64800a616bd382def816eea6
Author: Jiaan Geng <be...@163.com>
AuthorDate: Sat Jun 3 13:19:09 2023 +0800

    [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc]
    
    ### What changes were proposed in this pull request?
    Last year, I committed `Percentile[Cont|Disc]` functions for Spark SQL.
    Recently, I found the sql method of `Percentile[Cont|Disc]` doesn't display column's sql suitably.
    This PR will fix the bug.
    
    ### Why are the changes needed?
    Fix the bug doesn't display column's sql for `Percentile[Cont|Disc]`.
    
    ### Does this PR introduce _any_ user-facing change?
    'Yes'.
    Users could see the correct sql information.
    
    ### How was this patch tested?
    Test cases updated.
    
    Closes #41436 from beliefer/SPARK-37676_SPARK-37691_followup.
    
    Authored-by: Jiaan Geng <be...@163.com>
    Signed-off-by: Yuming Wang <yu...@ebay.com>
---
 .../expressions/aggregate/percentiles.scala        |  4 +--
 .../sql-tests/analyzer-results/percentiles.sql.out | 36 +++++++++++-----------
 .../postgreSQL/aggregates_part4.sql.out            |  8 ++---
 .../udf/postgreSQL/udf-aggregates_part4.sql.out    |  8 ++---
 .../sql-tests/results/percentiles.sql.out          | 24 +++++++--------
 .../results/postgreSQL/aggregates_part4.sql.out    |  8 ++---
 .../udf/postgreSQL/udf-aggregates_part4.sql.out    |  8 ++---
 7 files changed, 48 insertions(+), 48 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
index 6d54c52b918..b26502a1763 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala
@@ -364,7 +364,7 @@ case class PercentileCont(left: Expression, right: Expression, reverse: Boolean
   override def sql(isDistinct: Boolean): String = {
     val distinct = if (isDistinct) "DISTINCT " else ""
     val direction = if (reverse) " DESC" else ""
-    s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY v$direction)"
+    s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY ${left.sql}$direction)"
   }
   override protected def withNewChildrenInternal(
       newLeft: Expression, newRight: Expression): PercentileCont =
@@ -404,7 +404,7 @@ case class PercentileDisc(
   override def sql(isDistinct: Boolean): String = {
     val distinct = if (isDistinct) "DISTINCT " else ""
     val direction = if (reverse) " DESC" else ""
-    s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY v$direction)"
+    s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY ${left.sql}$direction)"
   }
 
   override protected def withNewChildrenInternal(
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out
index 9c945687830..7a72df5a3e3 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out
@@ -182,9 +182,9 @@ FROM basic_pays
 ORDER BY salary
 -- !query analysis
 Sort [salary#x ASC NULLS FIRST], true
-+- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0 [...]
-   +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_dis [...]
-      +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollow [...]
++- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, pe [...]
+   +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, [...]
+      +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedf [...]
          +- Project [employee_name#x, department#x, salary#x]
             +- SubqueryAlias basic_pays
                +- View (`basic_pays`, [employee_name#x,department#x,salary#x])
@@ -362,9 +362,9 @@ WINDOW w AS (PARTITION BY department)
 ORDER BY salary
 -- !query analysis
 Sort [salary#x ASC NULLS FIRST], true
-+- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0 [...]
-   +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_dis [...]
-      +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollow [...]
++- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, pe [...]
+   +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, [...]
+      +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(salary#x, cast(0.25 as double), false, 0, 0) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedf [...]
          +- Project [employee_name#x, department#x, salary#x]
             +- SubqueryAlias basic_pays
                +- View (`basic_pays`, [employee_name#x,department#x,salary#x])
@@ -390,9 +390,9 @@ WINDOW w AS (PARTITION BY department)
 ORDER BY salary
 -- !query analysis
 Sort [salary#x ASC NULLS FIRST], true
-+- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OVER  [...]
-   +- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY v DESC) OV [...]
-      +- Window [median(salary#x) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.5 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY  [...]
++- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY sal [...]
+   +- Project [employee_name#x, department#x, salary#x, median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.5) WITHIN GROUP (ORDER BY  [...]
+      +- Window [median(salary#x) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(salary#x, cast(0.5 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITIO [...]
          +- Project [employee_name#x, department#x, salary#x]
             +- Filter (salary#x > 8900)
                +- SubqueryAlias basic_pays
@@ -606,7 +606,7 @@ SELECT
   percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC)
 FROM intervals
 -- !query analysis
-Aggregate [percentile_cont(dt#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(dt#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
+Aggregate [percentile_cont(dt#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY dt)#x, percentile_cont(dt#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC)#x]
 +- SubqueryAlias intervals
    +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
       +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -625,7 +625,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, percentile_cont(ym#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(ym#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
++- Aggregate [k#x], [k#x, percentile_cont(ym#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY ym)#x, percentile_cont(ym#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -644,7 +644,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, percentile_cont(dt2#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_cont(dt2#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
++- Aggregate [k#x], [k#x, percentile_cont(dt2#x, cast(0.25 as double), false) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2)#x, percentile_cont(dt2#x, cast(0.25 as double), true) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -659,7 +659,7 @@ SELECT
   percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC)
 FROM intervals
 -- !query analysis
-Aggregate [percentile_disc(dt#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(dt#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
+Aggregate [percentile_disc(dt#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY dt)#x, percentile_disc(dt#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC)#x]
 +- SubqueryAlias intervals
    +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
       +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -678,7 +678,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, percentile_disc(ym#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(ym#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
++- Aggregate [k#x], [k#x, percentile_disc(ym#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY ym)#x, percentile_disc(ym#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -697,7 +697,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, percentile_disc(dt2#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v)#x, percentile_disc(dt2#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)#x]
++- Aggregate [k#x], [k#x, percentile_disc(dt2#x, cast(0.25 as double), false, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2)#x, percentile_disc(dt2#x, cast(0.25 as double), true, 0, 0) AS percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -713,7 +713,7 @@ SELECT
   percentile_cont(0.5) WITHIN GROUP (ORDER BY dt)
 FROM intervals
 -- !query analysis
-Aggregate [median(dt#x) AS median(dt)#x, percentile(dt#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt, 0.5, 1)#x, percentile_cont(dt#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [median(dt#x) AS median(dt)#x, percentile(dt#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt, 0.5, 1)#x, percentile_cont(dt#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY dt)#x]
 +- SubqueryAlias intervals
    +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
       +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -733,7 +733,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, median(ym#x) AS median(ym)#x, percentile(ym#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(ym, 0.5, 1)#x, percentile_cont(ym#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
++- Aggregate [k#x], [k#x, median(ym#x) AS median(ym)#x, percentile(ym#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(ym, 0.5, 1)#x, percentile_cont(ym#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY ym)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
@@ -753,7 +753,7 @@ GROUP BY k
 ORDER BY k
 -- !query analysis
 Sort [k#x ASC NULLS FIRST], true
-+- Aggregate [k#x], [k#x, median(dt2#x) AS median(dt2)#x, percentile(dt2#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt2, 0.5, 1)#x, percentile_cont(dt2#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
++- Aggregate [k#x], [k#x, median(dt2#x) AS median(dt2)#x, percentile(dt2#x, cast(0.5 as double), 1, 0, 0, false) AS percentile(dt2, 0.5, 1)#x, percentile_cont(dt2#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2)#x]
    +- SubqueryAlias intervals
       +- View (`intervals`, [k#x,dt#x,ym#x,dt2#x])
          +- Project [cast(k#x as int) AS k#x, cast(dt#x as interval month) AS dt#x, cast(ym#x as interval second) AS ym#x, cast(dt2#x as interval minute) AS dt2#x]
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out
index 6b33408ba1a..11670e7056c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/aggregates_part4.sql.out
@@ -2,7 +2,7 @@
 -- !query
 select percentile_cont(0.5) within group (order by b) from aggtest
 -- !query analysis
-Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY b)#x]
 +- SubqueryAlias spark_catalog.default.aggtest
    +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet
 
@@ -10,7 +10,7 @@ Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0
 -- !query
 select percentile_cont(0.5) within group (order by b), sum(b) from aggtest
 -- !query analysis
-Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x, sum(b#x) AS sum(b)#x]
+Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY b)#x, sum(b#x) AS sum(b)#x]
 +- SubqueryAlias spark_catalog.default.aggtest
    +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet
 
@@ -18,7 +18,7 @@ Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0
 -- !query
 select percentile_cont(0.5) within group (order by thousand) from tenk1
 -- !query analysis
-Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY thousand)#x]
 +- SubqueryAlias spark_catalog.default.tenk1
    +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
 
@@ -26,6 +26,6 @@ Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile
 -- !query
 select percentile_disc(0.5) within group (order by thousand) from tenk1
 -- !query analysis
-Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY thousand)#x]
 +- SubqueryAlias spark_catalog.default.tenk1
    +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out
index 6b33408ba1a..11670e7056c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-aggregates_part4.sql.out
@@ -2,7 +2,7 @@
 -- !query
 select percentile_cont(0.5) within group (order by b) from aggtest
 -- !query analysis
-Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY b)#x]
 +- SubqueryAlias spark_catalog.default.aggtest
    +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet
 
@@ -10,7 +10,7 @@ Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0
 -- !query
 select percentile_cont(0.5) within group (order by b), sum(b) from aggtest
 -- !query analysis
-Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x, sum(b#x) AS sum(b)#x]
+Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY b)#x, sum(b#x) AS sum(b)#x]
 +- SubqueryAlias spark_catalog.default.aggtest
    +- Relation spark_catalog.default.aggtest[a#x,b#x] parquet
 
@@ -18,7 +18,7 @@ Aggregate [percentile_cont(b#x, cast(0.5 as double), false) AS percentile_cont(0
 -- !query
 select percentile_cont(0.5) within group (order by thousand) from tenk1
 -- !query analysis
-Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile_cont(0.5) WITHIN GROUP (ORDER BY thousand)#x]
 +- SubqueryAlias spark_catalog.default.tenk1
    +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
 
@@ -26,6 +26,6 @@ Aggregate [percentile_cont(thousand#x, cast(0.5 as double), false) AS percentile
 -- !query
 select percentile_disc(0.5) within group (order by thousand) from tenk1
 -- !query analysis
-Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY v)#x]
+Aggregate [percentile_disc(thousand#x, cast(0.5 as double), false, 0, 0) AS percentile_disc(0.5) WITHIN GROUP (ORDER BY thousand)#x]
 +- SubqueryAlias spark_catalog.default.tenk1
    +- Relation spark_catalog.default.tenk1[unique1#x,unique2#x,two#x,four#x,ten#x,twenty#x,hundred#x,thousand#x,twothousand#x,fivethous#x,tenthous#x,odd#x,even#x,stringu1#x,stringu2#x,string4#x] parquet
diff --git a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
index 1e307d9a061..2b7a36395bf 100644
--- a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
@@ -144,7 +144,7 @@ SELECT
 FROM basic_pays
 ORDER BY salary
 -- !query schema
-struct<employee_name:string,department:string,salary:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,p [...]
+struct<employee_name:string,department:string,salary:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLL [...]
 -- !query output
 Leslie Thompson	IT	5186	5917.75	5186.0	7381.25	8113.0
 Anthony Bow	Accounting	6627	8543.75	8435.0	9746.5	9998.0
@@ -344,7 +344,7 @@ FROM basic_pays
 WINDOW w AS (PARTITION BY department)
 ORDER BY salary
 -- !query schema
-struct<employee_name:string,department:string,salary:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,p [...]
+struct<employee_name:string,department:string,salary:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLL [...]
 -- !query output
 Leslie Thompson	IT	5186	5917.75	5186.0	7381.25	8113.0
 Anthony Bow	Accounting	6627	8543.75	8435.0	9746.5	9998.0
@@ -380,7 +380,7 @@ WHERE salary > 8900
 WINDOW w AS (PARTITION BY department)
 ORDER BY salary
 -- !query schema
-struct<employee_name:string,department:string,salary:int,median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.5) WITHIN GROUP (ORDER  [...]
+struct<employee_name:string,department:string,salary:int,median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING):double,percentile_cont(0.5) WITHIN GRO [...]
 -- !query output
 Jeff Firrelli	Accounting	8992	9998.0	9998.0	9998.0	9998.0	9998.0
 Julie Firrelli	Sales	9181	9441.0	9441.0	9441.0	9441.0	9441.0
@@ -594,7 +594,7 @@ SELECT
   percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC)
 FROM intervals
 -- !query schema
-struct<percentile_cont(0.25) WITHIN GROUP (ORDER BY v):interval year to month,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):interval year to month>
+struct<percentile_cont(0.25) WITHIN GROUP (ORDER BY dt):interval year to month,percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC):interval year to month>
 -- !query output
 0-10	2-6
 
@@ -608,7 +608,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY v):interval day to second,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):interval day to second>
+struct<k:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY ym):interval day to second,percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC):interval day to second>
 -- !query output
 0	0 00:00:10.000000000	0 00:00:30.000000000
 1	0 00:00:12.500000000	0 00:00:17.500000000
@@ -626,7 +626,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY v):interval day to second,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):interval day to second>
+struct<k:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2):interval day to second,percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC):interval day to second>
 -- !query output
 0	0 00:10:00.000000000	0 00:30:00.000000000
 1	0 00:12:30.000000000	0 00:17:30.000000000
@@ -641,7 +641,7 @@ SELECT
   percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC)
 FROM intervals
 -- !query schema
-struct<percentile_disc(0.25) WITHIN GROUP (ORDER BY v):interval year to month,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):interval year to month>
+struct<percentile_disc(0.25) WITHIN GROUP (ORDER BY dt):interval year to month,percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC):interval year to month>
 -- !query output
 0-10	2-6
 
@@ -655,7 +655,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,percentile_disc(0.25) WITHIN GROUP (ORDER BY v):interval day to second,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):interval day to second>
+struct<k:int,percentile_disc(0.25) WITHIN GROUP (ORDER BY ym):interval day to second,percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC):interval day to second>
 -- !query output
 0	0 00:00:10.000000000	0 00:00:30.000000000
 1	0 00:00:10.000000000	0 00:00:20.000000000
@@ -673,7 +673,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,percentile_disc(0.25) WITHIN GROUP (ORDER BY v):interval day to second,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):interval day to second>
+struct<k:int,percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2):interval day to second,percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC):interval day to second>
 -- !query output
 0	0 00:10:00.000000000	0 00:30:00.000000000
 1	0 00:10:00.000000000	0 00:20:00.000000000
@@ -689,7 +689,7 @@ SELECT
   percentile_cont(0.5) WITHIN GROUP (ORDER BY dt)
 FROM intervals
 -- !query schema
-struct<median(dt):interval year to month,percentile(dt, 0.5, 1):interval year to month,percentile_cont(0.5) WITHIN GROUP (ORDER BY v):interval year to month>
+struct<median(dt):interval year to month,percentile(dt, 0.5, 1):interval year to month,percentile_cont(0.5) WITHIN GROUP (ORDER BY dt):interval year to month>
 -- !query output
 1-8	1-8	1-8
 
@@ -704,7 +704,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,median(ym):interval day to second,percentile(ym, 0.5, 1):interval day to second,percentile_cont(0.5) WITHIN GROUP (ORDER BY v):interval day to second>
+struct<k:int,median(ym):interval day to second,percentile(ym, 0.5, 1):interval day to second,percentile_cont(0.5) WITHIN GROUP (ORDER BY ym):interval day to second>
 -- !query output
 0	0 00:00:20.000000000	0 00:00:20.000000000	0 00:00:20.000000000
 1	0 00:00:15.000000000	0 00:00:15.000000000	0 00:00:15.000000000
@@ -723,7 +723,7 @@ FROM intervals
 GROUP BY k
 ORDER BY k
 -- !query schema
-struct<k:int,median(dt2):interval day to second,percentile(dt2, 0.5, 1):interval day to second,percentile_cont(0.5) WITHIN GROUP (ORDER BY v):interval day to second>
+struct<k:int,median(dt2):interval day to second,percentile(dt2, 0.5, 1):interval day to second,percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2):interval day to second>
 -- !query output
 0	0 00:20:00.000000000	0 00:20:00.000000000	0 00:20:00.000000000
 1	0 00:15:00.000000000	0 00:15:00.000000000	0 00:15:00.000000000
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out
index 45f00b36f16..1aaa514eb13 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part4.sql.out
@@ -2,7 +2,7 @@
 -- !query
 select percentile_cont(0.5) within group (order by b) from aggtest
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY b):double>
 -- !query output
 53.44850015640259
 
@@ -10,7 +10,7 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
 -- !query
 select percentile_cont(0.5) within group (order by b), sum(b) from aggtest
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double,sum(b):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY b):double,sum(b):double>
 -- !query output
 53.44850015640259	431.77260909229517
 
@@ -18,7 +18,7 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double,sum(b):double>
 -- !query
 select percentile_cont(0.5) within group (order by thousand) from tenk1
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY thousand):double>
 -- !query output
 499.5
 
@@ -26,6 +26,6 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
 -- !query
 select percentile_disc(0.5) within group (order by thousand) from tenk1
 -- !query schema
-struct<percentile_disc(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_disc(0.5) WITHIN GROUP (ORDER BY thousand):double>
 -- !query output
 499.0
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out
index 45f00b36f16..1aaa514eb13 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out
@@ -2,7 +2,7 @@
 -- !query
 select percentile_cont(0.5) within group (order by b) from aggtest
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY b):double>
 -- !query output
 53.44850015640259
 
@@ -10,7 +10,7 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
 -- !query
 select percentile_cont(0.5) within group (order by b), sum(b) from aggtest
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double,sum(b):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY b):double,sum(b):double>
 -- !query output
 53.44850015640259	431.77260909229517
 
@@ -18,7 +18,7 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double,sum(b):double>
 -- !query
 select percentile_cont(0.5) within group (order by thousand) from tenk1
 -- !query schema
-struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY thousand):double>
 -- !query output
 499.5
 
@@ -26,6 +26,6 @@ struct<percentile_cont(0.5) WITHIN GROUP (ORDER BY v):double>
 -- !query
 select percentile_disc(0.5) within group (order by thousand) from tenk1
 -- !query schema
-struct<percentile_disc(0.5) WITHIN GROUP (ORDER BY v):double>
+struct<percentile_disc(0.5) WITHIN GROUP (ORDER BY thousand):double>
 -- !query output
 499.0


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org