This is an automated email from the ASF dual-hosted git repository.
rui pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-gluten.git
The following commit(s) were added to refs/heads/main by this push:
new 33eadbfa6 [VL] Enable rand function (#5829)
33eadbfa6 is described below
commit 33eadbfa62e988aeef43c1c888abf9b601de4cab
Author: 高阳阳 <[email protected]>
AuthorDate: Thu May 23 20:10:54 2024 +0800
[VL] Enable rand function (#5829)
---
.../clickhouse/CHSparkPlanExecApi.scala | 7 -
.../execution/ScalarFunctionsValidateSuite.scala | 6 +
.../functions/RegistrationAllFunctions.cc | 4 +
cpp/velox/substrait/SubstraitParser.cc | 3 +
docs/velox-backend-support-progress.md | 2 +
.../gluten/backendsapi/SparkPlanExecApi.scala | 7 -
.../gluten/expression/ExpressionConverter.scala | 5 -
.../expression/UnaryExpressionTransformer.scala | 22 -
.../sql-tests/inputs/group-by-ordinal.sql | 96 ++
.../src/test/resources/sql-tests/inputs/random.sql | 17 +
.../sql-tests/results/group-by-ordinal.sql.out | 398 ++++++++
.../resources/sql-tests/results/group-by.sql.out | 2 +-
.../resources/sql-tests/results/random.sql.out | 84 ++
.../sql-tests/inputs/group-by-ordinal.sql | 96 ++
.../src/test/resources/sql-tests/inputs/random.sql | 17 +
.../sql-tests/results/group-by-ordinal.sql.out | 398 ++++++++
.../resources/sql-tests/results/group-by.sql.out | 2 +-
.../resources/sql-tests/results/random.sql.out | 84 ++
.../sql-tests/inputs/group-by-ordinal.sql | 96 ++
.../src/test/resources/sql-tests/inputs/random.sql | 17 +
.../sql-tests/results/group-by-ordinal.sql.out | 523 ++++++++++
.../resources/sql-tests/results/group-by.sql.out | 2 +-
.../resources/sql-tests/results/random.sql.out | 115 +++
.../sql-tests/inputs/group-by-ordinal.sql | 96 ++
.../src/test/resources/sql-tests/inputs/random.sql | 17 +
.../sql-tests/inputs/table-valued-functions.sql | 126 +++
.../sql-tests/results/group-by-ordinal.sql.out | 524 ++++++++++
.../resources/sql-tests/results/group-by.sql.out | 2 +-
.../resources/sql-tests/results/random.sql.out | 115 +++
.../results/table-valued-functions.sql.out | 1017 ++++++++++++++++++++
.../gluten/utils/velox/VeloxTestSettings.scala | 1 +
.../spark/sql/GlutenGeneratorFunctionSuite.scala | 11 +-
32 files changed, 3866 insertions(+), 46 deletions(-)
diff --git
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
index 6a154cd94..8c2b20db6 100644
---
a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
+++
b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala
@@ -387,13 +387,6 @@ class CHSparkPlanExecApi extends SparkPlanExecApi {
original: GetMapValue): ExpressionTransformer =
GetMapValueTransformer(substraitExprName, left, right,
original.failOnError, original)
- override def genRandTransformer(
- substraitExprName: String,
- explicitSeed: ExpressionTransformer,
- original: Rand): ExpressionTransformer = {
- GenericExpressionTransformer(substraitExprName, Seq(explicitSeed),
original)
- }
-
/**
* Generate ShuffleDependency for ColumnarShuffleExchangeExec.
*
diff --git
a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
index b3753ab83..3180842ad 100644
---
a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
+++
b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala
@@ -717,6 +717,12 @@ class ScalarFunctionsValidateSuite extends
FunctionsValidateTest {
}
}
+ test("Test rand function") {
+ runQueryAndCompare("""SELECT rand() from lineitem limit
100""".stripMargin, false) {
+ checkGlutenOperatorMatch[ProjectExecTransformer]
+ }
+ }
+
test("regexp_replace") {
runQueryAndCompare(
"SELECT regexp_replace(c_comment, '\\w', 'something') FROM customer
limit 50") {
diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc
b/cpp/velox/operators/functions/RegistrationAllFunctions.cc
index 5a6b0f6aa..b88d781b6 100644
--- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc
+++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc
@@ -27,6 +27,7 @@
#include "velox/functions/prestosql/window/WindowFunctionsRegistration.h"
#include "velox/functions/sparksql/Bitwise.h"
#include "velox/functions/sparksql/Hash.h"
+#include "velox/functions/sparksql/Rand.h"
#include "velox/functions/sparksql/Register.h"
#include "velox/functions/sparksql/aggregates/Register.h"
#include "velox/functions/sparksql/window/WindowFunctionsRegistration.h"
@@ -43,6 +44,9 @@ void registerFunctionOverwrite() {
velox::registerFunction<RoundFunction, int64_t, int64_t, int32_t>({"round"});
velox::registerFunction<RoundFunction, double, double, int32_t>({"round"});
velox::registerFunction<RoundFunction, float, float, int32_t>({"round"});
+ // TODO: the below rand function registry can be removed after presto
function registry is removed.
+ velox::registerFunction<velox::functions::sparksql::RandFunction, double,
velox::Constant<int32_t>>({"spark_rand"});
+ velox::registerFunction<velox::functions::sparksql::RandFunction, double,
velox::Constant<int64_t>>({"spark_rand"});
auto kRowConstructorWithNull =
RowConstructorWithNullCallToSpecialForm::kRowConstructorWithNull;
velox::exec::registerVectorFunction(
diff --git a/cpp/velox/substrait/SubstraitParser.cc
b/cpp/velox/substrait/SubstraitParser.cc
index 6f221b78e..f417618d8 100644
--- a/cpp/velox/substrait/SubstraitParser.cc
+++ b/cpp/velox/substrait/SubstraitParser.cc
@@ -392,6 +392,9 @@ std::unordered_map<std::string, std::string>
SubstraitParser::substraitVeloxFunc
{"bitwise_not", "spark_bitwise_not"},
{"bitwise_or", "spark_bitwise_or"},
{"bitwise_xor", "spark_bitwise_xor"},
+ // TODO: the below registry for rand functions can be removed
+ // after presto function registry is removed.
+ {"rand", "spark_rand"},
{"murmur3hash", "hash_with_seed"},
{"xxhash64", "xxhash64_with_seed"},
{"modulus", "remainder"},
diff --git a/docs/velox-backend-support-progress.md
b/docs/velox-backend-support-progress.md
index fb68740c7..ccb253c24 100644
--- a/docs/velox-backend-support-progress.md
+++ b/docs/velox-backend-support-progress.md
@@ -449,3 +449,5 @@ Gluten supports 199 functions. (Drag to right to see all
data types)
| stack | |
| S | | S | S | S | S | S | S
| S | S | S | S | S | S | S | S |
S | S | S | S |
| xxhash64 | xxhash64 | xxhash64
| | | | | | | |
| | | | | | | | |
| | | |
| uuid | uuid | uuid
| S | | | | | | |
| | | | | | | | |
| | | |
+| rand | rand | rand
| S | | | | | | |
| | | | | | | | |
| | | |
+
diff --git
a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
index f41b26374..429b926cd 100644
---
a/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
+++
b/gluten-core/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala
@@ -165,13 +165,6 @@ trait SparkPlanExecApi {
GenericExpressionTransformer(substraitExprName, Seq(srcExpr, regexExpr,
limitExpr), original)
}
- def genRandTransformer(
- substraitExprName: String,
- explicitSeed: ExpressionTransformer,
- original: Rand): ExpressionTransformer = {
- RandTransformer(substraitExprName, explicitSeed, original)
- }
-
/** Generate an expression transformer to transform GetMapValue to
Substrait. */
def genGetMapValueTransformer(
substraitExprName: String,
diff --git
a/gluten-core/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala
b/gluten-core/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala
index 6e1427e2f..2d514118a 100644
---
a/gluten-core/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala
+++
b/gluten-core/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala
@@ -453,11 +453,6 @@ object ExpressionConverter extends SQLConfHelper with
Logging {
LiteralTransformer(m.nullOnOverflow)),
m
)
- case rand: Rand =>
- BackendsApiManager.getSparkPlanExecApiInstance.genRandTransformer(
- substraitExprName,
- replaceWithExpressionTransformerInternal(rand.child, attributeSeq,
expressionsMap),
- rand)
case _: NormalizeNaNAndZero | _: PromotePrecision | _: TaggingExpression
=>
ChildTransformer(
substraitExprName,
diff --git
a/gluten-core/src/main/scala/org/apache/gluten/expression/UnaryExpressionTransformer.scala
b/gluten-core/src/main/scala/org/apache/gluten/expression/UnaryExpressionTransformer.scala
index 27f839525..bcbac60de 100644
---
a/gluten-core/src/main/scala/org/apache/gluten/expression/UnaryExpressionTransformer.scala
+++
b/gluten-core/src/main/scala/org/apache/gluten/expression/UnaryExpressionTransformer.scala
@@ -91,28 +91,6 @@ case class CheckOverflowTransformer(
}
}
-/**
- * User can specify a seed for this function. If lacked, spark will generate a
random number as
- * seed. We also need to pass a unique partitionIndex provided by framework to
native library for
- * each thread. Then, seed plus partitionIndex will be the actual seed for
generator, similar to
- * vanilla spark. This is based on the fact that partitioning is deterministic
and one partition is
- * corresponding to one task thread.
- */
-case class RandTransformer(
- substraitExprName: String,
- explicitSeed: ExpressionTransformer,
- original: Rand)
- extends LeafExpressionTransformer {
-
- override def doTransform(args: java.lang.Object): ExpressionNode = {
- if (!original.hideSeed) {
- // TODO: for user-specified seed, we need to pass partition index to
native engine.
- throw new GlutenNotSupportException("User-specified seed is not
supported.")
- }
- super.doTransform(args)
- }
-}
-
case class GetStructFieldTransformer(
substraitExprName: String,
child: ExpressionTransformer,
diff --git
a/gluten-ut/spark32/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
b/gluten-ut/spark32/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
new file mode 100644
index 000000000..b773396c0
--- /dev/null
+++ b/gluten-ut/spark32/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
@@ -0,0 +1,96 @@
+-- group by ordinal positions
+
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b);
+
+-- basic case
+select a, sum(b) from data group by 1;
+
+-- constant case
+select 1, 2, sum(b) from data group by 1, 2;
+
+-- duplicate group by column
+select a, 1, sum(b) from data group by a, 1;
+select a, 1, sum(b) from data group by 1, 2;
+
+-- group by a non-aggregate expression's ordinal
+select a, b + 2, count(2) from data group by a, 2;
+
+-- with alias
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2;
+
+-- foldable non-literal: this should be the same as no grouping.
+select sum(b) from data group by 1 + 0;
+
+-- negative cases: ordinal out of range
+select a, b from data group by -1;
+select a, b from data group by 0;
+select a, b from data group by 3;
+
+-- negative case: position is an aggregate expression
+select a, b, sum(b) from data group by 3;
+select a, b, sum(b) + 2 from data group by 3;
+
+-- negative case: nondeterministic expression
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2;
+
+-- negative case: star
+select * from data group by a, b, 1;
+
+-- group by ordinal followed by order by
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1;
+
+-- group by ordinal followed by having
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
+
+-- mixed cases: group-by ordinals and aliases
+select a, a AS k, count(b) from data group by k, 1;
+
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);
+
+-- mixed cases: can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, b);
+
+-- can use ordinal with cube
+select a, b, count(1) from data group by 1, 2 with cube;
+
+-- can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, 2);
+
+-- mixed cases: can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, b);
+
+-- can use ordinal with rollup
+select a, b, count(1) from data group by 1, 2 with rollup;
+
+-- can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));
+
+-- mixed cases: can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));
+
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));
+
+-- range error
+select a, b, count(1) from data group by a, -1;
+
+select a, b, count(1) from data group by a, 3;
+
+select a, b, count(1) from data group by cube(-1, 2);
+
+select a, b, count(1) from data group by cube(1, 3);
+
+-- turn off group by ordinal
+set spark.sql.groupByOrdinal=false;
+
+-- can now group by negative literal
+select sum(b) from data group by -1;
diff --git a/gluten-ut/spark32/src/test/resources/sql-tests/inputs/random.sql
b/gluten-ut/spark32/src/test/resources/sql-tests/inputs/random.sql
new file mode 100644
index 000000000..a1aae7b87
--- /dev/null
+++ b/gluten-ut/spark32/src/test/resources/sql-tests/inputs/random.sql
@@ -0,0 +1,17 @@
+-- rand with the seed 0
+SELECT rand(0);
+SELECT rand(cast(3 / 7 AS int));
+SELECT rand(NULL);
+SELECT rand(cast(NULL AS int));
+
+-- rand unsupported data type
+SELECT rand(1.0);
+
+-- randn with the seed 0
+SELECT randn(0L);
+SELECT randn(cast(3 / 7 AS long));
+SELECT randn(NULL);
+SELECT randn(cast(NULL AS long));
+
+-- randn unsupported data type
+SELECT rand('1')
diff --git
a/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
b/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
new file mode 100644
index 000000000..cc20dd33e
--- /dev/null
+++
b/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
@@ -0,0 +1,398 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 33
+
+
+-- !query
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select a, sum(b) from data group by 1
+-- !query schema
+struct<a:int,sum(b):bigint>
+-- !query output
+1 3
+2 3
+3 3
+
+
+-- !query
+select 1, 2, sum(b) from data group by 1, 2
+-- !query schema
+struct<1:int,2:int,sum(b):bigint>
+-- !query output
+1 2 9
+
+
+-- !query
+select a, 1, sum(b) from data group by a, 1
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, 1, sum(b) from data group by 1, 2
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, b + 2, count(2) from data group by a, 2
+-- !query schema
+struct<a:int,(b + 2):int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2
+-- !query schema
+struct<aa:int,bb:int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select sum(b) from data group by 1 + 0
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
+
+
+-- !query
+select a, b from data group by -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b from data group by 0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b, sum(b) from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got sum(data.b) AS
`sum(b)`; line 1 pos 39
+
+
+-- !query
+select a, b, sum(b) + 2 from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got (sum(data.b) + CAST(2
AS BIGINT)) AS `(sum(b) + 2)`; line 1 pos 43
+
+
+-- !query
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2
+-- !query schema
+struct<a:int,rand(0):double,sum(b):bigint>
+-- !query output
+1 0.5488135024422883 1
+1 0.7151893651681639 2
+2 0.5448831775801376 2
+2 0.6027633705776989 1
+3 0.4236547969336536 1
+3 0.6458941151817286 2
+
+
+-- !query
+select * from data group by a, b, 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+Star (*) is not allowed in select list when GROUP BY ordinal position is used
+
+
+-- !query
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1
+-- !query schema
+struct<a:int,count(a):bigint>
+-- !query output
+1 1
+
+
+-- !query
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0
+-- !query schema
+struct<count(a):bigint,a:int>
+-- !query output
+1 1
+
+
+-- !query
+select a, a AS k, count(b) from data group by k, 1
+-- !query schema
+struct<a:int,k:int,count(b):bigint>
+-- !query output
+1 1 2
+2 2 2
+3 3 2
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with cube
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with rollup
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos
44
+
+
+-- !query
+select a, b, count(1) from data group by a, 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got count(1) AS
`count(1)`; line 1 pos 44
+
+
+-- !query
+select a, b, count(1) from data group by cube(-1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos
46
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 3)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got count(1) AS
`count(1)`; line 1 pos 49
+
+
+-- !query
+set spark.sql.groupByOrdinal=false
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.groupByOrdinal false
+
+
+-- !query
+select sum(b) from data group by -1
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
diff --git
a/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by.sql.out
b/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by.sql.out
index a12e830c1..b54621f8e 100644
--- a/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by.sql.out
+++ b/gluten-ut/spark32/src/test/resources/sql-tests/results/group-by.sql.out
@@ -662,5 +662,5 @@ GROUP BY a IS NULL
-- !query schema
struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint>
-- !query output
-0.7604953758285915 7
+0.5488135024422883 7
1.0 2
diff --git
a/gluten-ut/spark32/src/test/resources/sql-tests/results/random.sql.out
b/gluten-ut/spark32/src/test/resources/sql-tests/results/random.sql.out
new file mode 100644
index 000000000..a9d334e7f
--- /dev/null
+++ b/gluten-ut/spark32/src/test/resources/sql-tests/results/random.sql.out
@@ -0,0 +1,84 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 10
+
+
+-- !query
+SELECT rand(0)
+-- !query schema
+struct<rand(0):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(3 / 7 AS int))
+-- !query schema
+struct<rand(CAST((3 / 7) AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(NULL)
+-- !query schema
+struct<rand(NULL):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(NULL AS int))
+-- !query schema
+struct<rand(CAST(NULL AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'rand(1.0BD)' due to data type mismatch: argument 1 requires
(int or bigint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+
+
+-- !query
+SELECT randn(0L)
+-- !query schema
+struct<randn(0):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(3 / 7 AS long))
+-- !query schema
+struct<randn(CAST((3 / 7) AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(NULL)
+-- !query schema
+struct<randn(NULL):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(NULL AS long))
+-- !query schema
+struct<randn(CAST(NULL AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT rand('1')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'rand('1')' due to data type mismatch: argument 1 requires (int
or bigint) type, however, ''1'' is of string type.; line 1 pos 7
diff --git
a/gluten-ut/spark33/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
b/gluten-ut/spark33/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
new file mode 100644
index 000000000..b773396c0
--- /dev/null
+++ b/gluten-ut/spark33/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
@@ -0,0 +1,96 @@
+-- group by ordinal positions
+
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b);
+
+-- basic case
+select a, sum(b) from data group by 1;
+
+-- constant case
+select 1, 2, sum(b) from data group by 1, 2;
+
+-- duplicate group by column
+select a, 1, sum(b) from data group by a, 1;
+select a, 1, sum(b) from data group by 1, 2;
+
+-- group by a non-aggregate expression's ordinal
+select a, b + 2, count(2) from data group by a, 2;
+
+-- with alias
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2;
+
+-- foldable non-literal: this should be the same as no grouping.
+select sum(b) from data group by 1 + 0;
+
+-- negative cases: ordinal out of range
+select a, b from data group by -1;
+select a, b from data group by 0;
+select a, b from data group by 3;
+
+-- negative case: position is an aggregate expression
+select a, b, sum(b) from data group by 3;
+select a, b, sum(b) + 2 from data group by 3;
+
+-- negative case: nondeterministic expression
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2;
+
+-- negative case: star
+select * from data group by a, b, 1;
+
+-- group by ordinal followed by order by
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1;
+
+-- group by ordinal followed by having
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
+
+-- mixed cases: group-by ordinals and aliases
+select a, a AS k, count(b) from data group by k, 1;
+
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);
+
+-- mixed cases: can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, b);
+
+-- can use ordinal with cube
+select a, b, count(1) from data group by 1, 2 with cube;
+
+-- can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, 2);
+
+-- mixed cases: can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, b);
+
+-- can use ordinal with rollup
+select a, b, count(1) from data group by 1, 2 with rollup;
+
+-- can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));
+
+-- mixed cases: can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));
+
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));
+
+-- range error
+select a, b, count(1) from data group by a, -1;
+
+select a, b, count(1) from data group by a, 3;
+
+select a, b, count(1) from data group by cube(-1, 2);
+
+select a, b, count(1) from data group by cube(1, 3);
+
+-- turn off group by ordinal
+set spark.sql.groupByOrdinal=false;
+
+-- can now group by negative literal
+select sum(b) from data group by -1;
diff --git a/gluten-ut/spark33/src/test/resources/sql-tests/inputs/random.sql
b/gluten-ut/spark33/src/test/resources/sql-tests/inputs/random.sql
new file mode 100644
index 000000000..a1aae7b87
--- /dev/null
+++ b/gluten-ut/spark33/src/test/resources/sql-tests/inputs/random.sql
@@ -0,0 +1,17 @@
+-- rand with the seed 0
+SELECT rand(0);
+SELECT rand(cast(3 / 7 AS int));
+SELECT rand(NULL);
+SELECT rand(cast(NULL AS int));
+
+-- rand unsupported data type
+SELECT rand(1.0);
+
+-- randn with the seed 0
+SELECT randn(0L);
+SELECT randn(cast(3 / 7 AS long));
+SELECT randn(NULL);
+SELECT randn(cast(NULL AS long));
+
+-- randn unsupported data type
+SELECT rand('1')
diff --git
a/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
b/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
new file mode 100644
index 000000000..cc20dd33e
--- /dev/null
+++
b/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
@@ -0,0 +1,398 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 33
+
+
+-- !query
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select a, sum(b) from data group by 1
+-- !query schema
+struct<a:int,sum(b):bigint>
+-- !query output
+1 3
+2 3
+3 3
+
+
+-- !query
+select 1, 2, sum(b) from data group by 1, 2
+-- !query schema
+struct<1:int,2:int,sum(b):bigint>
+-- !query output
+1 2 9
+
+
+-- !query
+select a, 1, sum(b) from data group by a, 1
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, 1, sum(b) from data group by 1, 2
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, b + 2, count(2) from data group by a, 2
+-- !query schema
+struct<a:int,(b + 2):int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2
+-- !query schema
+struct<aa:int,bb:int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select sum(b) from data group by 1 + 0
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
+
+
+-- !query
+select a, b from data group by -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b from data group by 0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos
31
+
+
+-- !query
+select a, b, sum(b) from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got sum(data.b) AS
`sum(b)`; line 1 pos 39
+
+
+-- !query
+select a, b, sum(b) + 2 from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got (sum(data.b) + CAST(2
AS BIGINT)) AS `(sum(b) + 2)`; line 1 pos 43
+
+
+-- !query
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2
+-- !query schema
+struct<a:int,rand(0):double,sum(b):bigint>
+-- !query output
+1 0.5488135024422883 1
+1 0.7151893651681639 2
+2 0.5448831775801376 2
+2 0.6027633705776989 1
+3 0.4236547969336536 1
+3 0.6458941151817286 2
+
+
+-- !query
+select * from data group by a, b, 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+Star (*) is not allowed in select list when GROUP BY ordinal position is used
+
+
+-- !query
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1
+-- !query schema
+struct<a:int,count(a):bigint>
+-- !query output
+1 1
+
+
+-- !query
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0
+-- !query schema
+struct<count(a):bigint,a:int>
+-- !query output
+1 1
+
+
+-- !query
+select a, a AS k, count(b) from data group by k, 1
+-- !query schema
+struct<a:int,k:int,count(b):bigint>
+-- !query output
+1 1 2
+2 2 2
+3 3 2
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with cube
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with rollup
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos
44
+
+
+-- !query
+select a, b, count(1) from data group by a, 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got count(1) AS
`count(1)`; line 1 pos 44
+
+
+-- !query
+select a, b, count(1) from data group by cube(-1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos
46
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 3)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+GROUP BY 3 refers to an expression that is or contains an aggregate function.
Aggregate functions are not allowed in GROUP BY, but got count(1) AS
`count(1)`; line 1 pos 49
+
+
+-- !query
+set spark.sql.groupByOrdinal=false
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.groupByOrdinal false
+
+
+-- !query
+select sum(b) from data group by -1
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
diff --git
a/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by.sql.out
b/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by.sql.out
index 48b35bf1e..408b9f942 100644
--- a/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by.sql.out
+++ b/gluten-ut/spark33/src/test/resources/sql-tests/results/group-by.sql.out
@@ -691,7 +691,7 @@ GROUP BY a IS NULL
-- !query schema
struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint>
-- !query output
-0.7604953758285915 7
+0.5488135024422883 7
1.0 2
diff --git
a/gluten-ut/spark33/src/test/resources/sql-tests/results/random.sql.out
b/gluten-ut/spark33/src/test/resources/sql-tests/results/random.sql.out
new file mode 100644
index 000000000..a9d334e7f
--- /dev/null
+++ b/gluten-ut/spark33/src/test/resources/sql-tests/results/random.sql.out
@@ -0,0 +1,84 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 10
+
+
+-- !query
+SELECT rand(0)
+-- !query schema
+struct<rand(0):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(3 / 7 AS int))
+-- !query schema
+struct<rand(CAST((3 / 7) AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(NULL)
+-- !query schema
+struct<rand(NULL):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(NULL AS int))
+-- !query schema
+struct<rand(CAST(NULL AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'rand(1.0BD)' due to data type mismatch: argument 1 requires
(int or bigint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+
+
+-- !query
+SELECT randn(0L)
+-- !query schema
+struct<randn(0):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(3 / 7 AS long))
+-- !query schema
+struct<randn(CAST((3 / 7) AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(NULL)
+-- !query schema
+struct<randn(NULL):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(NULL AS long))
+-- !query schema
+struct<randn(CAST(NULL AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT rand('1')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve 'rand('1')' due to data type mismatch: argument 1 requires (int
or bigint) type, however, ''1'' is of string type.; line 1 pos 7
diff --git
a/gluten-ut/spark34/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
b/gluten-ut/spark34/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
new file mode 100644
index 000000000..b773396c0
--- /dev/null
+++ b/gluten-ut/spark34/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
@@ -0,0 +1,96 @@
+-- group by ordinal positions
+
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b);
+
+-- basic case
+select a, sum(b) from data group by 1;
+
+-- constant case
+select 1, 2, sum(b) from data group by 1, 2;
+
+-- duplicate group by column
+select a, 1, sum(b) from data group by a, 1;
+select a, 1, sum(b) from data group by 1, 2;
+
+-- group by a non-aggregate expression's ordinal
+select a, b + 2, count(2) from data group by a, 2;
+
+-- with alias
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2;
+
+-- foldable non-literal: this should be the same as no grouping.
+select sum(b) from data group by 1 + 0;
+
+-- negative cases: ordinal out of range
+select a, b from data group by -1;
+select a, b from data group by 0;
+select a, b from data group by 3;
+
+-- negative case: position is an aggregate expression
+select a, b, sum(b) from data group by 3;
+select a, b, sum(b) + 2 from data group by 3;
+
+-- negative case: nondeterministic expression
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2;
+
+-- negative case: star
+select * from data group by a, b, 1;
+
+-- group by ordinal followed by order by
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1;
+
+-- group by ordinal followed by having
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
+
+-- mixed cases: group-by ordinals and aliases
+select a, a AS k, count(b) from data group by k, 1;
+
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);
+
+-- mixed cases: can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, b);
+
+-- can use ordinal with cube
+select a, b, count(1) from data group by 1, 2 with cube;
+
+-- can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, 2);
+
+-- mixed cases: can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, b);
+
+-- can use ordinal with rollup
+select a, b, count(1) from data group by 1, 2 with rollup;
+
+-- can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));
+
+-- mixed cases: can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));
+
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));
+
+-- range error
+select a, b, count(1) from data group by a, -1;
+
+select a, b, count(1) from data group by a, 3;
+
+select a, b, count(1) from data group by cube(-1, 2);
+
+select a, b, count(1) from data group by cube(1, 3);
+
+-- turn off group by ordinal
+set spark.sql.groupByOrdinal=false;
+
+-- can now group by negative literal
+select sum(b) from data group by -1;
diff --git a/gluten-ut/spark34/src/test/resources/sql-tests/inputs/random.sql
b/gluten-ut/spark34/src/test/resources/sql-tests/inputs/random.sql
new file mode 100644
index 000000000..a1aae7b87
--- /dev/null
+++ b/gluten-ut/spark34/src/test/resources/sql-tests/inputs/random.sql
@@ -0,0 +1,17 @@
+-- rand with the seed 0
+SELECT rand(0);
+SELECT rand(cast(3 / 7 AS int));
+SELECT rand(NULL);
+SELECT rand(cast(NULL AS int));
+
+-- rand unsupported data type
+SELECT rand(1.0);
+
+-- randn with the seed 0
+SELECT randn(0L);
+SELECT randn(cast(3 / 7 AS long));
+SELECT randn(NULL);
+SELECT randn(cast(NULL AS long));
+
+-- randn unsupported data type
+SELECT rand('1')
diff --git
a/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
b/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
new file mode 100644
index 000000000..5b8637012
--- /dev/null
+++
b/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
@@ -0,0 +1,523 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select a, sum(b) from data group by 1
+-- !query schema
+struct<a:int,sum(b):bigint>
+-- !query output
+1 3
+2 3
+3 3
+
+
+-- !query
+select 1, 2, sum(b) from data group by 1, 2
+-- !query schema
+struct<1:int,2:int,sum(b):bigint>
+-- !query output
+1 2 9
+
+
+-- !query
+select a, 1, sum(b) from data group by a, 1
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, 1, sum(b) from data group by 1, 2
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, b + 2, count(2) from data group by a, 2
+-- !query schema
+struct<a:int,(b + 2):int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2
+-- !query schema
+struct<aa:int,bb:int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select sum(b) from data group by 1 + 0
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
+
+
+-- !query
+select a, b from data group by -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 33,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b from data group by 0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "0",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "0"
+ } ]
+}
+
+
+-- !query
+select a, b from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, sum(b) from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "sum(data.b) AS `sum(b)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 40,
+ "stopIndex" : 40,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, sum(b) + 2 from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 44,
+ "stopIndex" : 44,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2
+-- !query schema
+struct<a:int,rand(0):double,sum(b):bigint>
+-- !query output
+1 0.5488135024422883 1
+1 0.7151893651681639 2
+2 0.5448831775801376 2
+2 0.6027633705776989 1
+3 0.4236547969336536 1
+3 0.6458941151817286 2
+
+-- !query
+select * from data group by a, b, 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "STAR_GROUP_BY_POS",
+ "sqlState" : "0A000"
+}
+
+
+-- !query
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1
+-- !query schema
+struct<a:int,count(a):bigint>
+-- !query output
+1 1
+
+
+-- !query
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0
+-- !query schema
+struct<count(a):bigint,a:int>
+-- !query output
+1 1
+
+
+-- !query
+select a, a AS k, count(b) from data group by k, 1
+-- !query schema
+struct<a:int,k:int,count(b):bigint>
+-- !query output
+1 1 2
+2 2 2
+3 3 2
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with cube
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with rollup
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 46,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by a, 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 45,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by cube(-1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 47,
+ "stopIndex" : 48,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 3)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 50,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+set spark.sql.groupByOrdinal=false
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.groupByOrdinal false
+
+
+-- !query
+select sum(b) from data group by -1
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
diff --git
a/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by.sql.out
b/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by.sql.out
index f56420926..a4a3f76fa 100644
--- a/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by.sql.out
+++ b/gluten-ut/spark34/src/test/resources/sql-tests/results/group-by.sql.out
@@ -907,7 +907,7 @@ GROUP BY a IS NULL
-- !query schema
struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint>
-- !query output
-0.7604953758285915 7
+0.5488135024422883 7
1.0 2
diff --git
a/gluten-ut/spark34/src/test/resources/sql-tests/results/random.sql.out
b/gluten-ut/spark34/src/test/resources/sql-tests/results/random.sql.out
new file mode 100644
index 000000000..f8460c1d4
--- /dev/null
+++ b/gluten-ut/spark34/src/test/resources/sql-tests/results/random.sql.out
@@ -0,0 +1,115 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT rand(0)
+-- !query schema
+struct<rand(0):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(3 / 7 AS int))
+-- !query schema
+struct<rand(CAST((3 / 7) AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(NULL)
+-- !query schema
+struct<rand(NULL):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(NULL AS int))
+-- !query schema
+struct<rand(CAST(NULL AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INT\" or \"BIGINT\")",
+ "sqlExpr" : "\"rand(1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "rand(1.0)"
+ } ]
+}
+
+
+-- !query
+SELECT randn(0L)
+-- !query schema
+struct<randn(0):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(3 / 7 AS long))
+-- !query schema
+struct<randn(CAST((3 / 7) AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(NULL)
+-- !query schema
+struct<randn(NULL):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(NULL AS long))
+-- !query schema
+struct<randn(CAST(NULL AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT rand('1')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INT\" or \"BIGINT\")",
+ "sqlExpr" : "\"rand(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "rand('1')"
+ } ]
+}
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
new file mode 100644
index 000000000..b773396c0
--- /dev/null
+++ b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/group-by-ordinal.sql
@@ -0,0 +1,96 @@
+-- group by ordinal positions
+
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b);
+
+-- basic case
+select a, sum(b) from data group by 1;
+
+-- constant case
+select 1, 2, sum(b) from data group by 1, 2;
+
+-- duplicate group by column
+select a, 1, sum(b) from data group by a, 1;
+select a, 1, sum(b) from data group by 1, 2;
+
+-- group by a non-aggregate expression's ordinal
+select a, b + 2, count(2) from data group by a, 2;
+
+-- with alias
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2;
+
+-- foldable non-literal: this should be the same as no grouping.
+select sum(b) from data group by 1 + 0;
+
+-- negative cases: ordinal out of range
+select a, b from data group by -1;
+select a, b from data group by 0;
+select a, b from data group by 3;
+
+-- negative case: position is an aggregate expression
+select a, b, sum(b) from data group by 3;
+select a, b, sum(b) + 2 from data group by 3;
+
+-- negative case: nondeterministic expression
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2;
+
+-- negative case: star
+select * from data group by a, b, 1;
+
+-- group by ordinal followed by order by
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1;
+
+-- group by ordinal followed by having
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0;
+
+-- mixed cases: group-by ordinals and aliases
+select a, a AS k, count(b) from data group by k, 1;
+
+-- can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, 2);
+
+-- mixed cases: can use ordinal in CUBE
+select a, b, count(1) from data group by cube(1, b);
+
+-- can use ordinal with cube
+select a, b, count(1) from data group by 1, 2 with cube;
+
+-- can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, 2);
+
+-- mixed cases: can use ordinal in ROLLUP
+select a, b, count(1) from data group by rollup(1, b);
+
+-- can use ordinal with rollup
+select a, b, count(1) from data group by 1, 2 with rollup;
+
+-- can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2));
+
+-- mixed cases: can use ordinal in GROUPING SETS
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2));
+
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2));
+
+-- range error
+select a, b, count(1) from data group by a, -1;
+
+select a, b, count(1) from data group by a, 3;
+
+select a, b, count(1) from data group by cube(-1, 2);
+
+select a, b, count(1) from data group by cube(1, 3);
+
+-- turn off group by ordinal
+set spark.sql.groupByOrdinal=false;
+
+-- can now group by negative literal
+select sum(b) from data group by -1;
diff --git a/gluten-ut/spark35/src/test/resources/sql-tests/inputs/random.sql
b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/random.sql
new file mode 100644
index 000000000..a1aae7b87
--- /dev/null
+++ b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/random.sql
@@ -0,0 +1,17 @@
+-- rand with the seed 0
+SELECT rand(0);
+SELECT rand(cast(3 / 7 AS int));
+SELECT rand(NULL);
+SELECT rand(cast(NULL AS int));
+
+-- rand unsupported data type
+SELECT rand(1.0);
+
+-- randn with the seed 0
+SELECT randn(0L);
+SELECT randn(cast(3 / 7 AS long));
+SELECT randn(NULL);
+SELECT randn(cast(NULL AS long));
+
+-- randn unsupported data type
+SELECT rand('1')
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/inputs/table-valued-functions.sql
b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/table-valued-functions.sql
new file mode 100644
index 000000000..79d427bc2
--- /dev/null
+++
b/gluten-ut/spark35/src/test/resources/sql-tests/inputs/table-valued-functions.sql
@@ -0,0 +1,126 @@
+-- unresolved function
+select * from dummy(3);
+
+-- range call with end
+select * from range(6 + cos(3));
+
+-- range call with start and end
+select * from range(5, 10);
+
+-- range call with step
+select * from range(0, 10, 2);
+
+-- range call with numPartitions
+select * from range(0, 10, 1, 200);
+
+-- range call with invalid number of arguments
+select * from range(1, 1, 1, 1, 1);
+
+-- range call with null
+select * from range(1, null);
+
+-- range call with incompatible type
+select * from range(array(1, 2, 3));
+
+-- range call with illegal step
+select * from range(0, 5, 0);
+
+-- range call with a mixed-case function name
+select * from RaNgE(2);
+
+-- range call with alias
+select i from range(0, 2) t(i);
+
+-- explode
+select * from explode(array(1, 2));
+select * from explode(map('a', 1, 'b', 2));
+
+-- explode with empty values
+select * from explode(array());
+select * from explode(map());
+
+-- explode with column aliases
+select * from explode(array(1, 2)) t(c1);
+select * from explode(map('a', 1, 'b', 2)) t(k, v);
+
+-- explode with non-deterministic values
+select * from explode(array(rand(0)));
+
+-- explode with erroneous input
+select * from explode(null);
+select * from explode(null) t(c1);
+select * from explode(1);
+select * from explode(1, 2);
+select * from explode(explode(array(1)));
+select * from explode(array(1, 2)) t(c1, c2);
+
+-- explode_outer
+select * from explode_outer(array(1, 2));
+select * from explode_outer(map('a', 1, 'b', 2));
+select * from explode_outer(array());
+select * from explode_outer(map());
+
+-- table-valued functions with join
+select * from range(2) join explode(array(1, 2));
+select * from range(2) join explode_outer(array());
+
+-- inline
+select * from inline(array(struct(1, 'a'), struct(2, 'b')));
+select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y);
+select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a')));
+
+-- inline with erroneous input
+select * from inline(null);
+select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c);
+
+-- inline_outer
+select * from inline_outer(array(struct(1, 'a'), struct(2, 'b')));
+select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1,
'a')));
+
+-- posexplode
+select * from posexplode(array());
+select * from posexplode(array(1, 2));
+select * from posexplode(array(1, 2)) t(pos, x);
+select * from posexplode(map());
+select * from posexplode(map('a', 1, 'b', 2));
+select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v);
+
+-- posexplode with erroneous input
+select * from posexplode(1);
+select * from posexplode(1, 2);
+select * from posexplode(explode(array(1)));
+select * from posexplode(array(1, 2)) t(x);
+
+-- posexplode
+select * from posexplode_outer(array());
+select * from posexplode_outer(array(1, 2));
+select * from posexplode_outer(map());
+select * from posexplode_outer(map('a', 1, 'b', 2));
+
+-- json_tuple
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y);
+select * from json_tuple('{"a": bad, "b": string}', 'a', 'b');
+
+-- json_tuple with erroneous input
+select * from json_tuple();
+select * from json_tuple('{"a": 1}');
+select * from json_tuple('{"a": 1}', 1);
+select * from json_tuple('{"a": 1}', null);
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x);
+
+-- stack
+select * from stack(1, 1, 2, 3);
+select * from stack(2, 1, 2, 3);
+select * from stack(3, 1, 2, 3) t(x);
+select * from stack(4, 1, 2, 3) t(x);
+select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c);
+select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c);
+
+-- stack with erroneous input
+select * from stack();
+select * from stack(2, 1, 2, 3) t(a, b, c);
+select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b');
+select * from stack(2, explode(array(1, 2, 3)));
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
b/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
new file mode 100644
index 000000000..b968b4e09
--- /dev/null
+++
b/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
@@ -0,0 +1,524 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select a, sum(b) from data group by 1
+-- !query schema
+struct<a:int,sum(b):bigint>
+-- !query output
+1 3
+2 3
+3 3
+
+
+-- !query
+select 1, 2, sum(b) from data group by 1, 2
+-- !query schema
+struct<1:int,2:int,sum(b):bigint>
+-- !query output
+1 2 9
+
+
+-- !query
+select a, 1, sum(b) from data group by a, 1
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, 1, sum(b) from data group by 1, 2
+-- !query schema
+struct<a:int,1:int,sum(b):bigint>
+-- !query output
+1 1 3
+2 1 3
+3 1 3
+
+
+-- !query
+select a, b + 2, count(2) from data group by a, 2
+-- !query schema
+struct<a:int,(b + 2):int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select a as aa, b + 2 as bb, count(2) from data group by 1, 2
+-- !query schema
+struct<aa:int,bb:int,count(2):bigint>
+-- !query output
+1 3 1
+1 4 1
+2 3 1
+2 4 1
+3 3 1
+3 4 1
+
+
+-- !query
+select sum(b) from data group by 1 + 0
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
+
+
+-- !query
+select a, b from data group by -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 33,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b from data group by 0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "0",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "0"
+ } ]
+}
+
+
+-- !query
+select a, b from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, sum(b) from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "sum(data.b) AS `sum(b)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 40,
+ "stopIndex" : 40,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, sum(b) + 2 from data group by 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 44,
+ "stopIndex" : 44,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, rand(0), sum(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2
+-- !query schema
+struct<a:int,rand(0):double,sum(b):bigint>
+-- !query output
+1 0.5488135024422883 1
+1 0.7151893651681639 2
+2 0.5448831775801376 2
+2 0.6027633705776989 1
+3 0.4236547969336536 1
+3 0.6458941151817286 2
+
+
+-- !query
+select * from data group by a, b, 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "STAR_GROUP_BY_POS",
+ "sqlState" : "0A000"
+}
+
+
+-- !query
+select a, count(a) from (select 1 as a) tmp group by 1 order by 1
+-- !query schema
+struct<a:int,count(a):bigint>
+-- !query output
+1 1
+
+
+-- !query
+select count(a), a from (select 1 as a) tmp group by 2 having a > 0
+-- !query schema
+struct<count(a):bigint,a:int>
+-- !query output
+1 1
+
+
+-- !query
+select a, a AS k, count(b) from data group by k, 1
+-- !query schema
+struct<a:int,k:int,count(b):bigint>
+-- !query output
+1 1 2
+2 2 2
+3 3 2
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with cube
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, 2)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by rollup(1, b)
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by 1, 2 with rollup
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL NULL 6
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (2), (1, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2))
+-- !query schema
+struct<a:int,b:int,count(1):bigint>
+-- !query output
+1 1 1
+1 2 1
+1 NULL 2
+2 1 1
+2 2 1
+2 NULL 2
+3 1 1
+3 2 1
+3 NULL 2
+NULL 1 3
+NULL 2 3
+
+
+-- !query
+select a, b, count(1) from data group by a, -1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 46,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by a, 3
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 45,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by cube(-1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 47,
+ "stopIndex" : 48,
+ "fragment" : "-1"
+ } ]
+}
+
+
+-- !query
+select a, b, count(1) from data group by cube(1, 3)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 50,
+ "fragment" : "3"
+ } ]
+}
+
+
+-- !query
+set spark.sql.groupByOrdinal=false
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.groupByOrdinal false
+
+
+-- !query
+select sum(b) from data group by -1
+-- !query schema
+struct<sum(b):bigint>
+-- !query output
+9
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by.sql.out
b/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by.sql.out
index f56420926..a4a3f76fa 100644
--- a/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by.sql.out
+++ b/gluten-ut/spark35/src/test/resources/sql-tests/results/group-by.sql.out
@@ -907,7 +907,7 @@ GROUP BY a IS NULL
-- !query schema
struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint>
-- !query output
-0.7604953758285915 7
+0.5488135024422883 7
1.0 2
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/results/random.sql.out
b/gluten-ut/spark35/src/test/resources/sql-tests/results/random.sql.out
new file mode 100644
index 000000000..17e6f871b
--- /dev/null
+++ b/gluten-ut/spark35/src/test/resources/sql-tests/results/random.sql.out
@@ -0,0 +1,115 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT rand(0)
+-- !query schema
+struct<rand(0):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(3 / 7 AS int))
+-- !query schema
+struct<rand(CAST((3 / 7) AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(NULL)
+-- !query schema
+struct<rand(NULL):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(cast(NULL AS int))
+-- !query schema
+struct<rand(CAST(NULL AS INT)):double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+SELECT rand(1.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INT\" or \"BIGINT\")",
+ "sqlExpr" : "\"rand(1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "rand(1.0)"
+ } ]
+}
+
+
+-- !query
+SELECT randn(0L)
+-- !query schema
+struct<randn(0):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(3 / 7 AS long))
+-- !query schema
+struct<randn(CAST((3 / 7) AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(NULL)
+-- !query schema
+struct<randn(NULL):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT randn(cast(NULL AS long))
+-- !query schema
+struct<randn(CAST(NULL AS BIGINT)):double>
+-- !query output
+1.6034991609278433
+
+
+-- !query
+SELECT rand('1')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INT\" or \"BIGINT\")",
+ "sqlExpr" : "\"rand(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "rand('1')"
+ } ]
+}
diff --git
a/gluten-ut/spark35/src/test/resources/sql-tests/results/table-valued-functions.sql.out
b/gluten-ut/spark35/src/test/resources/sql-tests/results/table-valued-functions.sql.out
new file mode 100644
index 000000000..0d5675fa6
--- /dev/null
+++
b/gluten-ut/spark35/src/test/resources/sql-tests/results/table-valued-functions.sql.out
@@ -0,0 +1,1017 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+select * from dummy(3)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION",
+ "messageParameters" : {
+ "name" : "`dummy`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 22,
+ "fragment" : "dummy(3)"
+ } ]
+}
+
+
+-- !query
+select * from range(6 + cos(3))
+-- !query schema
+struct<id:bigint>
+-- !query output
+0
+1
+2
+3
+4
+
+
+-- !query
+select * from range(5, 10)
+-- !query schema
+struct<id:bigint>
+-- !query output
+5
+6
+7
+8
+9
+
+
+-- !query
+select * from range(0, 10, 2)
+-- !query schema
+struct<id:bigint>
+-- !query output
+0
+2
+4
+6
+8
+
+
+-- !query
+select * from range(0, 10, 1, 200)
+-- !query schema
+struct<id:bigint>
+-- !query output
+0
+1
+2
+3
+4
+5
+6
+7
+8
+9
+
+
+-- !query
+select * from range(1, 1, 1, 1, 1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "5",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "[1, 2, 3, 4]",
+ "functionName" : "`range`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 34,
+ "fragment" : "range(1, 1, 1, 1, 1)"
+ } ]
+}
+
+
+-- !query
+select * from range(1, null)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "functionName" : "`range`",
+ "inputSql" : "\"NULL\"",
+ "inputType" : "\"VOID\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 28,
+ "fragment" : "range(1, null)"
+ } ]
+}
+
+
+-- !query
+select * from range(array(1, 2, 3))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "functionName" : "`range`",
+ "inputSql" : "\"array(1, 2, 3)\"",
+ "inputType" : "\"ARRAY<INT>\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 35,
+ "fragment" : "range(array(1, 2, 3))"
+ } ]
+}
+
+
+-- !query
+select * from range(0, 5, 0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "FAILED_FUNCTION_CALL",
+ "sqlState" : "38000",
+ "messageParameters" : {
+ "funcName" : "`range`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 28,
+ "fragment" : "range(0, 5, 0)"
+ } ]
+}
+
+
+-- !query
+select * from RaNgE(2)
+-- !query schema
+struct<id:bigint>
+-- !query output
+0
+1
+
+
+-- !query
+select i from range(0, 2) t(i)
+-- !query schema
+struct<i:bigint>
+-- !query output
+0
+1
+
+
+-- !query
+select * from explode(array(1, 2))
+-- !query schema
+struct<col:int>
+-- !query output
+1
+2
+
+
+-- !query
+select * from explode(map('a', 1, 'b', 2))
+-- !query schema
+struct<key:string,value:int>
+-- !query output
+a 1
+b 2
+
+
+-- !query
+select * from explode(array())
+-- !query schema
+struct<col:void>
+-- !query output
+
+
+
+-- !query
+select * from explode(map())
+-- !query schema
+struct<key:void,value:void>
+-- !query output
+
+
+
+-- !query
+select * from explode(array(1, 2)) t(c1)
+-- !query schema
+struct<c1:int>
+-- !query output
+1
+2
+
+
+-- !query
+select * from explode(map('a', 1, 'b', 2)) t(k, v)
+-- !query schema
+struct<k:string,v:int>
+-- !query output
+a 1
+b 2
+
+
+-- !query
+select * from explode(array(rand(0)))
+-- !query schema
+struct<col:double>
+-- !query output
+0.5488135024422883
+
+
+-- !query
+select * from explode(null)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"NULL\"",
+ "inputType" : "\"VOID\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"ARRAY\" or \"MAP\")",
+ "sqlExpr" : "\"explode(NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 27,
+ "fragment" : "explode(null)"
+ } ]
+}
+
+
+-- !query
+select * from explode(null) t(c1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"NULL\"",
+ "inputType" : "\"VOID\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"ARRAY\" or \"MAP\")",
+ "sqlExpr" : "\"explode(NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 33,
+ "fragment" : "explode(null) t(c1)"
+ } ]
+}
+
+
+-- !query
+select * from explode(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"ARRAY\" or \"MAP\")",
+ "sqlExpr" : "\"explode(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 24,
+ "fragment" : "explode(1)"
+ } ]
+}
+
+
+-- !query
+select * from explode(1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "2",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "1",
+ "functionName" : "`explode`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 27,
+ "fragment" : "explode(1, 2)"
+ } ]
+}
+
+
+-- !query
+select * from explode(explode(array(1)))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "expression" : "\"explode(explode(array(1)))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 40,
+ "fragment" : "explode(explode(array(1)))"
+ } ]
+}
+
+
+-- !query
+select * from explode(array(1, 2)) t(c1, c2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH",
+ "messageParameters" : {
+ "aliasesNum" : "2",
+ "funcName" : "`explode`",
+ "outColsNum" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 44,
+ "fragment" : "explode(array(1, 2)) t(c1, c2)"
+ } ]
+}
+
+
+-- !query
+select * from explode_outer(array(1, 2))
+-- !query schema
+struct<col:int>
+-- !query output
+1
+2
+
+
+-- !query
+select * from explode_outer(map('a', 1, 'b', 2))
+-- !query schema
+struct<key:string,value:int>
+-- !query output
+a 1
+b 2
+
+
+-- !query
+select * from explode_outer(array())
+-- !query schema
+struct<col:void>
+-- !query output
+NULL
+
+
+-- !query
+select * from explode_outer(map())
+-- !query schema
+struct<key:void,value:void>
+-- !query output
+NULL NULL
+
+
+-- !query
+select * from range(2) join explode(array(1, 2))
+-- !query schema
+struct<id:bigint,col:int>
+-- !query output
+0 1
+0 2
+1 1
+1 2
+
+
+-- !query
+select * from range(2) join explode_outer(array())
+-- !query schema
+struct<id:bigint,col:void>
+-- !query output
+0 NULL
+1 NULL
+
+
+-- !query
+select * from inline(array(struct(1, 'a'), struct(2, 'b')))
+-- !query schema
+struct<col1:int,col2:string>
+-- !query output
+1 a
+2 b
+
+
+-- !query
+select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y)
+-- !query schema
+struct<x:int,y:string>
+-- !query output
+1 a
+2 b
+
+
+-- !query
+select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a')))
+-- !query schema
+struct<col1:int,col2:string>
+-- !query output
+
+
+
+-- !query
+select * from inline(null)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"NULL\"",
+ "inputType" : "\"VOID\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"ARRAY<STRUCT>\"",
+ "sqlExpr" : "\"inline(NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 26,
+ "fragment" : "inline(null)"
+ } ]
+}
+
+
+-- !query
+select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH",
+ "messageParameters" : {
+ "aliasesNum" : "3",
+ "funcName" : "`inline`",
+ "outColsNum" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 66,
+ "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)"
+ } ]
+}
+
+
+-- !query
+select * from inline_outer(array(struct(1, 'a'), struct(2, 'b')))
+-- !query schema
+struct<col1:int,col2:string>
+-- !query output
+1 a
+2 b
+
+
+-- !query
+select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a')))
+-- !query schema
+struct<col1:int,col2:string>
+-- !query output
+NULL NULL
+
+
+-- !query
+select * from posexplode(array())
+-- !query schema
+struct<pos:int,col:void>
+-- !query output
+
+
+
+-- !query
+select * from posexplode(array(1, 2))
+-- !query schema
+struct<pos:int,col:int>
+-- !query output
+0 1
+1 2
+
+
+-- !query
+select * from posexplode(array(1, 2)) t(pos, x)
+-- !query schema
+struct<pos:int,x:int>
+-- !query output
+0 1
+1 2
+
+
+-- !query
+select * from posexplode(map())
+-- !query schema
+struct<pos:int,key:void,value:void>
+-- !query output
+
+
+
+-- !query
+select * from posexplode(map('a', 1, 'b', 2))
+-- !query schema
+struct<pos:int,key:string,value:int>
+-- !query output
+0 a 1
+1 b 2
+
+
+-- !query
+select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v)
+-- !query schema
+struct<pos:int,k:string,v:int>
+-- !query output
+0 a 1
+1 b 2
+
+
+-- !query
+select * from posexplode(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"ARRAY\" or \"MAP\")",
+ "sqlExpr" : "\"posexplode(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 27,
+ "fragment" : "posexplode(1)"
+ } ]
+}
+
+
+-- !query
+select * from posexplode(1, 2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "2",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "1",
+ "functionName" : "`posexplode`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 30,
+ "fragment" : "posexplode(1, 2)"
+ } ]
+}
+
+
+-- !query
+select * from posexplode(explode(array(1)))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "expression" : "\"posexplode(explode(array(1)))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 43,
+ "fragment" : "posexplode(explode(array(1)))"
+ } ]
+}
+
+
+-- !query
+select * from posexplode(array(1, 2)) t(x)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH",
+ "messageParameters" : {
+ "aliasesNum" : "1",
+ "funcName" : "`posexplode`",
+ "outColsNum" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 42,
+ "fragment" : "posexplode(array(1, 2)) t(x)"
+ } ]
+}
+
+
+-- !query
+select * from posexplode_outer(array())
+-- !query schema
+struct<pos:int,col:void>
+-- !query output
+NULL NULL
+
+
+-- !query
+select * from posexplode_outer(array(1, 2))
+-- !query schema
+struct<pos:int,col:int>
+-- !query output
+0 1
+1 2
+
+
+-- !query
+select * from posexplode_outer(map())
+-- !query schema
+struct<pos:int,key:void,value:void>
+-- !query output
+NULL NULL NULL
+
+
+-- !query
+select * from posexplode_outer(map('a', 1, 'b', 2))
+-- !query schema
+struct<pos:int,key:string,value:int>
+-- !query output
+0 a 1
+1 b 2
+
+
+-- !query
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b')
+-- !query schema
+struct<c0:string,c1:string>
+-- !query output
+1 2
+
+
+-- !query
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c')
+-- !query schema
+struct<c0:string,c1:string>
+-- !query output
+1 NULL
+
+
+-- !query
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a')
+-- !query schema
+struct<c0:string,c1:string>
+-- !query output
+1 1
+
+
+-- !query
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y)
+-- !query schema
+struct<x:string,y:string>
+-- !query output
+1 2
+
+
+-- !query
+select * from json_tuple('{"a": bad, "b": string}', 'a', 'b')
+-- !query schema
+struct<c0:string,c1:string>
+-- !query output
+NULL NULL
+
+
+-- !query
+select * from json_tuple()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "> 1",
+ "functionName" : "`json_tuple`"
+ }
+}
+
+
+-- !query
+select * from json_tuple('{"a": 1}')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "> 1",
+ "functionName" : "`json_tuple`"
+ }
+}
+
+
+-- !query
+select * from json_tuple('{"a": 1}', 1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "funcName" : "`json_tuple`",
+ "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 39,
+ "fragment" : "json_tuple('{\"a\": 1}', 1)"
+ } ]
+}
+
+
+-- !query
+select * from json_tuple('{"a": 1}', null)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "funcName" : "`json_tuple`",
+ "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 42,
+ "fragment" : "json_tuple('{\"a\": 1}', null)"
+ } ]
+}
+
+
+-- !query
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH",
+ "messageParameters" : {
+ "aliasesNum" : "1",
+ "funcName" : "`json_tuple`",
+ "outColsNum" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 62,
+ "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)"
+ } ]
+}
+
+
+-- !query
+select * from stack(1, 1, 2, 3)
+-- !query schema
+struct<col0:int,col1:int,col2:int>
+-- !query output
+1 2 3
+
+
+-- !query
+select * from stack(2, 1, 2, 3)
+-- !query schema
+struct<col0:int,col1:int>
+-- !query output
+1 2
+3 NULL
+
+
+-- !query
+select * from stack(3, 1, 2, 3) t(x)
+-- !query schema
+struct<x:int>
+-- !query output
+1
+2
+3
+
+
+-- !query
+select * from stack(4, 1, 2, 3) t(x)
+-- !query schema
+struct<x:int>
+-- !query output
+1
+2
+3
+NULL
+
+
+-- !query
+select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c)
+-- !query schema
+struct<a:int,b:decimal(2,1),c:string>
+-- !query output
+1 1.1 a
+2 2.2 b
+
+
+-- !query
+select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c)
+-- !query schema
+struct<a:int,b:decimal(2,1),c:string>
+-- !query output
+1 1.1 NULL
+2 NULL b
+
+
+-- !query
+select * from stack()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "> 1",
+ "functionName" : "`stack`"
+ }
+}
+
+
+-- !query
+select * from stack(2, 1, 2, 3) t(a, b, c)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH",
+ "messageParameters" : {
+ "aliasesNum" : "3",
+ "funcName" : "`stack`",
+ "outColsNum" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 42,
+ "fragment" : "stack(2, 1, 2, 3) t(a, b, c)"
+ } ]
+}
+
+
+-- !query
+select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.ExtendedAnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "columnIndex" : "1",
+ "leftParamIndex" : "2",
+ "leftType" : "\"STRING\"",
+ "rightParamIndex" : "5",
+ "rightType" : "\"DECIMAL(2,1)\"",
+ "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 50,
+ "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')"
+ } ]
+}
+
+
+-- !query
+select * from stack(2, explode(array(1, 2, 3)))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "expression" : "\"stack(2, explode(array(1, 2, 3)))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 47,
+ "fragment" : "stack(2, explode(array(1, 2, 3)))"
+ } ]
+}
diff --git
a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
index 40ecc3c35..76b666779 100644
---
a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
+++
b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala
@@ -1132,6 +1132,7 @@ class VeloxTestSettings extends BackendTestSettings {
.exclude("SPARK-41017: filter pushdown with nondeterministic predicates")
enableSuite[GlutenFileScanSuite]
enableSuite[GlutenGeneratorFunctionSuite]
+ .exclude("SPARK-45171: Handle evaluated nondeterministic expression")
enableSuite[GlutenInjectRuntimeFilterSuite]
// FIXME: yan
.exclude("Merge runtime bloom filters")
diff --git
a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala
b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala
index 2421e918b..b3d51e802 100644
---
a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala
+++
b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala
@@ -16,4 +16,13 @@
*/
package org.apache.spark.sql
-class GlutenGeneratorFunctionSuite extends GeneratorFunctionSuite with
GlutenSQLTestsTrait {}
+import org.apache.spark.sql.internal.SQLConf
+
+class GlutenGeneratorFunctionSuite extends GeneratorFunctionSuite with
GlutenSQLTestsTrait {
+ testGluten("SPARK-45171: Handle evaluated nondeterministic expression") {
+ withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") {
+ val df = sql("select explode(array(rand(0)))")
+ checkAnswer(df, Row(0.5488135024422883))
+ }
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]