From f4e081a80aad14b4f1d455288fa3875d5e15f66a Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Fri, 9 Oct 2020 15:19:39 +0800 Subject: [PATCH 01/14] [SPARK-13860][SQL] change stddev_samp and var_samp to return 0.0 instead of Double.NaN to align with TPCDS standard. Change-Id: Ia8c4bd6086a17d39dd7d843801fb2b6036cb1756 --- .../aggregate/CentralMomentAgg.scala | 7 ++- .../apache/spark/sql/internal/SQLConf.scala | 12 +++++ .../spark/sql/DataFrameAggregateSuite.scala | 44 +++++++++++-------- 3 files changed, 42 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 53759ca3d9165..1a3692711170e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -174,7 +175,8 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, Double.NaN, sqrt(m2 / (n - 1.0)))) + If(n === 1.0, if (SQLConf.get.legacyCentralMomentAggBehavior) Double.NaN else 0.0, + sqrt(m2 / (n - 1.0)))) } override def prettyName: String = @@ -218,7 +220,8 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, Double.NaN, m2 / (n - 1.0))) + If(n === 1.0, if (SQLConf.get.legacyCentralMomentAggBehavior) Double.NaN else 0.0, + m2 / (n - 1.0))) } override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("var_samp") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8cbdbfe16d2bc..95c213028d7fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2784,6 +2784,16 @@ object SQLConf { .stringConf .createWithDefault("") + val LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR = + buildConf("spark.sql.legacy.centralMomentAgg.enabled") + .internal() + .doc("When set to true, stddev_samp and var_samp will return Double.NaN, " + + "if applied to a set with a single element. Otherwise, will return 0.0, " + + "which is aligned with TPCDS standard.") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -3410,6 +3420,8 @@ class SQLConf extends Serializable with Logging { def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST) + def legacyCentralMomentAggBehavior: Boolean = getConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 353444b664412..af7688e03e03f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -456,25 +456,31 @@ class DataFrameAggregateSuite extends QueryTest } test("zero moments") { - val input = Seq((1, 2)).toDF("a", "b") - checkAnswer( - input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), - var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) - - checkAnswer( - input.agg( - expr("stddev(a)"), - expr("stddev_samp(a)"), - expr("stddev_pop(a)"), - expr("variance(a)"), - expr("var_samp(a)"), - expr("var_pop(a)"), - expr("skewness(a)"), - expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, - Double.NaN, Double.NaN)) + Seq(true, false).foreach { legacyCentralMomentAggBehavior => + withSQLConf( + SQLConf.LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR.key -> legacyCentralMomentAggBehavior.toString) { + val result: Double = if (legacyCentralMomentAggBehavior) Double.NaN else 0.0 + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), + Row(result, result, 0.0, result, result, 0.0, + Double.NaN, Double.NaN)) + + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(result, result, 0.0, result, result, 0.0, + Double.NaN, Double.NaN)) + } + } } test("null moments") { From 22956bbee208c90a36cd00da7cdd9a24f5d980b0 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Sat, 10 Oct 2020 11:10:23 +0800 Subject: [PATCH 02/14] return null instead of 0.0, updated UT and SQLConf desc. Change-Id: I41f3abf94bfe00548884cfe2cef5319d52dd37a1 --- .../aggregate/CentralMomentAgg.scala | 14 +- .../apache/spark/sql/internal/SQLConf.scala | 24 +-- .../spark/sql/DataFrameAggregateSuite.scala | 68 +++++--- .../sql/hive/execution/WindowQuerySuite.scala | 156 ++++++++++++------ 4 files changed, 174 insertions(+), 88 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 1a3692711170e..722661260e92a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -175,7 +175,8 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, if (SQLConf.get.legacyCentralMomentAggBehavior) Double.NaN else 0.0, + If(n === 1.0, + if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), sqrt(m2 / (n - 1.0)))) } @@ -220,7 +221,8 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, if (SQLConf.get.legacyCentralMomentAggBehavior) Double.NaN else 0.0, + If(n === 1.0, + if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), m2 / (n - 1.0))) } @@ -246,7 +248,9 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(m2 === 0.0, Double.NaN, sqrt(n) * m3 / sqrt(m2 * m2 * m2))) + If(m2 === 0.0, + if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), + sqrt(n) * m3 / sqrt(m2 * m2 * m2))) } } @@ -267,7 +271,9 @@ case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(m2 === 0.0, Double.NaN, n * m4 / (m2 * m2) - 3.0)) + If(m2 === 0.0, + if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), + n * m4 / (m2 * m2) - 3.0)) } override def prettyName: String = "kurtosis" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 95c213028d7fd..f7890401119fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2342,6 +2342,16 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_CENTRAL_MOMENT_AGG = + buildConf("spark.sql.legacy.centralMomentAgg") + .internal() + .doc("When set to true, central moment aggregation will return Double.NaN " + + "if divide by zero occurred during calculation. " + + "Otherwise, it will return null") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + val TRUNCATE_TABLE_IGNORE_PERMISSION_ACL = buildConf("spark.sql.truncateTable.ignorePermissionAcl.enabled") .internal() @@ -2784,16 +2794,6 @@ object SQLConf { .stringConf .createWithDefault("") - val LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR = - buildConf("spark.sql.legacy.centralMomentAgg.enabled") - .internal() - .doc("When set to true, stddev_samp and var_samp will return Double.NaN, " + - "if applied to a set with a single element. Otherwise, will return 0.0, " + - "which is aligned with TPCDS standard.") - .version("3.1.0") - .booleanConf - .createWithDefault(false) - /** * Holds information about keys that have been deprecated. * @@ -3374,6 +3374,8 @@ class SQLConf extends Serializable with Logging { def allowNegativeScaleOfDecimalEnabled: Boolean = getConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED) + def legacyCentralMomentAgg: Boolean = getConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG) + def truncateTableIgnorePermissionAcl: Boolean = getConf(SQLConf.TRUNCATE_TABLE_IGNORE_PERMISSION_ACL) @@ -3420,8 +3422,6 @@ class SQLConf extends Serializable with Logging { def disabledJdbcConnectionProviders: String = getConf(SQLConf.DISABLED_JDBC_CONN_PROVIDER_LIST) - def legacyCentralMomentAggBehavior: Boolean = getConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index af7688e03e03f..a717b2343117d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -456,30 +456,50 @@ class DataFrameAggregateSuite extends QueryTest } test("zero moments") { - Seq(true, false).foreach { legacyCentralMomentAggBehavior => - withSQLConf( - SQLConf.LEGACY_CENTRAL_MOMENT_AGG_BEHAVIOR.key -> legacyCentralMomentAggBehavior.toString) { - val result: Double = if (legacyCentralMomentAggBehavior) Double.NaN else 0.0 - val input = Seq((1, 2)).toDF("a", "b") - checkAnswer( - input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), - var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), - Row(result, result, 0.0, result, result, 0.0, - Double.NaN, Double.NaN)) - - checkAnswer( - input.agg( - expr("stddev(a)"), - expr("stddev_samp(a)"), - expr("stddev_pop(a)"), - expr("variance(a)"), - expr("var_samp(a)"), - expr("var_pop(a)"), - expr("skewness(a)"), - expr("kurtosis(a)")), - Row(result, result, 0.0, result, result, 0.0, - Double.NaN, Double.NaN)) - } + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) + + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) + } + } + + test("SPARK-13860: zero moments LEGACY_CENTRAL_MOMENT_AGG off") { + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { + val input = Seq((1, 2)).toDF("a", "b") + checkAnswer( + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), + Row(null, null, 0.0, null, null, 0.0, + null, null)) + + checkAnswer( + input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), + expr("variance(a)"), + expr("var_samp(a)"), + expr("var_pop(a)"), + expr("skewness(a)"), + expr("kurtosis(a)")), + Row(null, null, 0.0, null, null, 0.0, + null, null)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala index 15712a18ce751..6e35876937d1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils /** @@ -59,56 +60,115 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("windowing.q -- 15. testExpressions") { - // Moved because: - // - Spark uses a different default stddev (sample instead of pop) - // - Tiny numerical differences in stddev results. - // - Different StdDev behavior when n=1 (NaN instead of 0) - checkAnswer(sql(s""" - |select p_mfgr,p_name, p_size, - |rank() over(distribute by p_mfgr sort by p_name) as r, - |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, - |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, - |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, - |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, - |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, - |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, - |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, - |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, - |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, - |first_value(p_size) over w1 as fvW1 - |from part - |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name - | rows between 2 preceding and 2 following) + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { + // Moved because: + // - Spark uses a different default stddev (sample instead of pop) + // - Tiny numerical differences in stddev results. + // - Different StdDev behavior when n=1 (NaN instead of 0) + checkAnswer(sql( + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) """.stripMargin), - // scalastyle:off - Seq( - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), - Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), - Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), - Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), - Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, Double.NaN, 4, 14, 14), - Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), - Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), - Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), - Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), - Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0,Double.NaN, 2, 17, 17), - Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), - Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), - Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), - Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), - Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, Double.NaN, 0, 10, 10), - Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), - Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), - Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), - Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), - Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, Double.NaN, 1, 31, 31), - Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), - Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), - Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), - Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) + // scalastyle:off + Seq( + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, Double.NaN, 4, 14, 14), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, Double.NaN, 2, 17, 17), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), + Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, Double.NaN, 0, 10, 10), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, Double.NaN, 1, 31, 31), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) // scalastyle:on + } + } + + test("SPARK-13860: windowing.q -- 15. testExpressions LEGACY_CENTRAL_MOMENT_AGG off") { + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { + // Moved because: + // - Spark uses a different default stddev (sample instead of pop) + // - Tiny numerical differences in stddev results. + // - Different StdDev behavior when n=1 (NaN instead of 0) + checkAnswer(sql( + s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin), + // scalastyle:off + Seq( + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, null, 4, 14, 14), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, null, 2, 17, 17), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), + Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, null, 0, 10, 10), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, null, 1, 31, 31), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) + // scalastyle:on + } } test("windowing.q -- 20. testSTATs") { From d99e6472c06e69d6ed927aa19276f15788dc1500 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Sat, 10 Oct 2020 16:55:56 +0800 Subject: [PATCH 03/14] fix UT failures of change NaN into NULL. Change-Id: If6ea97ac4abeee5a72496aa985a38787d5eb9e52 --- .../postgreSQL/aggregates_part1.sql.out | 4 +- .../results/postgreSQL/window_part4.sql.out | 32 ++--- .../native/promoteStrings.sql.out | 8 +- .../postgreSQL/udf-aggregates_part1.sql.out | 4 +- .../sql-tests/results/udf/udf-window.sql.out | 6 +- .../sql-tests/results/window.sql.out | 8 +- .../sql/DataFrameWindowFunctionsSuite.scala | 111 ++++++++++++------ 7 files changed, 108 insertions(+), 65 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index f7bba96738eab..212365f92946c 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -143,7 +143,7 @@ SELECT var_pop(1.0), var_samp(2.0) -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query @@ -151,7 +151,7 @@ SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(3 -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 4dd4712345a89..f7439d873b4eb 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -195,7 +195,7 @@ struct -- !query output -NaN +NULL -- !query @@ -2558,7 +2558,7 @@ SELECT var_samp('1') FROM t -- !query schema struct -- !query output -NaN +NULL -- !query @@ -2566,7 +2566,7 @@ SELECT skewness('1') FROM t -- !query schema struct -- !query output -NaN +NULL -- !query @@ -2574,4 +2574,4 @@ SELECT kurtosis('1') FROM t -- !query schema struct -- !query output -NaN +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 76637bf578e6f..a428a7a9c923b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -143,7 +143,7 @@ SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query @@ -151,7 +151,7 @@ SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS -- !query schema struct -- !query output -0.0 NaN +0.0 NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index a84070535b658..01e4e60b4f92f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -289,12 +289,12 @@ ORDER BY cate, udf(val) struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NaN NULL 0.0 [3] [3] NULL NULL +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NaN 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NaN 1 NULL NULL NaN 0.0 [1] [1] NaN NaN +1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL 2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index a8875fd449bad..94eaeb5201f3c 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 29 +-- Number of queries: 32 -- !query @@ -313,12 +313,12 @@ ORDER BY cate, val struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NaN NULL 0.0 [3] [3] NULL NULL +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NaN 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NaN 1 NULL NULL NaN 0.0 [1] [1] NaN NaN +1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL 2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index c5dcdc44cc64f..2c287757df0aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -143,40 +143,83 @@ class DataFrameWindowFunctionsSuite extends QueryTest } test("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { - val df = Seq( - ("a", "p1", 10.0, 20.0), - ("b", "p1", 20.0, 10.0), - ("c", "p2", 20.0, 20.0), - ("d", "p2", 20.0, 20.0), - ("e", "p3", 0.0, 0.0), - ("f", "p3", 6.0, 12.0), - ("g", "p3", 6.0, 12.0), - ("h", "p3", 8.0, 16.0), - ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") - checkAnswer( - df.select( - $"key", - covar_samp("value1", "value2").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_samp("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - variance("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_samp("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev("value1").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) - ), - Seq( - Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), - Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), - Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), - Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), - Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("i", Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN))) + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("i", Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN))) + } + } + + test("SPARK-13860: " + + "covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window " + + "LEGACY_CENTRAL_MOMENT_AGG off") { + withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("i", Double.NaN, null, null, null, null))) + } } test("collect_list in ascending ordered window") { From c97ddc6bb7468de54d6bb81494a6f31b4a701c1f Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Sun, 11 Oct 2020 10:41:20 +0800 Subject: [PATCH 04/14] fix R UT failures. Change-Id: Ieb28cc15334c463de4f4e43719cf99eca48fffb1 --- R/pkg/tests/fulltests/test_sparkSQL.R | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 268f5734813ba..d789672a2f5a4 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -2147,7 +2147,7 @@ test_that("group by, agg functions", { df3 <- agg(gd, age = "stddev") expect_is(df3, "SparkDataFrame") df3_local <- collect(df3) - expect_true(is.nan(df3_local[df3_local$name == "Andy", ][1, 2])) + expect_true(is.null(df3_local[df3_local$name == "Andy", ][1, 2])) df4 <- agg(gd, sumAge = sum(df$age)) expect_is(df4, "SparkDataFrame") @@ -2178,7 +2178,7 @@ test_that("group by, agg functions", { df7 <- agg(gd2, value = "stddev") df7_local <- collect(df7) expect_true(abs(df7_local[df7_local$name == "ID1", ][1, 2] - 6.928203) < 1e-6) - expect_true(is.nan(df7_local[df7_local$name == "ID2", ][1, 2])) + expect_true(is.null(df7_local[df7_local$name == "ID2", ][1, 2])) mockLines3 <- c("{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Andy\", \"age\":30}", From ed5c6f008ac1c91214e45b8650e2b46aad4c2193 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 10:07:15 +0800 Subject: [PATCH 05/14] fix R ut failures and update SQLConf desc. Change-Id: I16c0074a6de22d1e165fa15a8f58b794e304b5b4 --- R/pkg/tests/fulltests/test_sparkSQL.R | 4 ++-- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index d789672a2f5a4..077dfc6770d94 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -2147,7 +2147,7 @@ test_that("group by, agg functions", { df3 <- agg(gd, age = "stddev") expect_is(df3, "SparkDataFrame") df3_local <- collect(df3) - expect_true(is.null(df3_local[df3_local$name == "Andy", ][1, 2])) + expect_true(is.na(df3_local[df3_local$name == "Andy", ][1, 2])) df4 <- agg(gd, sumAge = sum(df$age)) expect_is(df4, "SparkDataFrame") @@ -2178,7 +2178,7 @@ test_that("group by, agg functions", { df7 <- agg(gd2, value = "stddev") df7_local <- collect(df7) expect_true(abs(df7_local[df7_local$name == "ID1", ][1, 2] - 6.928203) < 1e-6) - expect_true(is.null(df7_local[df7_local$name == "ID2", ][1, 2])) + expect_true(is.na(df7_local[df7_local$name == "ID2", ][1, 2])) mockLines3 <- c("{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Andy\", \"age\":30}", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f7890401119fb..e53ec28cf590a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2346,8 +2346,8 @@ object SQLConf { buildConf("spark.sql.legacy.centralMomentAgg") .internal() .doc("When set to true, central moment aggregation will return Double.NaN " + - "if divide by zero occurred during calculation. " + - "Otherwise, it will return null") + "if divide by zero occurred during calculation. Otherwise, it will return null. " + + "Before version 3.1.0, it returns NaN in divideByZero case by default.") .version("3.1.0") .booleanConf .createWithDefault(false) From afc79115f41f7c546f14d21798bceaa186faeee8 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 11:11:59 +0800 Subject: [PATCH 06/14] update migration guide doc. Change-Id: Id43c92af199ed1ef99e9cd215f873d77492694f2 --- docs/sql-migration-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index feff2c7e9f543..e6659e3d1e799 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -24,6 +24,8 @@ license: | ## Upgrading from Spark SQL 3.0 to 3.1 + - In Spark 3.1, central moment aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.centralMomentAgg` to `true`. + - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`. - In Spark 3.1, SQL UI data adopts the `formatted` mode for the query plan explain results. To restore the behavior before Spark 3.1, you can set `spark.sql.ui.explainMode` to `extended`. From 5095df42051f18a5b752a85dd054db419cebb4f0 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 14:36:05 +0800 Subject: [PATCH 07/14] change SQLConf name to spark.sql.legacy.statisticalAggregate and include corr and covar_samp as well. update SQLConf desc, UT and migrate guide accordingly. Change-Id: Ifcff996964c0a40f10a7c24740e2dbaa74e46883 --- docs/sql-migration-guide.md | 2 +- .../aggregate/CentralMomentAgg.scala | 19 +-- .../catalyst/expressions/aggregate/Corr.scala | 6 +- .../expressions/aggregate/Covariance.scala | 6 +- .../apache/spark/sql/internal/SQLConf.scala | 10 +- .../spark/sql/DataFrameAggregateSuite.scala | 6 +- .../sql/DataFrameWindowFunctionsSuite.scala | 151 +++++++++++------ .../sql/hive/execution/WindowQuerySuite.scala | 155 ++++++------------ 8 files changed, 176 insertions(+), 179 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index e6659e3d1e799..c1de58d85d5bf 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -24,7 +24,7 @@ license: | ## Upgrading from Spark SQL 3.0 to 3.1 - - In Spark 3.1, central moment aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.centralMomentAgg` to `true`. + - In Spark 3.1, statistical aggregation function includes `std`, `stddev`, `stddev_samp`, `variance`, `var_samp`, `skewness`, `kurtosis`, `covar_samp`, `corr` will return `NULL` instead of `Double.NaN` when `DivideByZero` occurs during expression evaluation, for example, when `stddev_samp` applied on a single element set. In Spark version 3.0 and earlier, it will return `Double.NaN` in such case. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.statisticalAggregate` to `true`. - In Spark 3.1, grouping_id() returns long values. In Spark version 3.0 and earlier, this function returns int values. To restore the behavior before Spark 3.1, you can set `spark.sql.legacy.integerGroupingId` to `true`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 722661260e92a..1f1899132418c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -63,6 +63,9 @@ abstract class CentralMomentAgg(child: Expression) protected val m3 = AttributeReference("m3", DoubleType, nullable = false)() protected val m4 = AttributeReference("m4", DoubleType, nullable = false)() + protected val divideByZeroEvalResult: Expression = + if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + private def trimHigherOrder[T](expressions: Seq[T]) = expressions.take(momentOrder + 1) override val aggBufferAttributes = trimHigherOrder(Seq(n, avg, m2, m3, m4)) @@ -175,9 +178,7 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, - if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), - sqrt(m2 / (n - 1.0)))) + If(n === 1.0, divideByZeroEvalResult, sqrt(m2 / (n - 1.0)))) } override def prettyName: String = @@ -221,9 +222,7 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, - if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), - m2 / (n - 1.0))) + If(n === 1.0, divideByZeroEvalResult, m2 / (n - 1.0))) } override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("var_samp") @@ -248,9 +247,7 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(m2 === 0.0, - if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), - sqrt(n) * m3 / sqrt(m2 * m2 * m2))) + If(m2 === 0.0, divideByZeroEvalResult, sqrt(n) * m3 / sqrt(m2 * m2 * m2))) } } @@ -271,9 +268,7 @@ case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(m2 === 0.0, - if (SQLConf.get.legacyCentralMomentAgg) Double.NaN else Literal.create(null, DoubleType), - n * m4 / (m2 * m2) - 3.0)) + If(m2 === 0.0, divideByZeroEvalResult, n * m4 / (m2 * m2) - 3.0)) } override def prettyName: String = "kurtosis" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index 9ef05bb5d4fec..bfdb9dc88c56b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -43,6 +44,9 @@ abstract class PearsonCorrelation(x: Expression, y: Expression) protected val xMk = AttributeReference("xMk", DoubleType, nullable = false)() protected val yMk = AttributeReference("yMk", DoubleType, nullable = false)() + protected val divideByZeroEvalResult: Expression = + if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck, xMk, yMk) override val initialValues: Seq[Expression] = Array.fill(6)(Literal(0.0)) @@ -107,7 +111,7 @@ case class Corr(x: Expression, y: Expression) override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, Double.NaN, ck / sqrt(xMk * yMk))) + If(n === 1.0, divideByZeroEvalResult, ck / sqrt(xMk * yMk))) } override def prettyName: String = "corr" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index f03c2f2710a04..5929756678ec6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -38,6 +39,9 @@ abstract class Covariance(x: Expression, y: Expression) protected val yAvg = AttributeReference("yAvg", DoubleType, nullable = false)() protected val ck = AttributeReference("ck", DoubleType, nullable = false)() + protected val divideByZeroEvalResult: Expression = + if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck) override val initialValues: Seq[Expression] = Array.fill(4)(Literal(0.0)) @@ -108,7 +112,7 @@ case class CovPopulation(left: Expression, right: Expression) extends Covariance case class CovSample(left: Expression, right: Expression) extends Covariance(left, right) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), - If(n === 1.0, Double.NaN, ck / (n - 1.0))) + If(n === 1.0, divideByZeroEvalResult, ck / (n - 1.0))) } override def prettyName: String = "covar_samp" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e53ec28cf590a..e25c4e52e4e9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2342,11 +2342,11 @@ object SQLConf { .booleanConf .createWithDefault(false) - val LEGACY_CENTRAL_MOMENT_AGG = - buildConf("spark.sql.legacy.centralMomentAgg") + val LEGACY_STATISTICAL_AGGREGATE = + buildConf("spark.sql.legacy.statisticalAggregate") .internal() - .doc("When set to true, central moment aggregation will return Double.NaN " + - "if divide by zero occurred during calculation. Otherwise, it will return null. " + + .doc("When set to true, statistical aggregate function returns Double.NaN " + + "if divide by zero occurred during expression evaluation, otherwise, it returns null. " + "Before version 3.1.0, it returns NaN in divideByZero case by default.") .version("3.1.0") .booleanConf @@ -3374,7 +3374,7 @@ class SQLConf extends Serializable with Logging { def allowNegativeScaleOfDecimalEnabled: Boolean = getConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED) - def legacyCentralMomentAgg: Boolean = getConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG) + def legacyStatisticalAggregate: Boolean = getConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE) def truncateTableIgnorePermissionAcl: Boolean = getConf(SQLConf.TRUNCATE_TABLE_IGNORE_PERMISSION_ACL) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index a717b2343117d..d4e64aa03df0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -456,7 +456,7 @@ class DataFrameAggregateSuite extends QueryTest } test("zero moments") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { val input = Seq((1, 2)).toDF("a", "b") checkAnswer( input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), @@ -479,8 +479,8 @@ class DataFrameAggregateSuite extends QueryTest } } - test("SPARK-13860: zero moments LEGACY_CENTRAL_MOMENT_AGG off") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { + test("SPARK-13860: zero moments LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { val input = Seq((1, 2)).toDF("a", "b") checkAnswer( input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 2c287757df0aa..616e333033aa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -94,56 +94,111 @@ class DataFrameWindowFunctionsSuite extends QueryTest } test("corr, covar_pop, stddev_pop functions in specific window") { - val df = Seq( - ("a", "p1", 10.0, 20.0), - ("b", "p1", 20.0, 10.0), - ("c", "p2", 20.0, 20.0), - ("d", "p2", 20.0, 20.0), - ("e", "p3", 0.0, 0.0), - ("f", "p3", 6.0, 12.0), - ("g", "p3", 6.0, 12.0), - ("h", "p3", 8.0, 16.0), - ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") - checkAnswer( - df.select( - $"key", - corr("value1", "value2").over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - covar_pop("value1", "value2") - .over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_pop("value1") - .over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_pop("value1") - .over(Window.partitionBy("partitionId") + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - var_pop("value2") - .over(Window.partitionBy("partitionId") + covar_pop("value1", "value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("i", Double.NaN, 0.0, 0.0, 0.0, 0.0, 0.0))) + } + } + + test("SPARK-13860: " + + "corr, covar_pop, stddev_pop functions in specific window " + + "LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0), + ("i", "p4", 5.0, 5.0)).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over(Window.partitionBy("partitionId") .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), - stddev_pop("value2") - .over(Window.partitionBy("partitionId") - .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), - - // As stddev_pop(expr) = sqrt(var_pop(expr)) - // the "stddev_pop" column can be calculated from the "var_pop" column. - // - // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) - // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns. - Seq( - Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), - Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), - Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), - Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), - Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), - Row("i", Double.NaN, 0.0, 0.0, 0.0, 0.0, 0.0))) + covar_pop("value1", "value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over(Window.partitionBy("partitionId") + .orderBy("key").rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing))), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("i", null, 0.0, 0.0, 0.0, 0.0, 0.0))) + } } test("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { val df = Seq( ("a", "p1", 10.0, 20.0), ("b", "p1", 20.0, 10.0), @@ -183,8 +238,8 @@ class DataFrameWindowFunctionsSuite extends QueryTest test("SPARK-13860: " + "covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window " + - "LEGACY_CENTRAL_MOMENT_AGG off") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { + "LEGACY_STATISTICAL_AGGREGATE off") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "false") { val df = Seq( ("a", "p1", 10.0, 20.0), ("b", "p1", 20.0, 10.0), @@ -218,7 +273,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), - Row("i", Double.NaN, null, null, null, null))) + Row("i", null, null, null, null, null))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala index 6e35876937d1d..6bf7bd6cbb90e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils /** @@ -60,115 +59,55 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("windowing.q -- 15. testExpressions") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "true") { - // Moved because: - // - Spark uses a different default stddev (sample instead of pop) - // - Tiny numerical differences in stddev results. - // - Different StdDev behavior when n=1 (NaN instead of 0) - checkAnswer(sql( - s""" - |select p_mfgr,p_name, p_size, - |rank() over(distribute by p_mfgr sort by p_name) as r, - |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, - |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, - |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, - |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, - |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, - |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, - |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, - |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, - |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, - |first_value(p_size) over w1 as fvW1 - |from part - |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name - | rows between 2 preceding and 2 following) - """.stripMargin), - // scalastyle:off - Seq( - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), - Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), - Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), - Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), - Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, Double.NaN, 4, 14, 14), - Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), - Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), - Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), - Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), - Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, Double.NaN, 2, 17, 17), - Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), - Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), - Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), - Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), - Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, Double.NaN, 0, 10, 10), - Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), - Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), - Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), - Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), - Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, Double.NaN, 1, 31, 31), - Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), - Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), - Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), - Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) - // scalastyle:on - } - } - - test("SPARK-13860: windowing.q -- 15. testExpressions LEGACY_CENTRAL_MOMENT_AGG off") { - withSQLConf(SQLConf.LEGACY_CENTRAL_MOMENT_AGG.key -> "false") { - // Moved because: - // - Spark uses a different default stddev (sample instead of pop) - // - Tiny numerical differences in stddev results. - // - Different StdDev behavior when n=1 (NaN instead of 0) - checkAnswer(sql( - s""" - |select p_mfgr,p_name, p_size, - |rank() over(distribute by p_mfgr sort by p_name) as r, - |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, - |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, - |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, - |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, - |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, - |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, - |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, - |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, - |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, - |first_value(p_size) over w1 as fvW1 - |from part - |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name - | rows between 2 preceding and 2 following) + // Moved because: + // - Spark uses a different default stddev (sample instead of pop) + // - Tiny numerical differences in stddev results. + checkAnswer(sql(s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) """.stripMargin), - // scalastyle:off - Seq( - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), - Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), - Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), - Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), - Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), - Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, null, 4, 14, 14), - Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), - Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), - Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), - Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), - Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, null, 2, 17, 17), - Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), - Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), - Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), - Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), - Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, null, 0, 10, 10), - Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), - Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), - Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), - Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), - Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, null, 1, 31, 31), - Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), - Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), - Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), - Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) + // scalastyle:off + Seq( + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, null, 4, 14, 14), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0, null, 2, 17, 17), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), + Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, null, 0, 10, 10), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, null, 1, 31, 31), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) // scalastyle:on - } } test("windowing.q -- 20. testSTATs") { From e0769caa516e2c57302c6e39ea8a059b43fcdda3 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 18:13:17 +0800 Subject: [PATCH 08/14] code refine. Change-Id: I65269c5a3046c8dcbe1c72fa54c8c9a651c6914a --- .../sql/catalyst/analysis/TypeCoercion.scala | 18 ++++--- .../aggregate/CentralMomentAgg.scala | 49 +++++++++++++++---- .../catalyst/expressions/aggregate/Corr.scala | 17 +++++-- .../expressions/aggregate/Covariance.scala | 27 ++++++++-- 4 files changed, 86 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index deaa49bf423b1..b841a000f877c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -450,14 +450,20 @@ object TypeCoercion { case Abs(e @ StringType()) => Abs(Cast(e, DoubleType)) case Sum(e @ StringType()) => Sum(Cast(e, DoubleType)) case Average(e @ StringType()) => Average(Cast(e, DoubleType)) - case StddevPop(e @ StringType()) => StddevPop(Cast(e, DoubleType)) - case StddevSamp(e @ StringType()) => StddevSamp(Cast(e, DoubleType)) + case StddevPop(e @ StringType(), nullOnDivideByZero) => + StddevPop(Cast(e, DoubleType), nullOnDivideByZero) + case StddevSamp(e @ StringType(), nullOnDivideByZero) => + StddevSamp(Cast(e, DoubleType), nullOnDivideByZero) case UnaryMinus(e @ StringType()) => UnaryMinus(Cast(e, DoubleType)) case UnaryPositive(e @ StringType()) => UnaryPositive(Cast(e, DoubleType)) - case VariancePop(e @ StringType()) => VariancePop(Cast(e, DoubleType)) - case VarianceSamp(e @ StringType()) => VarianceSamp(Cast(e, DoubleType)) - case Skewness(e @ StringType()) => Skewness(Cast(e, DoubleType)) - case Kurtosis(e @ StringType()) => Kurtosis(Cast(e, DoubleType)) + case VariancePop(e @ StringType(), nullOnDivideByZero) => + VariancePop(Cast(e, DoubleType), nullOnDivideByZero) + case VarianceSamp(e @ StringType(), nullOnDivideByZero) => + VarianceSamp(Cast(e, DoubleType), nullOnDivideByZero) + case Skewness(e @ StringType(), nullOnDivideByZero) => + Skewness(Cast(e, DoubleType), nullOnDivideByZero) + case Kurtosis(e @ StringType(), nullOnDivideByZero) => + Kurtosis(Cast(e, DoubleType), nullOnDivideByZero) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 1f1899132418c..7251df402f2b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.types._ * * @param child to compute central moments of. */ -abstract class CentralMomentAgg(child: Expression) +abstract class CentralMomentAgg(child: Expression, nullOnDivideByZero: Boolean) extends DeclarativeAggregate with ImplicitCastInputTypes { /** @@ -63,8 +63,9 @@ abstract class CentralMomentAgg(child: Expression) protected val m3 = AttributeReference("m3", DoubleType, nullable = false)() protected val m4 = AttributeReference("m4", DoubleType, nullable = false)() - protected val divideByZeroEvalResult: Expression = - if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + protected lazy val divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + } private def trimHigherOrder[T](expressions: Seq[T]) = expressions.take(momentOrder + 1) @@ -149,7 +150,12 @@ abstract class CentralMomentAgg(child: Expression) group = "agg_funcs", since = "1.6.0") // scalastyle:on line.size.limit -case class StddevPop(child: Expression) extends CentralMomentAgg(child) { +case class StddevPop( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -172,7 +178,12 @@ case class StddevPop(child: Expression) extends CentralMomentAgg(child) { group = "agg_funcs", since = "1.6.0") // scalastyle:on line.size.limit -case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { +case class StddevSamp( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -195,7 +206,12 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { """, group = "agg_funcs", since = "1.6.0") -case class VariancePop(child: Expression) extends CentralMomentAgg(child) { +case class VariancePop( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -216,7 +232,12 @@ case class VariancePop(child: Expression) extends CentralMomentAgg(child) { """, group = "agg_funcs", since = "1.6.0") -case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { +case class VarianceSamp( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -239,7 +260,12 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { """, group = "agg_funcs", since = "1.6.0") -case class Skewness(child: Expression) extends CentralMomentAgg(child) { +case class Skewness( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override def prettyName: String = "skewness" @@ -262,7 +288,12 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { """, group = "agg_funcs", since = "1.6.0") -case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { +case class Kurtosis( + child: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends CentralMomentAgg(child, nullOnDivideByZero) { + + def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 4 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index bfdb9dc88c56b..8a2d811db1124 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.types._ * Definition of Pearson correlation can be found at * http://en.wikipedia.org/wiki/Pearson_product-moment_correlation_coefficient */ -abstract class PearsonCorrelation(x: Expression, y: Expression) +abstract class PearsonCorrelation(x: Expression, y: Expression, nullOnDivideByZero: Boolean) extends DeclarativeAggregate with ImplicitCastInputTypes { override def children: Seq[Expression] = Seq(x, y) @@ -44,8 +44,9 @@ abstract class PearsonCorrelation(x: Expression, y: Expression) protected val xMk = AttributeReference("xMk", DoubleType, nullable = false)() protected val yMk = AttributeReference("yMk", DoubleType, nullable = false)() - protected val divideByZeroEvalResult: Expression = - if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + protected lazy val divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + } override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck, xMk, yMk) @@ -106,8 +107,14 @@ abstract class PearsonCorrelation(x: Expression, y: Expression) group = "agg_funcs", since = "1.6.0") // scalastyle:on line.size.limit -case class Corr(x: Expression, y: Expression) - extends PearsonCorrelation(x, y) { +case class Corr( + x: Expression, + y: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends PearsonCorrelation(x, y, nullOnDivideByZero) { + + def this(x: Expression, y: Expression) = + this(x, y, SQLConf.get.legacyStatisticalAggregate) override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index 5929756678ec6..7cae6aab9d158 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._ * Compute the covariance between two expressions. * When applied on empty data (i.e., count is zero), it returns NULL. */ -abstract class Covariance(x: Expression, y: Expression) +abstract class Covariance(x: Expression, y: Expression, nullOnDivideByZero: Boolean) extends DeclarativeAggregate with ImplicitCastInputTypes { override def children: Seq[Expression] = Seq(x, y) @@ -39,8 +39,9 @@ abstract class Covariance(x: Expression, y: Expression) protected val yAvg = AttributeReference("yAvg", DoubleType, nullable = false)() protected val ck = AttributeReference("ck", DoubleType, nullable = false)() - protected val divideByZeroEvalResult: Expression = - if (SQLConf.get.legacyStatisticalAggregate) Double.NaN else Literal.create(null, DoubleType) + protected lazy val divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + } override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck) @@ -92,7 +93,15 @@ abstract class Covariance(x: Expression, y: Expression) """, group = "agg_funcs", since = "2.0.0") -case class CovPopulation(left: Expression, right: Expression) extends Covariance(left, right) { +case class CovPopulation( + left: Expression, + right: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends Covariance(left, right, nullOnDivideByZero) { + + def this(left: Expression, right: Expression) = + this(left, right, SQLConf.get.legacyStatisticalAggregate) + override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), ck / n) } @@ -109,7 +118,15 @@ case class CovPopulation(left: Expression, right: Expression) extends Covariance """, group = "agg_funcs", since = "2.0.0") -case class CovSample(left: Expression, right: Expression) extends Covariance(left, right) { +case class CovSample( + left: Expression, + right: Expression, + nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + extends Covariance(left, right, nullOnDivideByZero) { + + def this(left: Expression, right: Expression) = + this(left, right, SQLConf.get.legacyStatisticalAggregate) + override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), If(n === 1.0, divideByZeroEvalResult, ck / (n - 1.0))) From 811d2483a57534901859657d4c924a6501ac9749 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 18:40:26 +0800 Subject: [PATCH 09/14] update golden file for corr and covar_sample. update golden file for adding parameter nullOnDivideByZero. Change-Id: Ia7ce04335b226825c9b32a6428a08975bb48fad3 --- .../udf/postgreSQL/udf-aggregates_part1.sql.out | 16 ++++++++-------- .../sql-tests/results/udf/udf-group-by.sql.out | 4 ++-- .../sql-tests/results/udf/udf-window.sql.out | 4 ++-- .../resources/sql-tests/results/window.sql.out | 6 +++--- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index a428a7a9c923b..302be77b593b5 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -85,7 +85,7 @@ struct +struct -- !query output 151.38936080399804 @@ -101,7 +101,7 @@ struct +struct -- !query output 22918.738564643096 @@ -109,7 +109,7 @@ struct +struct -- !query output 131.18117242958306 @@ -125,7 +125,7 @@ struct +struct -- !query output 17208.5 @@ -141,7 +141,7 @@ struct +struct -- !query output 0.0 NULL @@ -265,7 +265,7 @@ NaN NaN SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) -- !query schema -struct +struct -- !query output 1.00000005E8 2.5 @@ -274,7 +274,7 @@ struct +struct -- !query output 7.000000000006E12 1.0 @@ -282,7 +282,7 @@ struct +struct -- !query output 653.6289553875104 871.5052738500139 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index da5256f5c0453..6816d0c22f735 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -128,7 +128,7 @@ NULL 1 SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData -- !query schema -struct +struct -- !query output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 @@ -247,7 +247,7 @@ struct<1:int> SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) -- !query schema -struct +struct -- !query output 1.0 1.0 3 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index 01e4e60b4f92f..928b9ebb12364 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -289,13 +289,13 @@ ORDER BY cate, udf(val) struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NaN NULL 0.0 [3] [3] NULL NULL +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 94eaeb5201f3c..9725555aa0b4f 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -313,13 +313,13 @@ ORDER BY cate, val struct,collect_set:array,skewness:double,kurtosis:double> -- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NaN NULL 0.0 [3] [3] NULL NULL +3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NaN 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 @@ -639,4 +639,4 @@ Gerard Hernandez SCM 6949 Pamela Castillo George Vanauf Sales 10563 Steve Patterson Steve Patterson Sales 9441 Steve Patterson Julie Firrelli Sales 9181 Steve Patterson -Foon Yue Tseng Sales 6660 Steve Patterson \ No newline at end of file +Foon Yue Tseng Sales 6660 Steve Patterson From c4ad6de8c27c92ec01565fef4756405821e92f58 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 20:25:25 +0800 Subject: [PATCH 10/14] fix AggregationQuerySuite UT failure. Change-Id: Ic31f74f6853e1ffe0383f62a98fbbc7a616e125b --- .../sql/hive/execution/AggregationQuerySuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 87771eed17b1b..70dcfb05c2ba9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -825,7 +825,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te """ |SELECT corr(b, c) FROM covar_tab WHERE a = 3 """.stripMargin), - Row(Double.NaN) :: Nil) + Row(null) :: Nil) checkAnswer( spark.sql( @@ -834,10 +834,10 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te """.stripMargin), Row(1, null) :: Row(2, null) :: - Row(3, Double.NaN) :: - Row(4, Double.NaN) :: - Row(5, Double.NaN) :: - Row(6, Double.NaN) :: Nil) + Row(3, null) :: + Row(4, null) :: + Row(5, null) :: + Row(6, null) :: Nil) val corr7 = spark.sql("SELECT corr(b, c) FROM covar_tab").collect()(0).getDouble(0) assert(math.abs(corr7 - 0.6633880657639323) < 1e-12) @@ -869,7 +869,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te // one row test val df3 = Seq.tabulate(1)(x => (1 * x, x * x * x - 2)).toDF("a", "b") - checkAnswer(df3.groupBy().agg(covar_samp("a", "b")), Row(Double.NaN)) + checkAnswer(df3.groupBy().agg(covar_samp("a", "b")), Row(null)) checkAnswer(df3.groupBy().agg(covar_pop("a", "b")), Row(0.0)) } From dc8efb6cb94fee3a7782498b162f8bd471642348 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Mon, 12 Oct 2020 22:57:56 +0800 Subject: [PATCH 11/14] code refine. Change-Id: Ia173d98cde3dee0e9f36dc1e1121879318981590 --- .../sql/catalyst/analysis/TypeCoercion.scala | 24 ++++++++-------- .../aggregate/CentralMomentAgg.scala | 28 +++++++++---------- .../catalyst/expressions/aggregate/Corr.scala | 8 +++--- .../expressions/aggregate/Covariance.scala | 12 ++++---- 4 files changed, 36 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index b841a000f877c..f72d9be205df3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -450,20 +450,20 @@ object TypeCoercion { case Abs(e @ StringType()) => Abs(Cast(e, DoubleType)) case Sum(e @ StringType()) => Sum(Cast(e, DoubleType)) case Average(e @ StringType()) => Average(Cast(e, DoubleType)) - case StddevPop(e @ StringType(), nullOnDivideByZero) => - StddevPop(Cast(e, DoubleType), nullOnDivideByZero) - case StddevSamp(e @ StringType(), nullOnDivideByZero) => - StddevSamp(Cast(e, DoubleType), nullOnDivideByZero) + case s @ StddevPop(e @ StringType(), _) => + s.withNewChildren(Seq(Cast(e, DoubleType))) + case s @ StddevSamp(e @ StringType(), _) => + s.withNewChildren(Seq(Cast(e, DoubleType))) case UnaryMinus(e @ StringType()) => UnaryMinus(Cast(e, DoubleType)) case UnaryPositive(e @ StringType()) => UnaryPositive(Cast(e, DoubleType)) - case VariancePop(e @ StringType(), nullOnDivideByZero) => - VariancePop(Cast(e, DoubleType), nullOnDivideByZero) - case VarianceSamp(e @ StringType(), nullOnDivideByZero) => - VarianceSamp(Cast(e, DoubleType), nullOnDivideByZero) - case Skewness(e @ StringType(), nullOnDivideByZero) => - Skewness(Cast(e, DoubleType), nullOnDivideByZero) - case Kurtosis(e @ StringType(), nullOnDivideByZero) => - Kurtosis(Cast(e, DoubleType), nullOnDivideByZero) + case v @ VariancePop(e @ StringType(), _) => + v.withNewChildren(Seq(Cast(e, DoubleType))) + case v @ VarianceSamp(e @ StringType(), _) => + v.withNewChildren(Seq(Cast(e, DoubleType))) + case s @ Skewness(e @ StringType(), _) => + s.withNewChildren(Seq(Cast(e, DoubleType))) + case k @ Kurtosis(e @ StringType(), _) => + k.withNewChildren(Seq(Cast(e, DoubleType))) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 7251df402f2b1..ef156c445f89b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -63,8 +63,8 @@ abstract class CentralMomentAgg(child: Expression, nullOnDivideByZero: Boolean) protected val m3 = AttributeReference("m3", DoubleType, nullable = false)() protected val m4 = AttributeReference("m4", DoubleType, nullable = false)() - protected lazy val divideByZeroEvalResult: Expression = { - if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + protected def divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } private def trimHigherOrder[T](expressions: Seq[T]) = expressions.take(momentOrder + 1) @@ -152,10 +152,10 @@ abstract class CentralMomentAgg(child: Expression, nullOnDivideByZero: Boolean) // scalastyle:on line.size.limit case class StddevPop( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -180,10 +180,10 @@ case class StddevPop( // scalastyle:on line.size.limit case class StddevSamp( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -208,10 +208,10 @@ case class StddevSamp( since = "1.6.0") case class VariancePop( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -234,10 +234,10 @@ case class VariancePop( since = "1.6.0") case class VarianceSamp( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 2 @@ -262,10 +262,10 @@ case class VarianceSamp( since = "1.6.0") case class Skewness( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override def prettyName: String = "skewness" @@ -290,10 +290,10 @@ case class Skewness( since = "1.6.0") case class Kurtosis( child: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends CentralMomentAgg(child, nullOnDivideByZero) { - def this(child: Expression) = this(child, SQLConf.get.legacyStatisticalAggregate) + def this(child: Expression) = this(child, !SQLConf.get.legacyStatisticalAggregate) override protected def momentOrder = 4 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index 8a2d811db1124..4c0c5081115ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -44,8 +44,8 @@ abstract class PearsonCorrelation(x: Expression, y: Expression, nullOnDivideByZe protected val xMk = AttributeReference("xMk", DoubleType, nullable = false)() protected val yMk = AttributeReference("yMk", DoubleType, nullable = false)() - protected lazy val divideByZeroEvalResult: Expression = { - if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + protected def divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck, xMk, yMk) @@ -110,11 +110,11 @@ abstract class PearsonCorrelation(x: Expression, y: Expression, nullOnDivideByZe case class Corr( x: Expression, y: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends PearsonCorrelation(x, y, nullOnDivideByZero) { def this(x: Expression, y: Expression) = - this(x, y, SQLConf.get.legacyStatisticalAggregate) + this(x, y, !SQLConf.get.legacyStatisticalAggregate) override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index 7cae6aab9d158..3c7ad2905fc51 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -39,8 +39,8 @@ abstract class Covariance(x: Expression, y: Expression, nullOnDivideByZero: Bool protected val yAvg = AttributeReference("yAvg", DoubleType, nullable = false)() protected val ck = AttributeReference("ck", DoubleType, nullable = false)() - protected lazy val divideByZeroEvalResult: Expression = { - if (nullOnDivideByZero) Double.NaN else Literal.create(null, DoubleType) + protected def divideByZeroEvalResult: Expression = { + if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck) @@ -96,11 +96,11 @@ abstract class Covariance(x: Expression, y: Expression, nullOnDivideByZero: Bool case class CovPopulation( left: Expression, right: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends Covariance(left, right, nullOnDivideByZero) { def this(left: Expression, right: Expression) = - this(left, right, SQLConf.get.legacyStatisticalAggregate) + this(left, right, !SQLConf.get.legacyStatisticalAggregate) override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), ck / n) @@ -121,11 +121,11 @@ case class CovPopulation( case class CovSample( left: Expression, right: Expression, - nullOnDivideByZero: Boolean = SQLConf.get.legacyStatisticalAggregate) + nullOnDivideByZero: Boolean = !SQLConf.get.legacyStatisticalAggregate) extends Covariance(left, right, nullOnDivideByZero) { def this(left: Expression, right: Expression) = - this(left, right, SQLConf.get.legacyStatisticalAggregate) + this(left, right, !SQLConf.get.legacyStatisticalAggregate) override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), From 6eee3c9042865b7bf3d268188852928d2cb5cd0c Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Tue, 13 Oct 2020 00:11:11 +0800 Subject: [PATCH 12/14] update golden file since nullOnDivideByZero change. Change-Id: I463c1f9696eaf975f0333d6120f749263fbc1592 --- .../udf/postgreSQL/udf-aggregates_part1.sql.out | 16 ++++++++-------- .../sql-tests/results/udf/udf-group-by.sql.out | 4 ++-- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 ++++---- .../approved-plans-v1_4/q17.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q17/explain.txt | 8 ++++---- .../approved-plans-v1_4/q17/simplified.txt | 2 +- .../approved-plans-v1_4/q39a.sf100/explain.txt | 16 ++++++++-------- .../q39a.sf100/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39a/explain.txt | 16 ++++++++-------- .../approved-plans-v1_4/q39a/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39b.sf100/explain.txt | 16 ++++++++-------- .../q39b.sf100/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39b/explain.txt | 16 ++++++++-------- .../approved-plans-v1_4/q39b/simplified.txt | 4 ++-- 14 files changed, 60 insertions(+), 60 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 302be77b593b5..347175adab401 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -85,7 +85,7 @@ struct +struct -- !query output 151.38936080399804 @@ -101,7 +101,7 @@ struct +struct -- !query output 22918.738564643096 @@ -109,7 +109,7 @@ struct +struct -- !query output 131.18117242958306 @@ -125,7 +125,7 @@ struct +struct -- !query output 17208.5 @@ -141,7 +141,7 @@ struct +struct -- !query output 0.0 NULL @@ -265,7 +265,7 @@ NaN NaN SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) -- !query schema -struct +struct -- !query output 1.00000005E8 2.5 @@ -274,7 +274,7 @@ struct +struct -- !query output 7.000000000006E12 1.0 @@ -282,7 +282,7 @@ struct +struct -- !query output 653.6289553875104 871.5052738500139 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index 6816d0c22f735..c18303ba15fc0 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -128,7 +128,7 @@ NULL 1 SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData -- !query schema -struct +struct -- !query output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 @@ -247,7 +247,7 @@ struct<1:int> SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) -- !query schema -struct +struct -- !query output 1.0 1.0 3 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index a17356ae04a03..f52a0081a9d70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -293,7 +293,7 @@ Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk (54) HashAggregate [codegen id : 17] Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double), true), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double), true), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double), true)] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double), true), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double), true), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double), true)] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double), true)#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double), true)#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double), true)#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double), true)#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double), true)#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double), true)#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double), true)#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double), true)#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double), true)#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index bfb59441f483b..2166a8a2bc4a6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (18) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double), true),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double), true),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double), true),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 4085b4ab988cb..0cabef4f46bfa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -248,7 +248,7 @@ Input [8]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, (45) HashAggregate [codegen id : 8] Input [6]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#29, i_item_desc#30] Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#11), partial_avg(cast(sr_return_quantity#11 as bigint)), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cast(cs_quantity#16 as bigint)), partial_stddev_samp(cast(cs_quantity#16 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double), true), partial_count(sr_return_quantity#11), partial_avg(cast(sr_return_quantity#11 as bigint)), partial_stddev_samp(cast(sr_return_quantity#11 as double), true), partial_count(cs_quantity#16), partial_avg(cast(cs_quantity#16 as bigint)), partial_stddev_samp(cast(cs_quantity#16 as double), true)] Aggregate Attributes [18]: [count#32, sum#33, count#34, n#35, avg#36, m2#37, count#38, sum#39, count#40, n#41, avg#42, m2#43, count#44, sum#45, count#46, n#47, avg#48, m2#49] Results [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] @@ -259,9 +259,9 @@ Arguments: hashpartitioning(i_item_id#29, i_item_desc#30, s_state#26, 5), true, (47) HashAggregate [codegen id : 9] Input [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#11), avg(cast(sr_return_quantity#11 as bigint)), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cast(cs_quantity#16 as bigint)), stddev_samp(cast(cs_quantity#16 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#69, avg(cast(ss_quantity#6 as bigint))#70, stddev_samp(cast(ss_quantity#6 as double))#71, count(sr_return_quantity#11)#72, avg(cast(sr_return_quantity#11 as bigint))#73, stddev_samp(cast(sr_return_quantity#11 as double))#74, count(cs_quantity#16)#75, avg(cast(cs_quantity#16 as bigint))#76, stddev_samp(cast(cs_quantity#16 as double))#77] -Results [15]: [i_item_id#29, i_item_desc#30, s_state#26, count(ss_quantity#6)#69 AS store_sales_quantitycount#78, avg(cast(ss_quantity#6 as bigint))#70 AS store_sales_quantityave#79, stddev_samp(cast(ss_quantity#6 as double))#71 AS store_sales_quantitystdev#80, (stddev_samp(cast(ss_quantity#6 as double))#71 / avg(cast(ss_quantity#6 as bigint))#70) AS store_sales_quantitycov#81, count(sr_return_quantity#11)#72 AS as_store_returns_quantitycount#82, avg(cast(sr_return_quantity#11 as bigint))#73 AS as_store_returns_quantityave#83, stddev_samp(cast(sr_return_quantity#11 as double))#74 AS as_store_returns_quantitystdev#84, (stddev_samp(cast(sr_return_quantity#11 as double))#74 / avg(cast(sr_return_quantity#11 as bigint))#73) AS store_returns_quantitycov#85, count(cs_quantity#16)#75 AS catalog_sales_quantitycount#86, avg(cast(cs_quantity#16 as bigint))#76 AS catalog_sales_quantityave#87, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitystdev#88, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitycov#89] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double), true), count(sr_return_quantity#11), avg(cast(sr_return_quantity#11 as bigint)), stddev_samp(cast(sr_return_quantity#11 as double), true), count(cs_quantity#16), avg(cast(cs_quantity#16 as bigint)), stddev_samp(cast(cs_quantity#16 as double), true)] +Aggregate Attributes [9]: [count(ss_quantity#6)#69, avg(cast(ss_quantity#6 as bigint))#70, stddev_samp(cast(ss_quantity#6 as double), true)#71, count(sr_return_quantity#11)#72, avg(cast(sr_return_quantity#11 as bigint))#73, stddev_samp(cast(sr_return_quantity#11 as double), true)#74, count(cs_quantity#16)#75, avg(cast(cs_quantity#16 as bigint))#76, stddev_samp(cast(cs_quantity#16 as double), true)#77] +Results [15]: [i_item_id#29, i_item_desc#30, s_state#26, count(ss_quantity#6)#69 AS store_sales_quantitycount#78, avg(cast(ss_quantity#6 as bigint))#70 AS store_sales_quantityave#79, stddev_samp(cast(ss_quantity#6 as double), true)#71 AS store_sales_quantitystdev#80, (stddev_samp(cast(ss_quantity#6 as double), true)#71 / avg(cast(ss_quantity#6 as bigint))#70) AS store_sales_quantitycov#81, count(sr_return_quantity#11)#72 AS as_store_returns_quantitycount#82, avg(cast(sr_return_quantity#11 as bigint))#73 AS as_store_returns_quantityave#83, stddev_samp(cast(sr_return_quantity#11 as double), true)#74 AS as_store_returns_quantitystdev#84, (stddev_samp(cast(sr_return_quantity#11 as double), true)#74 / avg(cast(sr_return_quantity#11 as bigint))#73) AS store_returns_quantitycov#85, count(cs_quantity#16)#75 AS catalog_sales_quantitycount#86, avg(cast(cs_quantity#16 as bigint))#76 AS catalog_sales_quantityave#87, (stddev_samp(cast(cs_quantity#16 as double), true)#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitystdev#88, (stddev_samp(cast(cs_quantity#16 as double), true)#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitycov#89] (48) TakeOrderedAndProject Input [15]: [i_item_id#29, i_item_desc#30, s_state#26, store_sales_quantitycount#78, store_sales_quantityave#79, store_sales_quantitystdev#80, store_sales_quantitycov#81, as_store_returns_quantitycount#82, as_store_returns_quantityave#83, as_store_returns_quantitystdev#84, store_returns_quantitycov#85, catalog_sales_quantitycount#86, catalog_sales_quantityave#87, catalog_sales_quantitystdev#88, catalog_sales_quantitycov#89] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index e9b95747c294f..d46824b8439f6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double), true),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double), true),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double), true),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index 3cf87e9bf2eaf..18b1eb7a4b33d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -158,7 +158,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#7, i_item_sk#9, w_ (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#9, w_warehouse_sk#11, w_warehouse_name#12, d_moy#7] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] @@ -169,9 +169,9 @@ Arguments: hashpartitioning(w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, (25) HashAggregate [codegen id : 5] Input [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stdev#27, mean#28] @@ -261,7 +261,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#33, i_item_sk#35, (46) HashAggregate [codegen id : 10] Input [5]: [inv_quantity_on_hand#4, i_item_sk#35, w_warehouse_sk#36, w_warehouse_name#37, d_moy#33] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] Results [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] @@ -272,9 +272,9 @@ Arguments: hashpartitioning(w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35 (48) HashAggregate [codegen id : 11] Input [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] -Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double))#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] +Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stdev#51, mean#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt index c0f519d2ec83b..31469ea618ed8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt @@ -12,7 +12,7 @@ WholeStageCodegen (14) WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) @@ -57,7 +57,7 @@ WholeStageCodegen (14) WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index b2cc849c603c8..c2e02b086c4cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -155,7 +155,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7 (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] @@ -166,9 +166,9 @@ Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d (25) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] @@ -250,7 +250,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk# (44) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] @@ -261,9 +261,9 @@ Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30 (46) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] +Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index f4e23c837b63a..6f9bd0a299b50 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (11) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) @@ -48,7 +48,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index b73801a52bb37..8c5b64c46914a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -158,7 +158,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#7, i_item_sk#9, w_ (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#9, w_warehouse_sk#11, w_warehouse_name#12, d_moy#7] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] @@ -169,9 +169,9 @@ Arguments: hashpartitioning(w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, (25) HashAggregate [codegen id : 5] Input [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stdev#27, mean#28] @@ -261,7 +261,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#33, i_item_sk#35, (46) HashAggregate [codegen id : 10] Input [5]: [inv_quantity_on_hand#4, i_item_sk#35, w_warehouse_sk#36, w_warehouse_name#37, d_moy#33] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] Results [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] @@ -272,9 +272,9 @@ Arguments: hashpartitioning(w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35 (48) HashAggregate [codegen id : 11] Input [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] -Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double))#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] +Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stdev#51, mean#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt index c0f519d2ec83b..31469ea618ed8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt @@ -12,7 +12,7 @@ WholeStageCodegen (14) WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) @@ -57,7 +57,7 @@ WholeStageCodegen (14) WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 92c2d5ed4700b..089588677134b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -155,7 +155,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7 (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] @@ -166,9 +166,9 @@ Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d (25) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] @@ -250,7 +250,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk# (44) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] @@ -261,9 +261,9 @@ Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30 (46) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] +Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index f4e23c837b63a..6f9bd0a299b50 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (11) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) @@ -48,7 +48,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) From 084c3fb876906fcf12faa348fe262e10c19ade7e Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Tue, 13 Oct 2020 09:37:23 +0800 Subject: [PATCH 13/14] update stringArgs to avoid change of golden files. Change-Id: Ib36d81e7a89b2b6d7867b2448b9b2b599c17e5bb --- .../aggregate/CentralMomentAgg.scala | 18 ++++++++++++++++++ .../catalyst/expressions/aggregate/Corr.scala | 3 +++ .../expressions/aggregate/Covariance.scala | 6 ++++++ .../postgreSQL/udf-aggregates_part1.sql.out | 16 ++++++++-------- .../sql-tests/results/udf/udf-group-by.sql.out | 4 ++-- .../resources/sql-tests/results/window.sql.out | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 8 ++++---- .../q17.sf100/simplified.txt | 2 +- .../approved-plans-v1_4/q17/explain.txt | 8 ++++---- .../approved-plans-v1_4/q17/simplified.txt | 2 +- .../approved-plans-v1_4/q39a.sf100/explain.txt | 16 ++++++++-------- .../q39a.sf100/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39a/explain.txt | 16 ++++++++-------- .../approved-plans-v1_4/q39a/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39b.sf100/explain.txt | 16 ++++++++-------- .../q39b.sf100/simplified.txt | 4 ++-- .../approved-plans-v1_4/q39b/explain.txt | 16 ++++++++-------- .../approved-plans-v1_4/q39b/simplified.txt | 4 ++-- 18 files changed, 88 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index ef156c445f89b..40075aeab9129 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -164,6 +164,9 @@ case class StddevPop( } override def prettyName: String = "stddev_pop" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the sample standard deviation of a column @@ -194,6 +197,9 @@ case class StddevSamp( override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("stddev_samp") + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the population variance of a column @@ -220,6 +226,9 @@ case class VariancePop( } override def prettyName: String = "var_pop" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the sample variance of a column @@ -247,6 +256,9 @@ case class VarianceSamp( } override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("var_samp") + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } @ExpressionDescription( @@ -275,6 +287,9 @@ case class Skewness( If(n === 0.0, Literal.create(null, DoubleType), If(m2 === 0.0, divideByZeroEvalResult, sqrt(n) * m3 / sqrt(m2 * m2 * m2))) } + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } @ExpressionDescription( @@ -303,4 +318,7 @@ case class Kurtosis( } override def prettyName: String = "kurtosis" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index 4c0c5081115ba..33a11e7633e6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -122,4 +122,7 @@ case class Corr( } override def prettyName: String = "corr" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index 3c7ad2905fc51..83a48341d449e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -106,6 +106,9 @@ case class CovPopulation( If(n === 0.0, Literal.create(null, DoubleType), ck / n) } override def prettyName: String = "covar_pop" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } @@ -132,4 +135,7 @@ case class CovSample( If(n === 1.0, divideByZeroEvalResult, ck / (n - 1.0))) } override def prettyName: String = "covar_samp" + + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) } diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index 347175adab401..a428a7a9c923b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -85,7 +85,7 @@ struct +struct -- !query output 151.38936080399804 @@ -101,7 +101,7 @@ struct +struct -- !query output 22918.738564643096 @@ -109,7 +109,7 @@ struct +struct -- !query output 131.18117242958306 @@ -125,7 +125,7 @@ struct +struct -- !query output 17208.5 @@ -141,7 +141,7 @@ struct +struct -- !query output 0.0 NULL @@ -265,7 +265,7 @@ NaN NaN SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) -- !query schema -struct +struct -- !query output 1.00000005E8 2.5 @@ -274,7 +274,7 @@ struct +struct -- !query output 7.000000000006E12 1.0 @@ -282,7 +282,7 @@ struct +struct -- !query output 653.6289553875104 871.5052738500139 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index c18303ba15fc0..da5256f5c0453 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -128,7 +128,7 @@ NULL 1 SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData -- !query schema -struct +struct -- !query output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 @@ -247,7 +247,7 @@ struct<1:int> SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) -- !query schema -struct +struct -- !query output 1.0 1.0 3 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 9725555aa0b4f..028dd7a12d25d 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -639,4 +639,4 @@ Gerard Hernandez SCM 6949 Pamela Castillo George Vanauf Sales 10563 Steve Patterson Steve Patterson Sales 9441 Steve Patterson Julie Firrelli Sales 9181 Steve Patterson -Foon Yue Tseng Sales 6660 Steve Patterson +Foon Yue Tseng Sales 6660 Steve Patterson \ No newline at end of file diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index f52a0081a9d70..a17356ae04a03 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -293,7 +293,7 @@ Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk (54) HashAggregate [codegen id : 17] Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double), true), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double), true), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double), true)] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double), true), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double), true), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double), true)] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double), true)#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double), true)#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double), true)#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double), true)#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double), true)#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double), true)#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double), true)#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double), true)#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double), true)#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index 2166a8a2bc4a6..bfb59441f483b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (18) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double), true),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double), true),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double), true),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (17) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index 0cabef4f46bfa..4085b4ab988cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -248,7 +248,7 @@ Input [8]: [ss_item_sk#2, ss_quantity#6, sr_return_quantity#11, cs_quantity#16, (45) HashAggregate [codegen id : 8] Input [6]: [ss_quantity#6, sr_return_quantity#11, cs_quantity#16, s_state#26, i_item_id#29, i_item_desc#30] Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double), true), partial_count(sr_return_quantity#11), partial_avg(cast(sr_return_quantity#11 as bigint)), partial_stddev_samp(cast(sr_return_quantity#11 as double), true), partial_count(cs_quantity#16), partial_avg(cast(cs_quantity#16 as bigint)), partial_stddev_samp(cast(cs_quantity#16 as double), true)] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#11), partial_avg(cast(sr_return_quantity#11 as bigint)), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cast(cs_quantity#16 as bigint)), partial_stddev_samp(cast(cs_quantity#16 as double))] Aggregate Attributes [18]: [count#32, sum#33, count#34, n#35, avg#36, m2#37, count#38, sum#39, count#40, n#41, avg#42, m2#43, count#44, sum#45, count#46, n#47, avg#48, m2#49] Results [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] @@ -259,9 +259,9 @@ Arguments: hashpartitioning(i_item_id#29, i_item_desc#30, s_state#26, 5), true, (47) HashAggregate [codegen id : 9] Input [21]: [i_item_id#29, i_item_desc#30, s_state#26, count#50, sum#51, count#52, n#53, avg#54, m2#55, count#56, sum#57, count#58, n#59, avg#60, m2#61, count#62, sum#63, count#64, n#65, avg#66, m2#67] Keys [3]: [i_item_id#29, i_item_desc#30, s_state#26] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double), true), count(sr_return_quantity#11), avg(cast(sr_return_quantity#11 as bigint)), stddev_samp(cast(sr_return_quantity#11 as double), true), count(cs_quantity#16), avg(cast(cs_quantity#16 as bigint)), stddev_samp(cast(cs_quantity#16 as double), true)] -Aggregate Attributes [9]: [count(ss_quantity#6)#69, avg(cast(ss_quantity#6 as bigint))#70, stddev_samp(cast(ss_quantity#6 as double), true)#71, count(sr_return_quantity#11)#72, avg(cast(sr_return_quantity#11 as bigint))#73, stddev_samp(cast(sr_return_quantity#11 as double), true)#74, count(cs_quantity#16)#75, avg(cast(cs_quantity#16 as bigint))#76, stddev_samp(cast(cs_quantity#16 as double), true)#77] -Results [15]: [i_item_id#29, i_item_desc#30, s_state#26, count(ss_quantity#6)#69 AS store_sales_quantitycount#78, avg(cast(ss_quantity#6 as bigint))#70 AS store_sales_quantityave#79, stddev_samp(cast(ss_quantity#6 as double), true)#71 AS store_sales_quantitystdev#80, (stddev_samp(cast(ss_quantity#6 as double), true)#71 / avg(cast(ss_quantity#6 as bigint))#70) AS store_sales_quantitycov#81, count(sr_return_quantity#11)#72 AS as_store_returns_quantitycount#82, avg(cast(sr_return_quantity#11 as bigint))#73 AS as_store_returns_quantityave#83, stddev_samp(cast(sr_return_quantity#11 as double), true)#74 AS as_store_returns_quantitystdev#84, (stddev_samp(cast(sr_return_quantity#11 as double), true)#74 / avg(cast(sr_return_quantity#11 as bigint))#73) AS store_returns_quantitycov#85, count(cs_quantity#16)#75 AS catalog_sales_quantitycount#86, avg(cast(cs_quantity#16 as bigint))#76 AS catalog_sales_quantityave#87, (stddev_samp(cast(cs_quantity#16 as double), true)#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitystdev#88, (stddev_samp(cast(cs_quantity#16 as double), true)#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitycov#89] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#11), avg(cast(sr_return_quantity#11 as bigint)), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cast(cs_quantity#16 as bigint)), stddev_samp(cast(cs_quantity#16 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#69, avg(cast(ss_quantity#6 as bigint))#70, stddev_samp(cast(ss_quantity#6 as double))#71, count(sr_return_quantity#11)#72, avg(cast(sr_return_quantity#11 as bigint))#73, stddev_samp(cast(sr_return_quantity#11 as double))#74, count(cs_quantity#16)#75, avg(cast(cs_quantity#16 as bigint))#76, stddev_samp(cast(cs_quantity#16 as double))#77] +Results [15]: [i_item_id#29, i_item_desc#30, s_state#26, count(ss_quantity#6)#69 AS store_sales_quantitycount#78, avg(cast(ss_quantity#6 as bigint))#70 AS store_sales_quantityave#79, stddev_samp(cast(ss_quantity#6 as double))#71 AS store_sales_quantitystdev#80, (stddev_samp(cast(ss_quantity#6 as double))#71 / avg(cast(ss_quantity#6 as bigint))#70) AS store_sales_quantitycov#81, count(sr_return_quantity#11)#72 AS as_store_returns_quantitycount#82, avg(cast(sr_return_quantity#11 as bigint))#73 AS as_store_returns_quantityave#83, stddev_samp(cast(sr_return_quantity#11 as double))#74 AS as_store_returns_quantitystdev#84, (stddev_samp(cast(sr_return_quantity#11 as double))#74 / avg(cast(sr_return_quantity#11 as bigint))#73) AS store_returns_quantitycov#85, count(cs_quantity#16)#75 AS catalog_sales_quantitycount#86, avg(cast(cs_quantity#16 as bigint))#76 AS catalog_sales_quantityave#87, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitystdev#88, (stddev_samp(cast(cs_quantity#16 as double))#77 / avg(cast(cs_quantity#16 as bigint))#76) AS catalog_sales_quantitycov#89] (48) TakeOrderedAndProject Input [15]: [i_item_id#29, i_item_desc#30, s_state#26, store_sales_quantitycount#78, store_sales_quantityave#79, store_sales_quantitystdev#80, store_sales_quantitycov#81, as_store_returns_quantitycount#82, as_store_returns_quantityave#83, as_store_returns_quantitystdev#84, store_returns_quantitycov#85, catalog_sales_quantitycount#86, catalog_sales_quantityave#87, catalog_sales_quantitystdev#88, catalog_sales_quantitycov#89] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt index d46824b8439f6..e9b95747c294f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/simplified.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov] WholeStageCodegen (9) - HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double), true),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double), true),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double), true),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(cast(ss_quantity as bigint)),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(cast(sr_return_quantity as bigint)),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cast(cs_quantity as bigint)),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] InputAdapter Exchange [i_item_id,i_item_desc,s_state] #1 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt index 18b1eb7a4b33d..3cf87e9bf2eaf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/explain.txt @@ -158,7 +158,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#7, i_item_sk#9, w_ (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#9, w_warehouse_sk#11, w_warehouse_name#12, d_moy#7] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] @@ -169,9 +169,9 @@ Arguments: hashpartitioning(w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, (25) HashAggregate [codegen id : 5] Input [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stdev#27, mean#28] @@ -261,7 +261,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#33, i_item_sk#35, (46) HashAggregate [codegen id : 10] Input [5]: [inv_quantity_on_hand#4, i_item_sk#35, w_warehouse_sk#36, w_warehouse_name#37, d_moy#33] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] Results [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] @@ -272,9 +272,9 @@ Arguments: hashpartitioning(w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35 (48) HashAggregate [codegen id : 11] Input [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] -Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] +Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double))#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stdev#51, mean#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt index 31469ea618ed8..c0f519d2ec83b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.sf100/simplified.txt @@ -12,7 +12,7 @@ WholeStageCodegen (14) WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) @@ -57,7 +57,7 @@ WholeStageCodegen (14) WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index c2e02b086c4cb..b2cc849c603c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -155,7 +155,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7 (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] @@ -166,9 +166,9 @@ Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d (25) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] @@ -250,7 +250,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk# (44) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] @@ -261,9 +261,9 @@ Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30 (46) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] +Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 6f9bd0a299b50..f4e23c837b63a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (11) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) @@ -48,7 +48,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt index 8c5b64c46914a..b73801a52bb37 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/explain.txt @@ -158,7 +158,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#7, i_item_sk#9, w_ (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#9, w_warehouse_sk#11, w_warehouse_name#12, d_moy#7] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] @@ -169,9 +169,9 @@ Arguments: hashpartitioning(w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, (25) HashAggregate [codegen id : 5] Input [9]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#12, w_warehouse_sk#11, i_item_sk#9, d_moy#7] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 5] Input [5]: [w_warehouse_sk#11, i_item_sk#9, d_moy#7, stdev#27, mean#28] @@ -261,7 +261,7 @@ Input [6]: [inv_warehouse_sk#3, inv_quantity_on_hand#4, d_moy#33, i_item_sk#35, (46) HashAggregate [codegen id : 10] Input [5]: [inv_quantity_on_hand#4, i_item_sk#35, w_warehouse_sk#36, w_warehouse_name#37, d_moy#33] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#38, avg#39, m2#40, sum#41, count#42] Results [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] @@ -272,9 +272,9 @@ Arguments: hashpartitioning(w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35 (48) HashAggregate [codegen id : 11] Input [9]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33, n#43, avg#44, m2#45, sum#46, count#47] Keys [4]: [w_warehouse_name#37, w_warehouse_sk#36, i_item_sk#35, d_moy#33] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] -Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#49, avg(cast(inv_quantity_on_hand#4 as bigint))#50] +Results [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stddev_samp(cast(inv_quantity_on_hand#4 as double))#49 AS stdev#51, avg(cast(inv_quantity_on_hand#4 as bigint))#50 AS mean#52] (49) Filter [codegen id : 11] Input [5]: [w_warehouse_sk#36, i_item_sk#35, d_moy#33, stdev#51, mean#52] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt index 31469ea618ed8..c0f519d2ec83b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.sf100/simplified.txt @@ -12,7 +12,7 @@ WholeStageCodegen (14) WholeStageCodegen (5) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #3 WholeStageCodegen (4) @@ -57,7 +57,7 @@ WholeStageCodegen (14) WholeStageCodegen (11) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #8 WholeStageCodegen (10) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 089588677134b..92c2d5ed4700b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -155,7 +155,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7 (23) HashAggregate [codegen id : 4] Input [5]: [inv_quantity_on_hand#4, i_item_sk#5, w_warehouse_sk#7, w_warehouse_name#8, d_moy#12] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#14, avg#15, m2#16, sum#17, count#18] Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] @@ -166,9 +166,9 @@ Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d (25) HashAggregate [codegen id : 10] Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12, n#19, avg#20, m2#21, sum#22, count#23] Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#5, d_moy#12] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] -Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#25, avg(cast(inv_quantity_on_hand#4 as bigint))#26] +Results [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stddev_samp(cast(inv_quantity_on_hand#4 as double))#25 AS stdev#27, avg(cast(inv_quantity_on_hand#4 as bigint))#26 AS mean#28] (26) Filter [codegen id : 10] Input [5]: [w_warehouse_sk#7, i_item_sk#5, d_moy#12, stdev#27, mean#28] @@ -250,7 +250,7 @@ Input [7]: [inv_date_sk#1, inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk# (44) HashAggregate [codegen id : 8] Input [5]: [inv_quantity_on_hand#4, i_item_sk#30, w_warehouse_sk#31, w_warehouse_name#32, d_moy#35] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double), true), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#4 as double)), partial_avg(cast(inv_quantity_on_hand#4 as bigint))] Aggregate Attributes [5]: [n#37, avg#38, m2#39, sum#40, count#41] Results [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] @@ -261,9 +261,9 @@ Arguments: hashpartitioning(w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30 (46) HashAggregate [codegen id : 9] Input [9]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35, n#42, avg#43, m2#44, sum#45, count#46] Keys [4]: [w_warehouse_name#32, w_warehouse_sk#31, i_item_sk#30, d_moy#35] -Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true), avg(cast(inv_quantity_on_hand#4 as bigint))] -Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] -Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double), true)#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double)), avg(cast(inv_quantity_on_hand#4 as bigint))] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#4 as double))#48, avg(cast(inv_quantity_on_hand#4 as bigint))#49] +Results [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#4 as double))#48 AS stdev#50, avg(cast(inv_quantity_on_hand#4 as bigint))#49 AS mean#51] (47) Filter [codegen id : 9] Input [5]: [w_warehouse_sk#31, i_item_sk#30, d_moy#35, stdev#50, mean#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 6f9bd0a299b50..f4e23c837b63a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -6,7 +6,7 @@ WholeStageCodegen (11) BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 WholeStageCodegen (4) @@ -48,7 +48,7 @@ WholeStageCodegen (11) WholeStageCodegen (9) Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] Filter [mean,stdev] - HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double), true),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(cast(inv_quantity_on_hand as bigint)),stdev,mean,n,avg,m2,sum,count] InputAdapter Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 WholeStageCodegen (8) From ddc522cd2a455a08e3151c363645cc3f001465d5 Mon Sep 17 00:00:00 2001 From: "xuewei.linxuewei" Date: Tue, 13 Oct 2020 16:28:33 +0800 Subject: [PATCH 14/14] code refine remove dup code. Change-Id: Idc061ac89bb65f1c6a0f20517b2489aaa903a7eb --- .../aggregate/CentralMomentAgg.scala | 21 +++---------------- .../catalyst/expressions/aggregate/Corr.scala | 6 +++--- .../expressions/aggregate/Covariance.scala | 9 +++----- 3 files changed, 9 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 40075aeab9129..2cc9adb5aa06e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -67,6 +67,9 @@ abstract class CentralMomentAgg(child: Expression, nullOnDivideByZero: Boolean) if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) + private def trimHigherOrder[T](expressions: Seq[T]) = expressions.take(momentOrder + 1) override val aggBufferAttributes = trimHigherOrder(Seq(n, avg, m2, m3, m4)) @@ -164,9 +167,6 @@ case class StddevPop( } override def prettyName: String = "stddev_pop" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the sample standard deviation of a column @@ -197,9 +197,6 @@ case class StddevSamp( override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("stddev_samp") - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the population variance of a column @@ -226,9 +223,6 @@ case class VariancePop( } override def prettyName: String = "var_pop" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } // Compute the sample variance of a column @@ -256,9 +250,6 @@ case class VarianceSamp( } override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("var_samp") - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } @ExpressionDescription( @@ -287,9 +278,6 @@ case class Skewness( If(n === 0.0, Literal.create(null, DoubleType), If(m2 === 0.0, divideByZeroEvalResult, sqrt(n) * m3 / sqrt(m2 * m2 * m2))) } - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } @ExpressionDescription( @@ -318,7 +306,4 @@ case class Kurtosis( } override def prettyName: String = "kurtosis" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index 33a11e7633e6c..737e8cd3ffa41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -48,6 +48,9 @@ abstract class PearsonCorrelation(x: Expression, y: Expression, nullOnDivideByZe if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) + override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck, xMk, yMk) override val initialValues: Seq[Expression] = Array.fill(6)(Literal(0.0)) @@ -122,7 +125,4 @@ case class Corr( } override def prettyName: String = "corr" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index 83a48341d449e..7c4d6ded6559e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -43,6 +43,9 @@ abstract class Covariance(x: Expression, y: Expression, nullOnDivideByZero: Bool if (nullOnDivideByZero) Literal.create(null, DoubleType) else Double.NaN } + override def stringArgs: Iterator[Any] = + super.stringArgs.filter(_.isInstanceOf[Expression]) + override val aggBufferAttributes: Seq[AttributeReference] = Seq(n, xAvg, yAvg, ck) override val initialValues: Seq[Expression] = Array.fill(4)(Literal(0.0)) @@ -106,9 +109,6 @@ case class CovPopulation( If(n === 0.0, Literal.create(null, DoubleType), ck / n) } override def prettyName: String = "covar_pop" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) } @@ -135,7 +135,4 @@ case class CovSample( If(n === 1.0, divideByZeroEvalResult, ck / (n - 1.0))) } override def prettyName: String = "covar_samp" - - override def stringArgs: Iterator[Any] = - super.stringArgs.filter(_.isInstanceOf[Expression]) }