From 5b8e848bbfbc0c99a5faf758e40b188b0bbebb7b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun <dongjoon@apache.org> Date: Wed, 27 Jul 2016 13:23:59 +0800 Subject: [PATCH] [SPARK-16621][SQL] Generate stable SQLs in SQLBuilder ## What changes were proposed in this pull request? Currently, the generated SQLs have not-stable IDs for generated attributes. The stable generated SQL will give more benefit for understanding or testing the queries. This PR provides stable SQL generation by the followings. - Provide unique ids for generated subqueries, `gen_subquery_xxx`. - Provide unique and stable ids for generated attributes, `gen_attr_xxx`. **Before** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res0: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4`) AS gen_subquery_0 ``` **After** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res2: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 ``` ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14257 from dongjoon-hyun/SPARK-16621. --- .../spark/sql/catalyst/SQLBuilder.scala | 23 +++++++++++++++---- sql/hive/src/test/resources/sqlgen/agg1.sql | 2 +- sql/hive/src/test/resources/sqlgen/agg2.sql | 2 +- sql/hive/src/test/resources/sqlgen/agg3.sql | 2 +- .../sqlgen/aggregate_functions_and_window.sql | 2 +- sql/hive/src/test/resources/sqlgen/case.sql | 2 +- .../test/resources/sqlgen/case_with_else.sql | 2 +- .../test/resources/sqlgen/case_with_key.sql | 2 +- .../sqlgen/case_with_key_and_else.sql | 2 +- .../src/test/resources/sqlgen/cluster_by.sql | 2 +- .../sqlgen/data_source_json_parquet_t0.sql | 2 +- .../sqlgen/data_source_orc_parquet_t0.sql | 2 +- .../sqlgen/data_source_parquet_parquet_t0.sql | 2 +- .../resources/sqlgen/distinct_aggregation.sql | 2 +- .../test/resources/sqlgen/distribute_by.sql | 2 +- .../sqlgen/distribute_by_with_sort_by.sql | 2 +- sql/hive/src/test/resources/sqlgen/except.sql | 2 +- .../sqlgen/filter_after_subquery.sql | 2 +- .../sqlgen/generate_with_other_1.sql | 2 +- .../sqlgen/generate_with_other_2.sql | 2 +- .../sqlgen/generator_in_lateral_view_1.sql | 2 +- .../sqlgen/generator_in_lateral_view_2.sql | 2 +- .../generator_non_referenced_table_1.sql | 2 +- .../generator_non_referenced_table_2.sql | 2 +- .../resources/sqlgen/generator_non_udtf_1.sql | 2 +- .../resources/sqlgen/generator_non_udtf_2.sql | 2 +- .../sqlgen/generator_referenced_table_1.sql | 2 +- .../sqlgen/generator_referenced_table_2.sql | 2 +- .../generator_with_ambiguous_names_1.sql | 2 +- .../generator_with_ambiguous_names_2.sql | 2 +- .../sqlgen/generator_without_from_1.sql | 2 +- .../sqlgen/generator_without_from_2.sql | 2 +- .../test/resources/sqlgen/grouping_sets_1.sql | 2 +- .../resources/sqlgen/grouping_sets_2_1.sql | 2 +- .../resources/sqlgen/grouping_sets_2_2.sql | 2 +- .../resources/sqlgen/grouping_sets_2_3.sql | 2 +- .../resources/sqlgen/grouping_sets_2_4.sql | 2 +- .../resources/sqlgen/grouping_sets_2_5.sql | 2 +- sql/hive/src/test/resources/sqlgen/in.sql | 2 +- .../src/test/resources/sqlgen/intersect.sql | 2 +- .../test/resources/sqlgen/join_2_tables.sql | 2 +- .../sqlgen/json_tuple_generator_1.sql | 2 +- .../sqlgen/json_tuple_generator_2.sql | 2 +- .../test/resources/sqlgen/multi_distinct.sql | 2 +- .../nested_generator_in_lateral_view_1.sql | 2 +- .../nested_generator_in_lateral_view_2.sql | 2 +- sql/hive/src/test/resources/sqlgen/not_in.sql | 2 +- .../src/test/resources/sqlgen/not_like.sql | 2 +- .../resources/sqlgen/predicate_subquery.sql | 2 +- .../sqlgen/regular_expressions_and_window.sql | 2 +- .../test/resources/sqlgen/rollup_cube_1_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_1_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_2_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_2_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_3_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_3_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_4_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_4_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_5_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_5_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_3.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_4.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_5.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_6.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_7_3.sql | 2 +- .../test/resources/sqlgen/rollup_cube_8_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_8_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_9_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_9_2.sql | 2 +- .../sqlgen/script_transformation_1.sql | 2 +- .../sqlgen/script_transformation_2.sql | 2 +- .../script_transformation_alias_list.sql | 2 +- ...pt_transformation_alias_list_with_type.sql | 2 +- ...ipt_transformation_row_format_multiple.sql | 2 +- .../script_transformation_row_format_one.sql | 2 +- ...script_transformation_row_format_serde.sql | 2 +- ...ransformation_row_format_without_serde.sql | 2 +- .../test/resources/sqlgen/select_distinct.sql | 2 +- .../resources/sqlgen/select_orc_table.sql | 2 +- .../resources/sqlgen/select_parquet_table.sql | 2 +- .../src/test/resources/sqlgen/self_join.sql | 2 +- .../sqlgen/self_join_with_group_by.sql | 2 +- .../resources/sqlgen/sort_by_after_having.sql | 2 +- .../test/resources/sqlgen/tablesample_1.sql | 2 +- .../test/resources/sqlgen/tablesample_2.sql | 2 +- .../test/resources/sqlgen/tablesample_3.sql | 2 +- .../test/resources/sqlgen/tablesample_4.sql | 2 +- .../test/resources/sqlgen/tablesample_5.sql | 2 +- .../test/resources/sqlgen/tablesample_6.sql | 2 +- .../resources/sqlgen/three_child_union.sql | 2 +- .../test/resources/sqlgen/type_widening.sql | 2 +- .../test/resources/sqlgen/union_distinct.sql | 2 +- .../test/resources/sqlgen/window_basic_1.sql | 2 +- .../test/resources/sqlgen/window_basic_2.sql | 2 +- .../test/resources/sqlgen/window_basic_3.sql | 2 +- .../resources/sqlgen/window_with_join.sql | 2 +- .../window_with_the_same_window_with_agg.sql | 2 +- ...w_with_the_same_window_with_agg_filter.sql | 2 +- ...ith_the_same_window_with_agg_functions.sql | 2 +- ...w_with_the_same_window_with_agg_having.sql | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 9 +++----- 105 files changed, 124 insertions(+), 114 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index 9a02e3c8f3..5d93419f35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable.Map import scala.util.control.NonFatal import org.apache.spark.internal.Logging @@ -38,14 +39,23 @@ import org.apache.spark.sql.types.{ByteType, DataType, IntegerType, NullType} * representations (e.g. logical plans that operate on local Scala collections), or are simply not * supported by this builder (yet). */ -class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { +class SQLBuilder private ( + logicalPlan: LogicalPlan, + nextSubqueryId: AtomicLong, + nextGenAttrId: AtomicLong, + exprIdMap: Map[Long, Long]) extends Logging { require(logicalPlan.resolved, "SQLBuilder only supports resolved logical query plans. Current plan:\n" + logicalPlan) + def this(logicalPlan: LogicalPlan) = + this(logicalPlan, new AtomicLong(0), new AtomicLong(0), Map.empty[Long, Long]) + def this(df: Dataset[_]) = this(df.queryExecution.analyzed) - private val nextSubqueryId = new AtomicLong(0) private def newSubqueryName(): String = s"gen_subquery_${nextSubqueryId.getAndIncrement()}" + private def normalizedName(n: NamedExpression): String = synchronized { + "gen_attr_" + exprIdMap.getOrElseUpdate(n.exprId.id, nextGenAttrId.getAndIncrement()) + } def toSQL: String = { val canonicalizedPlan = Canonicalizer.execute(logicalPlan) @@ -70,7 +80,7 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { try { val replaced = finalPlan.transformAllExpressions { case s: SubqueryExpression => - val query = new SQLBuilder(s.query).toSQL + val query = new SQLBuilder(s.query, nextSubqueryId, nextGenAttrId, exprIdMap).toSQL val sql = s match { case _: ListQuery => query case _: Exists => s"EXISTS($query)" @@ -169,6 +179,11 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { qualifiedName + " TABLESAMPLE(" + fraction + " PERCENT)" }.getOrElse(qualifiedName) + case relation: CatalogRelation => + val m = relation.catalogTable + val qualifiedName = s"${quoteIdentifier(m.database)}.${quoteIdentifier(m.identifier.table)}" + qualifiedName + case Sort(orders, _, RepartitionByExpression(partitionExprs, child, _)) if orders.map(_.child) == partitionExprs => build(toSQL(child), "CLUSTER BY", partitionExprs.map(_.sql).mkString(", ")) @@ -376,8 +391,6 @@ class SQLBuilder(logicalPlan: LogicalPlan) extends Logging { ) } - private def normalizedName(n: NamedExpression): String = "gen_attr_" + n.exprId.id - object Canonicalizer extends RuleExecutor[LogicalPlan] { override protected def batches: Seq[Batch] = Seq( Batch("Prepare", FixedPoint(100), diff --git a/sql/hive/src/test/resources/sqlgen/agg1.sql b/sql/hive/src/test/resources/sqlgen/agg1.sql index 9953640a6f..05403a9dd8 100644 --- a/sql/hive/src/test/resources/sqlgen/agg1.sql +++ b/sql/hive/src/test/resources/sqlgen/agg1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` HAVING (`gen_attr_1` > CAST(0 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg2.sql b/sql/hive/src/test/resources/sqlgen/agg2.sql index 0caea28b02..65d71714fe 100644 --- a/sql/hive/src/test/resources/sqlgen/agg2.sql +++ b/sql/hive/src/test/resources/sqlgen/agg2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` ORDER BY `gen_attr_1` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg3.sql b/sql/hive/src/test/resources/sqlgen/agg3.sql index 437afa7790..14b19392cd 100644 --- a/sql/hive/src/test/resources/sqlgen/agg3.sql +++ b/sql/hive/src/test/resources/sqlgen/agg3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` ASC, `gen_attr_2` ASC) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql index c94f53b0a9..e3e372d5ec 100644 --- a/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/aggregate_functions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(c) + COUNT(a) OVER () FROM parquet_t2 GROUP BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, count(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT (`gen_attr_1` + `gen_attr_2`) AS `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, count(`gen_attr_3`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_2` FROM (SELECT max(`gen_attr_5`) AS `gen_attr_1`, `gen_attr_3` FROM (SELECT `a` AS `gen_attr_3`, `b` AS `gen_attr_4`, `c` AS `gen_attr_5`, `d` AS `gen_attr_6` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_4`) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/case.sql b/sql/hive/src/test/resources/sqlgen/case.sql index f6a8c32cf0..99630e88cf 100644 --- a/sql/hive/src/test/resources/sqlgen/case.sql +++ b/sql/hive/src/test/resources/sqlgen/case.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE WHEN id % 2 > 0 THEN 0 WHEN id % 2 = 0 THEN 1 END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((id % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((`gen_attr` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((id % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END` FROM (SELECT CASE WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_else.sql index 8f1595d4b8..aed8f08804 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_else.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_else.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE WHEN id % 2 > 0 THEN 0 ELSE 1 END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM (SELECT CASE WHEN ((`gen_attr` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM (SELECT CASE WHEN ((`gen_attr_1` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key.sql b/sql/hive/src/test/resources/sqlgen/case_with_key.sql index 88353e321e..dff65f1083 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql index bc44f25d1f..af3e169b54 100644 --- a/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql +++ b/sql/hive/src/test/resources/sqlgen/case_with_key_and_else.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT CASE id WHEN 0 THEN 'foo' WHEN 1 THEN 'bar' ELSE 'baz' END FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM (SELECT CASE WHEN (`gen_attr_1` = CAST(0 AS BIGINT)) THEN "foo" WHEN (`gen_attr_1` = CAST(1 AS BIGINT)) THEN "bar" ELSE "baz" END AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/cluster_by.sql b/sql/hive/src/test/resources/sqlgen/cluster_by.sql index 3bc1dfc581..3154791c3c 100644 --- a/sql/hive/src/test/resources/sqlgen/cluster_by.sql +++ b/sql/hive/src/test/resources/sqlgen/cluster_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 CLUSTER BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql index a4f46da9ff..e41b645937 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_json_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM json_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`json_parquet_t0`) AS gen_subquery_0) AS json_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`json_parquet_t0`) AS gen_subquery_0) AS json_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql index 214dbcf236..f5ceccde8c 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_orc_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM orc_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`orc_parquet_t0`) AS gen_subquery_0) AS orc_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`orc_parquet_t0`) AS gen_subquery_0) AS orc_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql index eccf03eb12..2bccefe55e 100644 --- a/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql +++ b/sql/hive/src/test/resources/sqlgen/data_source_parquet_parquet_t0.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_parquet_t0`) AS gen_subquery_0) AS parquet_parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_parquet_t0`) AS gen_subquery_0) AS parquet_parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql index 0fa21ad3ba..bced711cae 100644 --- a/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql +++ b/sql/hive/src/test/resources/sqlgen/distinct_aggregation.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(DISTINCT id) FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(DISTINCT id)` FROM (SELECT count(DISTINCT `gen_attr`) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_0` AS `count(DISTINCT id)` FROM (SELECT count(DISTINCT `gen_attr_1`) AS `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by.sql index 75a1a2a6b6..72863dcaf5 100644 --- a/sql/hive/src/test/resources/sqlgen/distribute_by.sql +++ b/sql/hive/src/test/resources/sqlgen/distribute_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 DISTRIBUTE BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 DISTRIBUTE BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 DISTRIBUTE BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql index a8e766e81d..96b9b2dae8 100644 --- a/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql +++ b/sql/hive/src/test/resources/sqlgen/distribute_by_with_sort_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 DISTRIBUTE BY id SORT BY id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr`) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 CLUSTER BY `gen_attr_0`) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/except.sql b/sql/hive/src/test/resources/sqlgen/except.sql index b1c64288d6..7a7d27fcd6 100644 --- a/sql/hive/src/test/resources/sqlgen/except.sql +++ b/sql/hive/src/test/resources/sqlgen/except.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 EXCEPT SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) EXCEPT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 ) EXCEPT ( SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql index 0156dffd0a..9cd6514d77 100644 --- a/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/filter_after_subquery.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a FROM (SELECT key + 1 AS a FROM parquet_t1) t WHERE a > 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t WHERE (`gen_attr` > CAST(5 AS BIGINT))) AS t +SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT (`gen_attr_1` + CAST(1 AS BIGINT)) AS `gen_attr_0` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t WHERE (`gen_attr_0` > CAST(5 AS BIGINT))) AS t diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql index 15d23c67ae..805197a4ea 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql @@ -5,4 +5,4 @@ WHERE id > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr`, gen_subquery_0.`gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS parquet_t3 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_0.`gen_attr_2`, gen_subquery_0.`gen_attr_3`, gen_subquery_0.`gen_attr_4`, gen_subquery_0.`gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr_1` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql index e2b94ce673..ef9a596197 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql @@ -7,4 +7,4 @@ WHERE val > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr` WHERE (`gen_attr` > CAST(2 AS BIGINT)) ORDER BY `gen_attr` ASC, `gen_attr` ASC LIMIT 5) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0` WHERE (`gen_attr_0` > CAST(2 AS BIGINT)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql index c62b895c67..2f6596ef42 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT val, id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql index 4c68eef214..239980dd80 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_in_lateral_view_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT val, id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql index 059c45ec47..7fe0298c8e 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql index 8f03dedbf9..8db834acc7 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_referenced_table_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) AS val FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_0 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql index 0bfe38036e..fef65e0068 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr), id FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_1 AS `gen_attr`) AS parquet_t3 +SELECT `gen_attr_0` AS `col`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_1 AS `gen_attr_0`) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql index e4edb6876c..e0e310888f 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_non_udtf_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) AS val, id as a FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `a` FROM (SELECT `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `a` FROM (SELECT `gen_attr_0`, `gen_attr_2` AS `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_3`, `arr2` AS `gen_attr_4`, `json` AS `gen_attr_5`, `id` AS `gen_attr_2` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql index 5cf57f82d2..ea5db850be 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT `arr` AS `gen_attr_1`, `arr2` AS `gen_attr_2`, `json` AS `gen_attr_3`, `id` AS `gen_attr_4` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_1`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql index ed42894807..8f75b82547 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_referenced_table_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(arr) AS val FROM parquet_t3 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT `arr` AS `gen_attr_1`, `arr2` AS `gen_attr_2`, `json` AS `gen_attr_3`, `id` AS `gen_attr_4` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_1`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql index 5a4aec961a..984cce8a2c 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_1.sql @@ -3,4 +3,4 @@ SELECT exp.id, parquet_t3.id FROM parquet_t3 LATERAL VIEW EXPLODE(arr) exp AS id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `id`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql index 3c3e858ded..5c55b164c7 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_with_ambiguous_names_2.sql @@ -3,4 +3,4 @@ SELECT exp.id, parquet_t3.id FROM parquet_t3 LATERAL VIEW OUTER EXPLODE(arr) exp AS id -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_2 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `id`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql index d56e89c8bb..ee22fe8728 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `col` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 +SELECT `gen_attr_0` AS `col` FROM (SELECT `gen_attr_0` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql index d7acb69240..0acded74b3 100644 --- a/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generator_without_from_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT EXPLODE(ARRAY(1,2,3)) AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val` FROM (SELECT `gen_attr` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr`) AS gen_subquery_0 +SELECT `gen_attr_0` AS `val` FROM (SELECT `gen_attr_0` FROM (SELECT 1) gen_subquery_1 LATERAL VIEW explode(array(1, 2, 3)) gen_subquery_2 AS `gen_attr_0`) AS gen_subquery_0 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql index 72e429fe05..db2b2cc732 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_1.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id() AS k3 FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 GROUPING SETS (key % 5, key - 5) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))))) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))))) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql index 25e86e73cb..b2c426c660 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`), (`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`), (`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql index 24bf020434..96ee8e8595 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql index f72bc8acd6..9b8b230c87 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql index 2fac7f6618..c35db74a5c 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS(()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS(()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql index 2e60c9c4d2..e47f6d5dcf 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql @@ -2,4 +2,4 @@ SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS ((), (a), (a, b)) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((), (`gen_attr`), (`gen_attr`, `gen_attr`)) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((), (`gen_attr_5`), (`gen_attr_5`, `gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/in.sql b/sql/hive/src/test/resources/sqlgen/in.sql index 1e0251e586..7cff62b1af 100644 --- a/sql/hive/src/test/resources/sqlgen/in.sql +++ b/sql/hive/src/test/resources/sqlgen/in.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 WHERE id IN (1, 2, 3) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0 WHERE (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0 WHERE (CAST(`gen_attr_0` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/intersect.sql b/sql/hive/src/test/resources/sqlgen/intersect.sql index 608fa59f8a..4143a6208d 100644 --- a/sql/hive/src/test/resources/sqlgen/intersect.sql +++ b/sql/hive/src/test/resources/sqlgen/intersect.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 INTERSECT SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 ) INTERSECT ( SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 ) INTERSECT ( SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/join_2_tables.sql b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql index bb99797b8b..9dd200c3c0 100644 --- a/sql/hive/src/test/resources/sqlgen/join_2_tables.sql +++ b/sql/hive/src/test/resources/sqlgen/join_2_tables.sql @@ -4,4 +4,4 @@ FROM parquet_t1 a, parquet_t1 b GROUP BY a.KEY, b.KEY HAVING MAX(a.KEY) > 0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)`, `gen_attr` AS `KEY`, `gen_attr` AS `KEY` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, `gen_attr`, `gen_attr`, max(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `count(value)`, `gen_attr_1` AS `KEY`, `gen_attr_2` AS `KEY` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, max(`gen_attr_2`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 GROUP BY `gen_attr_2`, `gen_attr_1` HAVING (`gen_attr_3` > CAST(0 AS BIGINT))) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql index 9e5fe1156e..6f5562a20c 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_1.sql @@ -3,4 +3,4 @@ SELECT c0, c1, c2 FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c0`, `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt +SELECT `gen_attr_0` AS `c0`, `gen_attr_1` AS `c1`, `gen_attr_2` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql index 55dd491a32..0d4f67f184 100644 --- a/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql +++ b/sql/hive/src/test/resources/sqlgen/json_tuple_generator_2.sql @@ -3,4 +3,4 @@ SELECT a, b, c FROM parquet_t3 LATERAL VIEW JSON_TUPLE(json, 'f1', 'f2', 'f3') jt AS a, b, c -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `c` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr`, `gen_attr`, `gen_attr`) AS jt +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_2` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_5`, `json` AS `gen_attr_3`, `id` AS `gen_attr_6` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW json_tuple(`gen_attr_3`, "f1", "f2", "f3") gen_subquery_1 AS `gen_attr_0`, `gen_attr_1`, `gen_attr_2`) AS jt diff --git a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql index b5c7cf0b0c..3ca526fcc4 100644 --- a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM parquet_t2 GROUP BY a -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `count(DISTINCT b)`, `gen_attr` AS `count(DISTINCT c)`, `gen_attr` AS `sum(d)` FROM (SELECT `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, count(DISTINCT `gen_attr`) AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`) AS parquet_t2 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `count(DISTINCT b)`, `gen_attr_3` AS `count(DISTINCT c)`, `gen_attr_5` AS `sum(d)` FROM (SELECT `gen_attr_0`, count(DISTINCT `gen_attr_2`) AS `gen_attr_1`, count(DISTINCT `gen_attr_4`) AS `gen_attr_3`, sum(`gen_attr_6`) AS `gen_attr_5` FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_2`, `c` AS `gen_attr_4`, `d` AS `gen_attr_6` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_0`) AS parquet_t2 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql index 3d79f95654..e681c2b635 100644 --- a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_1.sql @@ -4,4 +4,4 @@ FROM parquet_t3 LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array LATERAL VIEW EXPLODE(nested_array) exp1 AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql index 18b29a7bc5..e9d6522c91 100644 --- a/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql +++ b/sql/hive/src/test/resources/sqlgen/nested_generator_in_lateral_view_2.sql @@ -4,4 +4,4 @@ FROM parquet_t3 LATERAL VIEW EXPLODE(arr2) exp1 AS nested_array LATERAL VIEW OUTER EXPLODE(nested_array) exp1 AS val -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `val`, `gen_attr` AS `id` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `arr` AS `gen_attr`, `arr2` AS `gen_attr`, `json` AS `gen_attr`, `id` AS `gen_attr` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr`) gen_subquery_2 AS `gen_attr` LATERAL VIEW OUTER explode(`gen_attr`) gen_subquery_3 AS `gen_attr`) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW OUTER explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0`) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/not_in.sql b/sql/hive/src/test/resources/sqlgen/not_in.sql index a50ee282e5..797d22e8e9 100644 --- a/sql/hive/src/test/resources/sqlgen/not_in.sql +++ b/sql/hive/src/test/resources/sqlgen/not_in.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM t0 WHERE id NOT IN (1, 2, 3) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT (CAST(`gen_attr` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))))) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT (CAST(`gen_attr_0` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))))) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/not_like.sql b/sql/hive/src/test/resources/sqlgen/not_like.sql index 2f66695189..da39a62225 100644 --- a/sql/hive/src/test/resources/sqlgen/not_like.sql +++ b/sql/hive/src/test/resources/sqlgen/not_like.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM t0 WHERE id + 5 NOT LIKE '1%' -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0 WHERE (NOT CAST((`gen_attr_0` + CAST(5 AS BIGINT)) AS STRING) LIKE "1%")) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql index 2e06b4fe77..6e5bd98600 100644 --- a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from t1 b where exists (select * from t1 a) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a` FROM (SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr` AS `a` FROM ((SELECT `gen_attr` FROM (SELECT `a` AS `gen_attr` FROM `default`.`t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_1)) AS b +SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT `a` AS `gen_attr_0` FROM `default`.`t1`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_1` AS `a` FROM ((SELECT `gen_attr_1` FROM (SELECT `a` AS `gen_attr_1` FROM `default`.`t1`) AS gen_subquery_2) AS gen_subquery_1) AS gen_subquery_1)) AS b diff --git a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql index a7909485fb..37cd5568ba 100644 --- a/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql +++ b/sql/hive/src/test/resources/sqlgen/regular_expressions_and_window.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(key) OVER (PARTITION BY key % 3) + key FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT (`gen_attr_1` + `gen_attr_2`) AS `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_1` FROM (SELECT `gen_attr_2`, (`gen_attr_2` % CAST(3 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql index 4b2c78ea40..c54963ab5c 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_2` AS `cnt`, `gen_attr_3` AS `(key % CAST(5 AS BIGINT))`, `gen_attr_4` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, grouping_id() AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_6` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_5` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_5` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql index 62cf1bb9d8..6c869063c3 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_1_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT count(*) as cnt, key%5, grouping_id() FROM parquet_t1 GROUP BY key % 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `(key % CAST(5 AS BIGINT))`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_2` AS `cnt`, `gen_attr_3` AS `(key % CAST(5 AS BIGINT))`, `gen_attr_4` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, grouping_id() AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_6` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_5` % CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_5` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql index d58910af69..9628e38572 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_3` AS `count(value)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_4` AS `gen_attr_1`, count(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_4` GROUPING SETS((`gen_attr_5`, `gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql index 9a58c15203..d6b61929df 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_2_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, value, count(value) FROM parquet_t1 GROUP BY key, value WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_3` AS `count(value)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_4` AS `gen_attr_1`, count(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_4` GROUPING SETS((`gen_attr_5`, `gen_attr_4`), (`gen_attr_5`), (`gen_attr_4`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql index cd4d428b07..d04b6578fc 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_3` AS `count(value)`, `gen_attr_5` AS `grouping_id()` FROM (SELECT `gen_attr_6` AS `gen_attr_0`, count(`gen_attr_4`) AS `gen_attr_3`, grouping_id() AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_6`, `gen_attr_4` GROUPING SETS((`gen_attr_6`, `gen_attr_4`), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql index 4f42d7eda8..80a5d93438 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_3_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT key, count(value), grouping_id() FROM parquet_t1 GROUP BY key, value WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(value)`, `gen_attr` AS `grouping_id()` FROM (SELECT `gen_attr` AS `gen_attr`, count(`gen_attr`) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `key`, `gen_attr_3` AS `count(value)`, `gen_attr_5` AS `grouping_id()` FROM (SELECT `gen_attr_6` AS `gen_attr_0`, count(`gen_attr_4`) AS `gen_attr_3`, grouping_id() AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_6`, `gen_attr_4` GROUPING SETS((`gen_attr_6`, `gen_attr_4`), (`gen_attr_6`), (`gen_attr_4`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql index be4908ea16..619a554875 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_1.sql @@ -2,4 +2,4 @@ SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 GROUP BY key % 5, key - 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_8` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql index eebef6a5f3..8bf1645191 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_4_2.sql @@ -2,4 +2,4 @@ SELECT count(*) as cnt, key % 5 as k1, key - 5 as k2, grouping_id() FROM parquet_t1 GROUP BY key % 5, key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `grouping_id()` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_8` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql index 9474233cb7..17e78a0a70 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_1.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - FROM (SELECT key, key%2, key - 5 FROM parquet_t1) t GROUP BY key%5, key-5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql index d36f43d665..72506ef72a 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_5_2.sql @@ -3,4 +3,4 @@ SELECT count(*) AS cnt, key % 5 AS k1, key - 5 AS k2, grouping_id(key % 5, key - FROM (SELECT key, key % 2, key - 5 FROM parquet_t1) t GROUP BY key % 5, key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `cnt`, `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `k3` FROM (SELECT count(1) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(2 AS BIGINT)) AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr` % CAST(5 AS BIGINT)), (`gen_attr` - CAST(5 AS BIGINT))), ((`gen_attr` % CAST(5 AS BIGINT))), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `cnt`, `gen_attr_4` AS `k1`, `gen_attr_5` AS `k2`, `gen_attr_6` AS `k3` FROM (SELECT count(1) AS `gen_attr_3`, (`gen_attr_7` % CAST(5 AS BIGINT)) AS `gen_attr_4`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_5`, grouping_id() AS `gen_attr_6` FROM (SELECT `gen_attr_7`, (`gen_attr_7` % CAST(2 AS BIGINT)) AS `gen_attr_8`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_9` FROM (SELECT `key` AS `gen_attr_7`, `value` AS `gen_attr_12` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY (`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS(((`gen_attr_7` % CAST(5 AS BIGINT)), (`gen_attr_7` - CAST(5 AS BIGINT))), ((`gen_attr_7` % CAST(5 AS BIGINT))), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql index de980784c7..22df578518 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql index 8956ac4c42..f44b652343 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(c)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), (`gen_attr_6`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql index b9cef62dcf..40f6924913 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql index ba6457ab1a..608e644dee 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `sum(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ()) ORDER BY `gen_attr` ASC, `gen_attr` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql index 763fecd951..26885a26e2 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_5.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `(a + b)`, `gen_attr_1` AS `b`, `gen_attr_4` AS `sum((a - b))` FROM (SELECT (`gen_attr_5` + `gen_attr_6`) AS `gen_attr_3`, `gen_attr_6` AS `gen_attr_1`, sum((`gen_attr_5` - `gen_attr_6`)) AS `gen_attr_4` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_7`, `d` AS `gen_attr_8` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr_5` + `gen_attr_6`), `gen_attr_6` GROUPING SETS(((`gen_attr_5` + `gen_attr_6`), `gen_attr_6`), ((`gen_attr_5` + `gen_attr_6`)), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql index 00ecbf6078..dd97c976af 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_6.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a + b, b, sum(a - b) FROM parquet_t2 GROUP BY a + b, b WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(a + b)`, `gen_attr` AS `b`, `gen_attr` AS `sum((a - b))` FROM (SELECT (`gen_attr` + `gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum((`gen_attr` - `gen_attr`)) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr` + `gen_attr`), `gen_attr` GROUPING SETS(((`gen_attr` + `gen_attr`), `gen_attr`), ((`gen_attr` + `gen_attr`)), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `(a + b)`, `gen_attr_1` AS `b`, `gen_attr_4` AS `sum((a - b))` FROM (SELECT (`gen_attr_5` + `gen_attr_6`) AS `gen_attr_3`, `gen_attr_6` AS `gen_attr_1`, sum((`gen_attr_5` - `gen_attr_6`)) AS `gen_attr_4` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_7`, `d` AS `gen_attr_8` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY (`gen_attr_5` + `gen_attr_6`), `gen_attr_6` GROUPING SETS(((`gen_attr_5` + `gen_attr_6`), `gen_attr_6`), ((`gen_attr_5` + `gen_attr_6`)), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql index 6f57130f3e..aae2d75d79 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping_id(a, b) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping_id(a, b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping_id() AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping_id(a, b)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping_id() AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql index dab7852a3e..9958c8f38b 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping(b) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(b)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping(b)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping(`gen_attr_5`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql index 50964e5b01..fd012043cf 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_7_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, grouping(a) FROM parquet_t2 GROUP BY cube(a, b) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `a`, `gen_attr` AS `b`, `gen_attr` AS `grouping(a)` FROM (SELECT `gen_attr` AS `gen_attr`, `gen_attr` AS `gen_attr`, grouping(`gen_attr`) AS `gen_attr` FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` GROUPING SETS((`gen_attr`, `gen_attr`), (`gen_attr`), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `grouping(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, grouping(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql index 84af580a36..61c27067e1 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_1.sql @@ -3,4 +3,4 @@ SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `k1`, `gen_attr_4` AS `k2`, `gen_attr_5` AS `hgid` FROM (SELECT `gen_attr_6` AS `gen_attr_3`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_4`, hash(grouping_id()) AS `gen_attr_5` FROM (SELECT hash(`gen_attr_10`) AS `gen_attr_6`, `gen_attr_10` AS `gen_attr_7` FROM (SELECT `key` AS `gen_attr_10`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT))), (`gen_attr_6`), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql index b0f65b7d41..16f254fa41 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_8_2.sql @@ -3,4 +3,4 @@ SELECT hkey AS k1, value - 5 AS k2, hash(grouping_id()) AS hgid FROM (SELECT hash(key) as hkey, key as value FROM parquet_t1) t GROUP BY hkey, value-5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `k1`, `gen_attr` AS `k2`, `gen_attr` AS `hgid` FROM (SELECT `gen_attr` AS `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, hash(grouping_id()) AS `gen_attr` FROM (SELECT hash(`gen_attr`) AS `gen_attr`, `gen_attr` AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 +SELECT `gen_attr_3` AS `k1`, `gen_attr_4` AS `k2`, `gen_attr_5` AS `hgid` FROM (SELECT `gen_attr_6` AS `gen_attr_3`, (`gen_attr_7` - CAST(5 AS BIGINT)) AS `gen_attr_4`, hash(grouping_id()) AS `gen_attr_5` FROM (SELECT hash(`gen_attr_10`) AS `gen_attr_6`, `gen_attr_10` AS `gen_attr_7` FROM (SELECT `key` AS `gen_attr_10`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t GROUP BY `gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_6`, (`gen_attr_7` - CAST(5 AS BIGINT))), (`gen_attr_6`), ((`gen_attr_7` - CAST(5 AS BIGINT))), ())) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql index 171400dd78..cfce175843 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_1.sql @@ -5,4 +5,4 @@ FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t GROUP BY cnt, t.key - 5 WITH ROLLUP -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ())) AS gen_subquery_2 +SELECT `gen_attr_3` AS `(key - CAST(5 AS BIGINT))`, `gen_attr_0` AS `cnt`, `gen_attr_4` AS `sum(cnt)` FROM (SELECT (`gen_attr_6` - CAST(5 AS BIGINT)) AS `gen_attr_3`, `gen_attr_5` AS `gen_attr_0`, sum(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `gen_attr_6`, count(1) AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_10` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_6` = `gen_attr_9`) GROUP BY `gen_attr_6`) AS t GROUP BY `gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT))), (`gen_attr_5`), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql index fe98b6c343..d950674b74 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_9_2.sql @@ -5,4 +5,4 @@ FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key GROUP BY x.key) t GROUP BY cnt, t.key - 5 WITH CUBE -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(key - CAST(5 AS BIGINT))`, `gen_attr` AS `cnt`, `gen_attr` AS `sum(cnt)` FROM (SELECT (`gen_attr` - CAST(5 AS BIGINT)) AS `gen_attr`, `gen_attr` AS `gen_attr`, sum(`gen_attr`) AS `gen_attr` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS t GROUP BY `gen_attr`, (`gen_attr` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr`, (`gen_attr` - CAST(5 AS BIGINT))), (`gen_attr`), ((`gen_attr` - CAST(5 AS BIGINT))), ())) AS gen_subquery_2 +SELECT `gen_attr_3` AS `(key - CAST(5 AS BIGINT))`, `gen_attr_0` AS `cnt`, `gen_attr_4` AS `sum(cnt)` FROM (SELECT (`gen_attr_6` - CAST(5 AS BIGINT)) AS `gen_attr_3`, `gen_attr_5` AS `gen_attr_0`, sum(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `gen_attr_6`, count(1) AS `gen_attr_5` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_10` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_6` = `gen_attr_9`) GROUP BY `gen_attr_6`) AS t GROUP BY `gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT)) GROUPING SETS((`gen_attr_5`, (`gen_attr_6` - CAST(5 AS BIGINT))), (`gen_attr_5`), ((`gen_attr_6` - CAST(5 AS BIGINT))), ())) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql index 75e43d0dc1..1736d74b0c 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (a, b, c, d) USING 'cat' FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `key`, `gen_attr_5` AS `value` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string) FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql index 6a68edc72e..07f59d6bff 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (*) USING 'cat' FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) USING 'cat' AS (`gen_attr` string, `gen_attr` string) FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `key`, `gen_attr_5` AS `value` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string) FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql index 111df4bdf2..fc0cabec23 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT TRANSFORM (a, b, c, d) USING 'cat' AS (d1, d2, d3, d4) FROM parquet_t2 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `d1`, `gen_attr` AS `d2`, `gen_attr` AS `d3`, `gen_attr` AS `d4` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`, `gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` string, `gen_attr` string, `gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `a` AS `gen_attr`, `b` AS `gen_attr`, `c` AS `gen_attr`, `d` AS `gen_attr` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_4` AS `d1`, `gen_attr_5` AS `d2`, `gen_attr_6` AS `d3`, `gen_attr_7` AS `d4` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr_4` string, `gen_attr_5` string, `gen_attr_6` string, `gen_attr_7` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `a` AS `gen_attr_0`, `b` AS `gen_attr_1`, `c` AS `gen_attr_2`, `d` AS `gen_attr_3` FROM `default`.`parquet_t2`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql index 4ab412cfd1..a45f9a2c62 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_alias_list_with_type.sql @@ -3,4 +3,4 @@ FROM (FROM parquet_t1 SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t SELECT thing1 + 1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `(thing1 + 1)` FROM (SELECT (`gen_attr` + 1) AS `gen_attr` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr` int, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t) AS gen_subquery_1 +SELECT `gen_attr_0` AS `(thing1 + 1)` FROM (SELECT (`gen_attr_1` + 1) AS `gen_attr_0` FROM (SELECT TRANSFORM (`gen_attr_2`, `gen_attr_3`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`gen_attr_1` int, `gen_attr_4` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS t) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql index 1e30e45d45..30d37c78b5 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_multiple.sql @@ -5,4 +5,4 @@ USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_1` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr_0`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`gen_attr_1` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql index dd62289878..0b694e0d6d 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_one.sql @@ -3,4 +3,4 @@ SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`gen_attr` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_1` AS `tKey` FROM (SELECT TRANSFORM (`gen_attr_0`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`gen_attr_1` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql index 2ad3698157..14cff37385 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_serde.sql @@ -7,4 +7,4 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_2` AS `tKey`, `gen_attr_3` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`gen_attr_2` string, `gen_attr_3` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql index a90b42d9b4..d20caf7afc 100644 --- a/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql +++ b/sql/hive/src/test/resources/sqlgen/script_transformation_row_format_without_serde.sql @@ -5,4 +5,4 @@ USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `tKey`, `gen_attr` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr`, `gen_attr`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`gen_attr` string, `gen_attr` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 +SELECT `gen_attr_2` AS `tKey`, `gen_attr_3` AS `tValue` FROM (SELECT TRANSFORM (`gen_attr_0`, `gen_attr_1`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`gen_attr_2` string, `gen_attr_3` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/select_distinct.sql b/sql/hive/src/test/resources/sqlgen/select_distinct.sql index 3bc8e55379..09d93cac8e 100644 --- a/sql/hive/src/test/resources/sqlgen/select_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/select_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT DISTINCT id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT DISTINCT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT DISTINCT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/select_orc_table.sql b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql index eae67f9540..18ff021798 100644 --- a/sql/hive/src/test/resources/sqlgen/select_orc_table.sql +++ b/sql/hive/src/test/resources/sqlgen/select_orc_table.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from orc_t -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`orc_t`) AS gen_subquery_0) AS orc_t +SELECT `gen_attr_0` AS `c1`, `gen_attr_1` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `c1` AS `gen_attr_0`, `c2` AS `gen_attr_1` FROM `default`.`orc_t`) AS gen_subquery_0) AS orc_t diff --git a/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql index a085bab84c..d2eac9c08f 100644 --- a/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql +++ b/sql/hive/src/test/resources/sqlgen/select_parquet_table.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from parquet_t -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `c1`, `gen_attr` AS `c2` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `c1` AS `gen_attr`, `c2` AS `gen_attr` FROM `default`.`parquet_t`) AS gen_subquery_0) AS parquet_t +SELECT `gen_attr_0` AS `c1`, `gen_attr_1` AS `c2` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `c1` AS `gen_attr_0`, `c2` AS `gen_attr_1` FROM `default`.`parquet_t`) AS gen_subquery_0) AS parquet_t diff --git a/sql/hive/src/test/resources/sqlgen/self_join.sql b/sql/hive/src/test/resources/sqlgen/self_join.sql index 8947ccda82..d6dcee2f67 100644 --- a/sql/hive/src/test/resources/sqlgen/self_join.sql +++ b/sql/hive/src/test/resources/sqlgen/self_join.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT x.key FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key` FROM (SELECT `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS x +SELECT `gen_attr_0` AS `key` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_2` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_1`)) AS x diff --git a/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql index 6db053f286..1dedb44dbf 100644 --- a/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql +++ b/sql/hive/src/test/resources/sqlgen/self_join_with_group_by.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT x.key, COUNT(*) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key group by x.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `count(1)` FROM (SELECT `gen_attr`, count(1) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`) GROUP BY `gen_attr`) AS x +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`) GROUP BY `gen_attr_0`) AS x diff --git a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql index e9a6afdc7e..da60204297 100644 --- a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql +++ b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `count(value)` FROM (SELECT `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT count(`gen_attr`) AS `gen_attr`, max(`gen_attr`) AS `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr` HAVING (`gen_attr` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr` ASC) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_1`) AS `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING (`gen_attr_2` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr_1` ASC) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql index 54c8deeb65..291f2f59d7 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM parquet_t0 TABLESAMPLE(100 PERCENT) s -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql index 13dcadb41a..6a92d7aef7 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM parquet_t0 TABLESAMPLE(100 PERCENT) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_3.sql b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql index c1ed57f5e3..4a17d7105e 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_3.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM t0 TABLESAMPLE(100 PERCENT) s -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql index c22b20802c..873de051a6 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 TABLESAMPLE(100 PERCENT) -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT)) AS gen_subquery_0) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql index fa69dd46c7..f958b2f111 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT s.id FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE 1=0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS s +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql index bc72560c42..688a102d1d 100644 --- a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql +++ b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) WHERE 1=0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0` TABLESAMPLE(0.1 PERCENT)) AS gen_subquery_0 WHERE (1 = 0)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/three_child_union.sql b/sql/hive/src/test/resources/sqlgen/three_child_union.sql index c3781117d8..713c7502f5 100644 --- a/sql/hive/src/test/resources/sqlgen/three_child_union.sql +++ b/sql/hive/src/test/resources/sqlgen/three_child_union.sql @@ -3,4 +3,4 @@ SELECT id FROM parquet_t0 UNION ALL SELECT id FROM parquet_t0 UNION ALL SELECT id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1) UNION ALL (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_2)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_1) UNION ALL (SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_2)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/type_widening.sql b/sql/hive/src/test/resources/sqlgen/type_widening.sql index 96e6cc2fec..ebb8a92afd 100644 --- a/sql/hive/src/test/resources/sqlgen/type_widening.sql +++ b/sql/hive/src/test/resources/sqlgen/type_widening.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT CAST(CAST(`gen_attr` AS INT) AS BIGINT) AS `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`parquet_t0`) AS gen_subquery_1)) AS parquet_t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_0) UNION ALL (SELECT CAST(CAST(`gen_attr_0` AS INT) AS BIGINT) AS `gen_attr_1` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`parquet_t0`) AS gen_subquery_1)) AS parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/union_distinct.sql b/sql/hive/src/test/resources/sqlgen/union_distinct.sql index c5895ae766..46644b89eb 100644 --- a/sql/hive/src/test/resources/sqlgen/union_distinct.sql +++ b/sql/hive/src/test/resources/sqlgen/union_distinct.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT * FROM t0 UNION SELECT * FROM t0 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `id` FROM ((SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_0) UNION DISTINCT (SELECT `gen_attr` FROM (SELECT `id` AS `gen_attr` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 +SELECT `gen_attr_0` AS `id` FROM ((SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`t0`) AS gen_subquery_0) UNION DISTINCT (SELECT `gen_attr_1` FROM (SELECT `id` AS `gen_attr_1` FROM `default`.`t0`) AS gen_subquery_1)) AS t0 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql index f86ca5e086..000c4e735a 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_1.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT MAX(value) OVER (PARTITION BY key % 3) FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` % CAST(3 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT `gen_attr_0` FROM (SELECT gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_2`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_2` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_0` FROM (SELECT `gen_attr_1`, (`gen_attr_3` % CAST(3 AS BIGINT)) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql index 15559ac8e1..ec55d4b714 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql @@ -2,4 +2,4 @@ SELECT key, value, ROUND(AVG(key) OVER (), 2) FROM parquet_t1 ORDER BY key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr`, `gen_attr`, round(`gen_attr`, 2) AS `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, avg(`gen_attr`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr` ASC) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` ASC) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql index 43f992882c..c0ac9541e6 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql @@ -2,4 +2,4 @@ SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, (`gen_attr` + CAST(1 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr`, (`gen_attr` % CAST(7 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `value`, `gen_attr_1` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, gen_subquery_1.`gen_attr_4`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_4` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, (`gen_attr_5` + CAST(1 AS BIGINT)) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, (`gen_attr_5` % CAST(7 AS BIGINT)) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_0` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_join.sql b/sql/hive/src/test/resources/sqlgen/window_with_join.sql index 00c45c8fea..030a4c0907 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_join.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_join.sql @@ -2,4 +2,4 @@ SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, gen_subquery_2.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr` = `gen_attr`)) AS gen_subquery_2) AS gen_subquery_3) AS x +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_2.`gen_attr_0`, gen_subquery_2.`gen_attr_2`, gen_subquery_2.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_2`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`)) AS gen_subquery_2) AS gen_subquery_3) AS x diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql index 65d3d3aeb6..7b99539a05 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `count(key)` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, count(`gen_attr`) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `count(key)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, count(`gen_attr_0`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql index 03e0962f44..591a654a38 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `dr`, `gen_attr` AS `ca` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, DENSE_RANK() OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr`, count(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC, `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `ca` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2`, count(`gen_attr_0`) OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql index 090207f9b8..d9169eab6e 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql index fcc2cf7de8..f0a820811e 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max FROM parquet_t1 GROUP BY key, value HAVING key > 5 -------------------------------------------------------------------------------- -SELECT `gen_attr` AS `key`, `gen_attr` AS `value`, `gen_attr` AS `max` FROM (SELECT `gen_attr`, `gen_attr`, `gen_attr` FROM (SELECT gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, gen_subquery_1.`gen_attr`, max(`gen_attr`) OVER (PARTITION BY `gen_attr` ORDER BY `gen_attr` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr` FROM (SELECT `gen_attr`, `gen_attr`, (`gen_attr` % CAST(5 AS BIGINT)) AS `gen_attr` FROM (SELECT `key` AS `gen_attr`, `value` AS `gen_attr` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr`, `gen_attr` HAVING (`gen_attr` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1` HAVING (`gen_attr_0` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index ebece38773..aaff272bba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -97,17 +97,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { /** * Compare the generated SQL with the expected answer string. - * Note that there exists a normalization for both arguments for the convenience. - * - Remove the id from the generated attributes, e.g., `gen_attr_1` -> `gen_attr`. */ private def checkSQLStructure(originalSQL: String, convertedSQL: String, answerFile: String) = { - val normalizedGenSQL = convertedSQL.replaceAll("`gen_attr_\\d+`", "`gen_attr`") if (answerFile != null) { val separator = "-" * 80 if (regenerateGoldenFiles) { val path = Paths.get(s"$goldenSQLPath/$answerFile.sql") val header = "-- This file is automatically generated by LogicalPlanToSQLSuite." - val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$normalizedGenSQL\n" + val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$convertedSQL\n" Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" @@ -119,8 +116,8 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") - val normalizedExpectSQL = sqls(1).trim() - assert(normalizedGenSQL == normalizedExpectSQL) + val expectedSQL = sqls(1).trim() + assert(convertedSQL == expectedSQL) } } } -- GitLab