From 3a3f8acb5d27bce8d9cdce4538e62745b3bc8757 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 1 Aug 2016 00:00:11 -0700 Subject: [PATCH] [SPARK-16756][SQL] Add `sql` function to LogicalPlan and `NonSQLPlan` trait --- .../sql/catalyst/analysis/Analyzer.scala | 5 +- .../sql/catalyst/analysis/unresolved.scala | 8 +- .../sql/catalyst/catalog/interface.scala | 2 + .../sql/catalyst/expressions/subquery.scala | 33 ++- .../plans/logical/LocalRelation.scala | 10 + .../catalyst/plans/logical/LogicalPlan.scala | 14 ++ .../plans/logical/ScriptTransformation.scala | 17 ++ .../plans/logical/basicLogicalOperators.scala | 233 +++++++++++++++++- .../sql/catalyst/plans/logical/object.scala | 22 +- .../catalyst/plans/logical/partitioning.scala | 6 +- .../analysis/AnalysisErrorSuite.scala | 2 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../analysis/UnsupportedOperationsSuite.scala | 6 +- .../sql/catalyst/plans/LogicalPlanSuite.scala | 3 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 3 +- .../spark/sql/catalyst/SQLBuilder.scala | 2 +- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 2 +- .../sql/execution/command/commands.scala | 5 +- .../spark/sql/execution/command/tables.scala | 1 + .../datasources/LogicalRelation.scala | 4 +- .../spark/sql/execution/datasources/ddl.scala | 5 +- .../streaming/StreamingRelation.scala | 7 +- .../sql/execution/streaming/memory.scala | 4 +- .../sql/execution/SparkPlannerSuite.scala | 4 +- .../spark/sql/hive/MetastoreRelation.scala | 4 + 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 +- .../sqlgen/broadcast_join_subquery.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 +- .../test/resources/sqlgen/inline_tables.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 +- sql/hive/src/test/resources/sqlgen/range.sql | 2 +- .../resources/sqlgen/range_with_splits.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 +- sql/hive/src/test/resources/sqlgen/subq2.sql | 2 +- .../resources/sqlgen/subquery_exists_1.sql | 2 +- .../resources/sqlgen/subquery_exists_2.sql | 2 +- .../sqlgen/subquery_exists_having_1.sql | 2 +- .../sqlgen/subquery_exists_having_2.sql | 2 +- .../sqlgen/subquery_exists_having_3.sql | 2 +- .../src/test/resources/sqlgen/subquery_in.sql | 2 +- .../resources/sqlgen/subquery_in_having_1.sql | 2 +- .../resources/sqlgen/subquery_in_having_2.sql | 2 +- .../sqlgen/subquery_not_exists_1.sql | 2 +- .../sqlgen/subquery_not_exists_2.sql | 2 +- .../sqlgen/subquery_not_exists_having_1.sql | 2 +- .../sqlgen/subquery_not_exists_having_2.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/ExpressionToSQLSuite.scala | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 13 +- .../hive/execution/HiveComparisonTest.scala | 3 +- 149 files changed, 485 insertions(+), 181 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e559f235c5a38..cb8aff78f4b5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1252,8 +1252,9 @@ class Analyzer( // Try resolving the ordering as though it is in the aggregate clause. try { val unresolvedSortOrders = sortOrder.filter(s => !s.resolved || containsAggregate(s)) - val aliasedOrdering = - unresolvedSortOrders.map(o => Alias(o.child, "aggOrder")(isGenerated = true)) + val aliasedOrdering = unresolvedSortOrders.zipWithIndex.map { + case (o, i) => Alias(o.child, s"aggOrder$i")(isGenerated = true) + } val aggregatedOrdering = aggregate.copy(aggregateExpressions = aliasedOrdering) val resolvedAggregate: Aggregate = execute(aggregatedOrdering).asInstanceOf[Aggregate] val resolvedAliasedOrdering: Seq[Alias] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 235ae04782455..cc708e71d9755 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, NonSQLPlan} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.types.{DataType, StructType} @@ -47,6 +47,8 @@ case class UnresolvedRelation( override def output: Seq[Attribute] = Nil override lazy val resolved = false + + override def sql: String = tableName } /** @@ -59,7 +61,7 @@ case class UnresolvedRelation( case class UnresolvedInlineTable( names: Seq[String], rows: Seq[Seq[Expression]]) - extends LeafNode { + extends LeafNode with NonSQLPlan { lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved)) override lazy val resolved = false @@ -73,7 +75,7 @@ case class UnresolvedInlineTable( * }}} */ case class UnresolvedTableValuedFunction(functionName: String, functionArgs: Seq[Expression]) - extends LeafNode { + extends LeafNode with NonSQLPlan { override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 83e01f95c06af..1527fd2ade5fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -260,4 +260,6 @@ case class SimpleCatalogRelation( require( metadata.identifier.database == Some(databaseName), "provided database does not match the one specified in the table definition") + + override def sql: String = catalogTable.identifier.quotedString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index e2e7d98e33459..84b1388a49d77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias} import org.apache.spark.sql.types._ /** @@ -74,6 +74,7 @@ case class ScalarSubquery( override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" + override def sql: String = s"(${plan.sql})" } object ScalarSubquery { @@ -108,6 +109,36 @@ case class PredicateSubquery( case _ => false } override def toString: String = s"predicate-subquery#${exprId.id} $conditionString" + override def sql: String = { + if (nullAware) { + val (in, correlated) = children.partition(_.isInstanceOf[EqualTo]) + val (outer, inner) = in.zipWithIndex.map { + case (EqualTo(l, r), i) if plan.outputSet.intersect(r.references).nonEmpty => + (l, Alias(r, s"_c$i")()) + case (EqualTo(r, l), i) => + (l, Alias(r, s"_c$i")()) + }.unzip + val filtered = if (correlated.nonEmpty) { + Filter(children.reduce(And), plan) + } else { + plan + } + val value = outer match { + case Seq(expr) => expr + case exprs => CreateStruct(exprs) + } + children.head.map(_.sql).mkString(" AND ") + s"${value.sql} IN (${filtered.sql})" + } else { + val conditionSQL = children.map(_.sql).mkString(" AND ") + val subquery = if (conditionSQL.isEmpty) { + plan.sql + } else { + s"${plan.sql} ${if (plan.sql.contains("WHERE")) "AND" else "WHERE"} $conditionSQL" + } + s"EXISTS ($subquery)" + } + } } object PredicateSubquery { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index 890865d177845..775e98b3532a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -88,4 +88,14 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil) " AS " + inlineTableName + output.map(_.name).mkString("(", ", ", ")") } + + override def sql: String = { + require(data.nonEmpty) + val types = output.map(_.dataType) + val rows = data.map { row => + val cells = row.toSeq(types).zip(types).map { case (v, tpe) => Literal(v, tpe).sql } + cells.mkString("(", ", ", ")") + } + "VALUES " + rows.mkString(", ") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 6d7799151d93b..44f7168f177d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -270,6 +270,12 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * Refreshes (or invalidates) any metadata/data cached in the plan recursively. */ def refresh(): Unit = children.foreach(_.refresh()) + + /** + * Returns SQL representation of this plan. For the plans extending [[NonSQLPlan]], + * this method may return an arbitrary user facing string. + */ + def sql: String } /** @@ -332,3 +338,11 @@ abstract class BinaryNode extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(left, right) } + +/** + * LogicalPlan that don't have SQL representation should extend this trait. + * For example, `ReturnAnswer` and `Range`. + */ +trait NonSQLPlan extends LogicalPlan { + final override def sql: String = throw new UnsupportedOperationException(this.getClass.getName()) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index e176e9b82bf33..3e2754d14460e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -34,6 +34,23 @@ case class ScriptTransformation( child: LogicalPlan, ioschema: ScriptInputOutputSchema) extends UnaryNode { override def references: AttributeSet = AttributeSet(input.flatMap(_.references)) + + override def sql: String = { + val inputRowFormatSQL = ioschema.inputRowFormatSQL.getOrElse( + throw new UnsupportedOperationException( + s"unsupported row format ${ioschema.inputRowFormat}")) + val outputRowFormatSQL = ioschema.outputRowFormatSQL.getOrElse( + throw new UnsupportedOperationException( + s"unsupported row format ${ioschema.outputRowFormat}")) + + val outputSchema = output.map { attr => + s"${attr.sql} ${attr.dataType.simpleString}" + }.mkString(", ") + + s"SELECT TRANSFORM (${input.map(_.sql).mkString(", ")}) $inputRowFormatSQL " + + s"USING \'${script}\' AS ($outputSchema) $outputRowFormatSQL " + + (if (child == OneRowRelation) "" else s"FROM ${child.sql}") + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 010aec7ba1a42..9c4d5bbd963ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -33,7 +33,7 @@ import org.apache.spark.sql.types._ * Rules can pattern-match on this node in order to apply transformations that only take effect * at the top of the logical query plan. */ -case class ReturnAnswer(child: LogicalPlan) extends UnaryNode { +case class ReturnAnswer(child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output } @@ -54,6 +54,40 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend override def validConstraints: Set[Expression] = child.constraints.union(getAliasedConstraints(projectList)) + + override def sql: String = { + if (projectList.exists(_.find(_.isInstanceOf[NonSQLExpression]).isDefined)) { + throw new UnsupportedOperationException("NonSQLExpression") + } + child match { + case OneRowRelation => + s"SELECT ${projectList.map(_.sql).mkString(", ")}" + + case g @ Generate(_, _, _, _, _, grandChild) => grandChild match { + case OneRowRelation => + s"SELECT ${g.generator.sql} AS ${g.generatorOutput.map(_.sql).mkString(", ")}" + + case f @ Filter(condition, grandGrandChild) => + s"SELECT ${projectList.map(_.sql).mkString(", ")} " + + s"FROM ${grandGrandChild.sql} " + + s"LATERAL VIEW ${if (g.outer) "OUTER " else ""}${g.generator.sql} " + + s"${g.qualifier.map(_ + " ").getOrElse("")}" + + s"AS ${g.generatorOutput.map(_.sql).mkString(", ")} " + + s"WHERE ${condition.sql}" + + case _ => + s"SELECT ${projectList.map(_.sql).mkString(", ")} " + + s"FROM ${grandChild.sql} " + + s"LATERAL VIEW ${if (g.outer) "OUTER " else ""}${g.generator.sql} " + + s"${g.qualifier.map(_ + " ").getOrElse("")}" + + s"AS ${g.generatorOutput.map(_.sql).mkString(", ")}" + } + case _ => + val subquery = child.sql + val childSQL = s" FROM ${if (subquery.startsWith("SELECT")) s"($subquery)" else subquery}" + s"SELECT ${projectList.map(_.sql).mkString(", ")}${childSQL}" + } + } } /** @@ -100,6 +134,13 @@ case class Generate( if (join) child.output ++ qualified else qualified } + + override def sql: String = { + val columnAliases = generatorOutput.map(_.sql).mkString(", ") + val outerStr = if (outer) "OUTER " else "" + s"${child.sql} LATERAL VIEW $outerStr" + + s"${generator.sql} ${qualifier.map(_ + " ").getOrElse("")}AS $columnAliases" + } } case class Filter(condition: Expression, child: LogicalPlan) @@ -113,6 +154,11 @@ case class Filter(condition: Expression, child: LogicalPlan) .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(predicates.toSet) } + + override def sql: String = child match { + case _: Aggregate => s"${child.sql} HAVING ${condition.sql}" + case _ => s"${child.sql} WHERE ${condition.sql}" + } } abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -168,6 +214,8 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation Statistics(sizeInBytes = sizeInBytes, isBroadcastable = isBroadcastable) } + + override def sql: String = s"(${left.sql}) INTERSECT (${right.sql})" } case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { @@ -188,6 +236,8 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le override lazy val statistics: Statistics = { left.statistics.copy() } + + override def sql: String = s"(${left.sql}) EXCEPT (${right.sql})" } /** Factory for constructing new `Union` nodes. */ @@ -265,6 +315,15 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) .reduce(merge(_, _)) } + + override def sql: String = { + if (children.length > 1) { + val childrenSql = children.map(c => s"(${c.sql})") + childrenSql.mkString(" UNION ALL ") + } else { + children.head.sql + } + } } case class Join( @@ -343,6 +402,10 @@ case class Join( // they could explode the size. super.statistics.copy(isBroadcastable = false) } + + override def sql: String = { + s"${left.sql} ${joinType.sql} JOIN ${right.sql}${condition.map(" ON " + _.sql).getOrElse("")}" + } } /** @@ -353,6 +416,8 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { // set isBroadcastable to true so the child will be broadcasted override lazy val statistics: Statistics = super.statistics.copy(isBroadcastable = true) + + override def sql: String = s"/*+ BROADCAST */ ${child.sql}" } case class InsertIntoTable( @@ -361,7 +426,7 @@ case class InsertIntoTable( child: LogicalPlan, overwrite: Boolean, ifNotExists: Boolean) - extends LogicalPlan { + extends LogicalPlan with NonSQLPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty @@ -396,13 +461,14 @@ case class InsertIntoTable( * @param cteRelations A sequence of pair (alias, the CTE definition) that this CTE defined * Each CTE can see the base tables and the previously defined CTEs only. */ -case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { +case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) + extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output } case class WithWindowDefinition( windowDefinitions: Map[String, WindowSpecDefinition], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output } @@ -418,6 +484,19 @@ case class Sort( child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output override def maxRows: Option[Long] = child.maxRows + override def sql: String = { + if (order.isEmpty) { + child.sql + } else { + child match { + case r @ RepartitionByExpression(partitionExprs, grandChild, _) + if order.map(_.child) == partitionExprs => + s"${grandChild.sql} CLUSTER BY ${partitionExprs.map(_.sql).mkString(", ")}" + case _ => + s"${child.sql} ${if (global) "ORDER" else "SORT"} BY ${order.map(_.sql).mkString(", ")}" + } + } + } } /** Factory for constructing new `Range` nodes. */ @@ -460,6 +539,14 @@ case class Range( } } + override def sql: String = { + if (numSlices.isDefined) { + s"range($start, $end, $step, ${numSlices.get})" + } else { + s"range($start, $end, $step)" + } + } + override def newInstance(): Range = copy(output = output.map(_.newInstance())) override lazy val statistics: Statistics = { @@ -502,6 +589,93 @@ case class Aggregate( super.statistics } } + + private def sameOutput(output1: Seq[Attribute], output2: Seq[Attribute]): Boolean = { + output1.size == output2.size && + output1.zip(output2).forall{ case (left, right) => left.semanticEquals(right) } + } + + private def isGroupingSet(e: Expand, p: Project) = { + groupingExpressions.forall(_.isInstanceOf[Attribute]) && sameOutput( + e.output.drop(p.child.output.length), + groupingExpressions.map(_.asInstanceOf[Attribute])) + } + + override def sql: String = child match { + case OneRowRelation => + s"SELECT ${aggregateExpressions.map(_.sql).mkString(", ")}" + + case e @ Expand(_, _, p @ Project(_, _)) if isGroupingSet(e, p) => + // The last column of Expand is always grouping ID + val gid = e.output.last + + val numOriginalOutput = p.child.output.length + // Assumption: Aggregate's groupingExpressions is composed of + // 1) the grouping attributes + // 2) gid, which is always the last one + val groupByAttributes = groupingExpressions.dropRight(1).map(_.asInstanceOf[Attribute]) + // Assumption: Project's projectList is composed of + // 1) the original output (Project's child.output), + // 2) the aliased group by expressions. + val expandedAttributes = p.output.drop(numOriginalOutput) + val groupByExprs = p.projectList.drop(numOriginalOutput).map(_.asInstanceOf[Alias].child) + val groupingSQL = groupByExprs.map(_.sql).mkString(", ") + + // a map from group by attributes to the original group by expressions. + val groupByAttrMap = AttributeMap(groupByAttributes.zip(groupByExprs)) + // a map from expanded attributes to the original group by expressions. + val expandedAttrMap = AttributeMap(expandedAttributes.zip(groupByExprs)) + + val groupingSet: Seq[Seq[Expression]] = e.projections.map { project => + // Assumption: expand.projections is composed of + // 1) the original output (Project's child.output), + // 2) expanded attributes(or null literal) + // 3) gid, which is always the last one in each project in Expand + project.drop(numOriginalOutput).dropRight(1).collect { + case attr: Attribute if expandedAttrMap.contains(attr) => expandedAttrMap(attr) + } + } + val groupingSetSQL = "GROUPING SETS(" + + groupingSet.map(e => s"(${e.map(_.sql).mkString(", ")})").mkString(", ") + ")" + + val aggExprs = aggregateExpressions.map { case aggExpr => + val originalAggExpr = aggExpr.transformDown { + // grouping_id() is converted to VirtualColumn.groupingIdName by Analyzer. Revert it back. + case ar: AttributeReference if ar == gid => GroupingID(Nil) + case ar: AttributeReference if groupByAttrMap.contains(ar) => groupByAttrMap(ar) + case a @ Cast(BitwiseAnd( + ShiftRight(ar: AttributeReference, Literal(value: Any, IntegerType)), + Literal(1, IntegerType)), ByteType) if ar == gid => + // for converting an expression to its original SQL format grouping(col) + val idx = groupByExprs.length - 1 - value.asInstanceOf[Int] + groupByExprs.lift(idx).map(Grouping).getOrElse(a) + } + + originalAggExpr match { + // Ancestor operators may reference the output of this grouping set, and we use exprId to + // generate a unique name for each attribute, so we should make sure the transformed + // aggregate expression won't change the output, i.e. exprId and alias name should remain + // the same. + case ne: NamedExpression if ne.exprId == aggExpr.exprId => ne + case e => Alias(e, aggExpr.name)(exprId = aggExpr.exprId) + } + } + + s"SELECT ${aggExprs.map(_.sql).mkString(", ")} " + + s"FROM ${if (p.sql.startsWith("SELECT")) s"(${p.sql})" else p.sql} " + + s"GROUP BY ${groupingSQL} " + + groupingSetSQL + + case _ => + val groupingSQL = if (groupingExpressions.isEmpty) { + "" + } else { + " GROUP BY " + groupingExpressions.map(_.sql).mkString(", ") + } + s"SELECT ${aggregateExpressions.map(_.sql).distinct.mkString(", ")} " + + s"FROM ${if (child.sql.startsWith("SELECT")) s"(${child.sql})" else child.sql}" + + groupingSQL + } } case class Window( @@ -514,6 +688,10 @@ case class Window( child.output ++ windowExpressions.map(_.toAttribute) def windowOutputSet: AttributeSet = AttributeSet(windowExpressions.map(_.toAttribute)) + + override def sql: String = { + s"SELECT ${(child.output ++ windowExpressions).map(_.sql).mkString(", ")} FROM (${child.sql})" + } } object Expand { @@ -591,7 +769,7 @@ object Expand { case class Expand( projections: Seq[Seq[Expression]], output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) @@ -623,7 +801,7 @@ case class GroupingSets( bitmasks: Seq[Int], groupByExprs: Seq[Expression], child: LogicalPlan, - aggregations: Seq[NamedExpression]) extends UnaryNode { + aggregations: Seq[NamedExpression]) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = aggregations.map(_.toAttribute) @@ -637,7 +815,7 @@ case class Pivot( pivotColumn: Expression, pivotValues: Seq[Literal], aggregates: Seq[Expression], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = groupByExprs.map(_.toAttribute) ++ aggregates match { case agg :: Nil => pivotValues.map(value => AttributeReference(value.toString, agg.dataType)()) case _ => pivotValues.flatMap{ value => @@ -678,6 +856,10 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN } child.statistics.copy(sizeInBytes = sizeInBytes) } + + override def sql: String = { + s"${child.sql} LIMIT ${limitExpr.sql}" + } } case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { @@ -699,6 +881,8 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo } child.statistics.copy(sizeInBytes = sizeInBytes) } + + override def sql: String = child.sql } case class SubqueryAlias( @@ -708,6 +892,12 @@ case class SubqueryAlias( extends UnaryNode { override def output: Seq[Attribute] = child.output.map(_.withQualifier(Some(alias))) + + override def sql: String = child match { + case LocalRelation(output, _) => + s"${child.sql} AS $alias${output.map(_.name).mkString("(", ", ", ")")}" + case _ => if (child.sql.equals(alias)) child.sql else s"(${child.sql}) AS $alias" + } } /** @@ -742,6 +932,20 @@ case class Sample( } override protected def otherCopyArgs: Seq[AnyRef] = isTableSample :: Nil + + override def sql: String = { + val repeatable = if (withReplacement) s" REPEATABLE ($seed)" else "" + child match { + case SubqueryAlias(alias, _: NonSQLPlan, _) => + s"$alias TABLESAMPLE(${upperBound * 100} PERCENT)$repeatable" + + case SubqueryAlias(alias, grandChild, _) => + s"${grandChild.sql} TABLESAMPLE(${upperBound * 100} PERCENT)$repeatable $alias" + + case _ => + s"${child.sql} TABLESAMPLE(${upperBound * 100} PERCENT)$repeatable" + } + } } /** @@ -750,6 +954,15 @@ case class Sample( case class Distinct(child: LogicalPlan) extends UnaryNode { override def maxRows: Option[Long] = child.maxRows override def output: Seq[Attribute] = child.output + + override def sql: String = child match { + case Union(children) => + val childrenSql = children.map(c => s"(${c.sql})") + childrenSql.mkString(" UNION DISTINCT ") + + case p: Project => + "SELECT DISTINCT " + p.sql.substring("SELECT ".length) + } } /** @@ -759,7 +972,7 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { * of the output requires some specific ordering or distribution of the data. */ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) - extends UnaryNode { + extends UnaryNode with NonSQLPlan { require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") override def output: Seq[Attribute] = child.output } @@ -767,7 +980,7 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) /** * A relation with one row. This is used in "SELECT ..." without a from clause. */ -case object OneRowRelation extends LeafNode { +case object OneRowRelation extends LeafNode with NonSQLPlan { override def maxRows: Option[Long] = Some(1) override def output: Seq[Attribute] = Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index fefe5a3953a6e..3f76b8b86e3d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -75,7 +75,7 @@ trait ObjectConsumer extends UnaryNode { case class DeserializeToObject( deserializer: Expression, outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectProducer + child: LogicalPlan) extends UnaryNode with ObjectProducer with NonSQLPlan /** * Takes the input object from child and turns it into unsafe row using the given serializer @@ -83,7 +83,7 @@ case class DeserializeToObject( */ case class SerializeFromObject( serializer: Seq[NamedExpression], - child: LogicalPlan) extends ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer with NonSQLPlan { override def output: Seq[Attribute] = serializer.map(_.toAttribute) } @@ -107,7 +107,7 @@ object MapPartitions { case class MapPartitions( func: Iterator[Any] => Iterator[Any], outputObjAttr: Attribute, - child: LogicalPlan) extends ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer with NonSQLPlan object MapPartitionsInR { def apply( @@ -141,7 +141,7 @@ case class MapPartitionsInR( inputSchema: StructType, outputSchema: StructType, outputObjAttr: Attribute, - child: LogicalPlan) extends ObjectConsumer with ObjectProducer { + child: LogicalPlan) extends ObjectConsumer with ObjectProducer with NonSQLPlan { override lazy val schema = outputSchema override protected def stringArgs: Iterator[Any] = Iterator(inputSchema, outputSchema, @@ -171,7 +171,7 @@ case class MapElements( argumentClass: Class[_], argumentSchema: StructType, outputObjAttr: Attribute, - child: LogicalPlan) extends ObjectConsumer with ObjectProducer + child: LogicalPlan) extends ObjectConsumer with ObjectProducer with NonSQLPlan object TypedFilter { def apply[T : Encoder](func: AnyRef, child: LogicalPlan): TypedFilter = { @@ -198,7 +198,7 @@ case class TypedFilter( argumentClass: Class[_], argumentSchema: StructType, deserializer: Expression, - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output @@ -245,7 +245,7 @@ case class AppendColumns( argumentSchema: StructType, deserializer: Expression, serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output ++ newColumns @@ -259,7 +259,7 @@ case class AppendColumnsWithObject( func: Any => Any, childSerializer: Seq[NamedExpression], newColumnsSerializer: Seq[NamedExpression], - child: LogicalPlan) extends ObjectConsumer { + child: LogicalPlan) extends ObjectConsumer with NonSQLPlan { override def output: Seq[Attribute] = (childSerializer ++ newColumnsSerializer).map(_.toAttribute) } @@ -298,7 +298,7 @@ case class MapGroups( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectProducer + child: LogicalPlan) extends UnaryNode with ObjectProducer with NonSQLPlan /** Factory for constructing new `FlatMapGroupsInR` nodes. */ object FlatMapGroupsInR { @@ -340,7 +340,7 @@ case class FlatMapGroupsInR( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectProducer{ + child: LogicalPlan) extends UnaryNode with ObjectProducer with NonSQLPlan { override lazy val schema = outputSchema @@ -394,4 +394,4 @@ case class CoGroup( rightAttr: Seq[Attribute], outputObjAttr: Attribute, left: LogicalPlan, - right: LogicalPlan) extends BinaryNode with ObjectProducer + right: LogicalPlan) extends BinaryNode with ObjectProducer with NonSQLPlan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 28cbce8748fcd..ebe6d96508fda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -28,7 +28,7 @@ abstract class RedistributeData extends UnaryNode { } case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) - extends RedistributeData + extends RedistributeData with NonSQLPlan /** * This method repartitions data using [[Expression]]s into `numPartitions`, and receives @@ -46,4 +46,8 @@ case class RepartitionByExpression( case Some(n) => require(n > 0, s"Number of partitions ($n) must be positive.") case None => // Ok } + + override def sql: String = { + s"${child.sql} DISTRIBUTE BY ${partitionExpressions.map(_.sql).mkString(", ")}" + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 13bf034f831cf..f5445e47eeb5a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -86,7 +86,7 @@ case class TestFunction( override def dataType: DataType = StringType } -case class UnresolvedTestPlan() extends LeafNode { +case class UnresolvedTestPlan() extends LeafNode with NonSQLPlan { override lazy val resolved = false override def output: Seq[Attribute] = Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 8971edc7d3b9a..cda8864785287 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -129,7 +129,7 @@ class AnalysisSuite extends AnalysisTest { val b = testRelation2.output(1) val c = testRelation2.output(2) val alias_a3 = count(a).as("a3") - val alias_b = b.as("aggOrder") + val alias_b = b.as("aggOrder0") // Case 1: when the child of Sort is not Aggregate, // the sort reference is handled by the rule ResolveSortReferences diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 6df47acaba85b..4226503606669 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.IntegerType /** A dummy command for testing unsupported operations. */ -case class DummyCommand() extends LogicalPlan with Command { +case class DummyCommand() extends LogicalPlan with Command with NonSQLPlan { override def output: Seq[Attribute] = Nil override def children: Seq[LogicalPlan] = Nil } @@ -420,12 +420,12 @@ class UnsupportedOperationsSuite extends SparkFunSuite { new StreamingPlanWrapper(plan) } - case class StreamingPlanWrapper(child: LogicalPlan) extends UnaryNode { + case class StreamingPlanWrapper(child: LogicalPlan) extends UnaryNode with NonSQLPlan { override def output: Seq[Attribute] = child.output override def isStreaming: Boolean = true } - case class TestStreamingRelation(output: Seq[Attribute]) extends LeafNode { + case class TestStreamingRelation(output: Seq[Attribute]) extends LeafNode with NonSQLPlan { def this(attribute: Attribute) = this(Seq(attribute)) override def isStreaming: Boolean = true } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala index cc86f1f6e2f48..c6c310056f416 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala @@ -78,7 +78,8 @@ class LogicalPlanSuite extends SparkFunSuite { override def isStreaming(): Boolean = true } - case class TestBinaryRelation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + case class TestBinaryRelation(left: LogicalPlan, right: LogicalPlan) + extends BinaryNode with NonSQLPlan { override def output: Seq[Attribute] = left.output ++ right.output } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6a188e7e55126..56d4d9297606e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, NonSQLPlan} import org.apache.spark.sql.types.{IntegerType, NullType, StringType} case class Dummy(optKey: Option[Expression]) extends Expression with CodegenFallback { @@ -34,7 +35,7 @@ case class Dummy(optKey: Option[Expression]) extends Expression with CodegenFall } case class ComplexPlan(exprs: Seq[Seq[Expression]]) - extends org.apache.spark.sql.catalyst.plans.logical.LeafNode { + extends LeafNode with NonSQLPlan { override def output: Seq[Attribute] = Nil } 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 dde91b0a8606e..8e6c42b7c4843 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 @@ -583,7 +583,7 @@ class SQLBuilder private ( database: String, table: String, output: Seq[Attribute], - sample: Option[(Double, Double)] = None) extends LeafNode { + sample: Option[(Double, Double)] = None) extends LeafNode with NonSQLPlan { def withSample(lowerBound: Double, upperBound: Double): SQLTable = this.copy(sample = Some(lowerBound -> upperBound)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 6c4248c60e893..31cdb303819d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -79,7 +79,7 @@ object ExternalRDD { case class ExternalRDD[T]( outputObjAttr: Attribute, rdd: RDD[T])(session: SparkSession) - extends LeafNode with ObjectProducer with MultiInstanceRelation { + extends LeafNode with ObjectProducer with MultiInstanceRelation with NonSQLPlan { override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil @@ -131,7 +131,7 @@ case class ExternalRDDScanExec[T]( case class LogicalRDD( output: Seq[Attribute], rdd: RDD[InternalRow])(session: SparkSession) - extends LeafNode with MultiInstanceRelation { + extends LeafNode with MultiInstanceRelation with NonSQLPlan { override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 479934a7afc75..e0a3399f2820e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -65,7 +65,7 @@ case class InMemoryRelation( @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, val batchStats: CollectionAccumulator[InternalRow] = child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) - extends logical.LeafNode with MultiInstanceRelation { + extends logical.LeafNode with MultiInstanceRelation with logical.NonSQLPlan { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index cce1489abd301..059bdbde4c4d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -23,8 +23,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, NonSQLPlan} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.execution.streaming.IncrementalExecution @@ -35,7 +34,7 @@ import org.apache.spark.sql.types._ * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends LogicalPlan with logical.Command { +trait RunnableCommand extends LogicalPlan with Command with NonSQLPlan { override def output: Seq[Attribute] = Seq.empty override def children: Seq[LogicalPlan] = Seq.empty def run(sparkSession: SparkSession): Seq[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index b4a15b8b2882e..4a4c246db07b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogT import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, NonSQLPlan, UnaryNode} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 2a8e147011f55..41af907af9844 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, NonSQLPlan, Statistics} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.util.Utils @@ -98,4 +98,6 @@ case class LogicalRelation( } override def simpleString: String = s"Relation[${Utils.truncatedString(output, ",")}] $relation" + + override def sql: String = metastoreTableIdentifier.get.identifier } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index 1b1e2123b7c47..f85ec83a87675 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -21,12 +21,13 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, NonSQLPlan} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.types._ case class CreateTable(tableDesc: CatalogTable, mode: SaveMode, query: Option[LogicalPlan]) - extends LogicalPlan with Command { + extends LogicalPlan with Command with NonSQLPlan { assert(tableDesc.provider.isDefined, "The table to be created must have a provider.") if (query.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index e8b00094add3a..fd224c46e8a86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, NonSQLPlan} import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource @@ -39,7 +39,7 @@ object StreamingRelation { * passing to [[StreamExecution]] to run a query. */ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: Seq[Attribute]) - extends LeafNode { + extends LeafNode with NonSQLPlan { override def isStreaming: Boolean = true override def toString: String = sourceName } @@ -48,7 +48,8 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: * Used to link a streaming [[Source]] of data into a * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. */ -case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) extends LeafNode { +case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) + extends LeafNode with NonSQLPlan { override def isStreaming: Boolean = true override def toString: String = source.toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e37f0c77795c3..1e81c334fdb6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -27,7 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, NonSQLPlan} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -170,6 +170,6 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi /** * Used to query the data that has been written into a [[MemorySink]]. */ -case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode { +case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode with NonSQLPlan { def this(sink: MemorySink) = this(sink, sink.schema.toAttributes) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala index aecfd3062147c..4e20c27b0cd2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Strategy import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.test.SharedSQLContext class SparkPlannerSuite extends SharedSQLContext { @@ -27,7 +27,7 @@ class SparkPlannerSuite extends SharedSQLContext { test("Ensure to go down only the first branch, not any other possible branches") { - case object NeverPlanned extends LeafNode { + case object NeverPlanned extends LeafNode with NonSQLPlan { override def output: Seq[Attribute] = Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index d62bc983d0279..97bb2b8e75c24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -242,4 +242,8 @@ private[hive] case class MetastoreRelation( override def newInstance(): MetastoreRelation = { MetastoreRelation(databaseName, tableName)(catalogTable, client, sparkSession) } + + override def sql: String = { + catalogTable.identifier.quotedString + } } diff --git a/sql/hive/src/test/resources/sqlgen/agg1.sql b/sql/hive/src/test/resources/sqlgen/agg1.sql index 05403a9dd8927..918cbeaaf66f5 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_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 +SELECT `count(value)` FROM (SELECT count(parquet_t1.`value`) AS `count(value)`, max(parquet_t1.`key`) AS `max(key)` FROM parquet_t1 GROUP BY parquet_t1.`key` HAVING (`max(key)` > CAST(0 AS BIGINT))) diff --git a/sql/hive/src/test/resources/sqlgen/agg2.sql b/sql/hive/src/test/resources/sqlgen/agg2.sql index 65d71714fe850..4e5c56a6767c2 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_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 +SELECT `count(value)` FROM (SELECT count(parquet_t1.`value`) AS `count(value)`, max(parquet_t1.`key`) AS `aggOrder0` FROM parquet_t1 GROUP BY parquet_t1.`key` ORDER BY `aggOrder0` ASC) diff --git a/sql/hive/src/test/resources/sqlgen/agg3.sql b/sql/hive/src/test/resources/sqlgen/agg3.sql index 14b19392cdce3..827126b4d5e41 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_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 +SELECT `count(value)` FROM (SELECT count(parquet_t1.`value`) AS `count(value)`, parquet_t1.`key` AS `aggOrder0`, max(parquet_t1.`key`) AS `aggOrder1` FROM parquet_t1 GROUP BY parquet_t1.`key` ORDER BY `aggOrder0` ASC, `aggOrder1` ASC) 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 e3e372d5eccdd..356de78d5089b 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_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 +SELECT `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT `_w0`, parquet_t2.`a`, `_we0`, (`_w0` + `_we0`) AS `(max(c) + count(a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))` FROM (SELECT `_w0`, parquet_t2.`a`, count(parquet_t2.`a`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `_we0` FROM (SELECT max(parquet_t2.`c`) AS `_w0`, parquet_t2.`a` FROM parquet_t2 GROUP BY parquet_t2.`a`, parquet_t2.`b`))) diff --git a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql index ec881a216e0b0..40761a626b034 100644 --- a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql @@ -5,4 +5,4 @@ FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) ORDER BY subq.key1, z.value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_3 +SELECT subq.`key1`, z.`value` FROM (SELECT x.`key` AS `key1`, x.`value` AS `value1`, y.`key` AS `key2`, y.`value` AS `value2` FROM (`default`.`src1`) AS x INNER JOIN (`default`.`src`) AS y ON (x.`key` = y.`key`)) AS subq INNER JOIN (`default`.`srcpart`) AS z ON (((subq.`key1` = z.`key`) AND (z.`ds` = '2008-04-08')) AND (CAST(z.`hr` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY subq.`key1` ASC, z.`value` ASC diff --git a/sql/hive/src/test/resources/sqlgen/case.sql b/sql/hive/src/test/resources/sqlgen/case.sql index 99630e88cff66..7fc18ca89a90b 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_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 +SELECT CASE WHEN ((parquet_t0.`id` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 WHEN ((parquet_t0.`id` % CAST(2 AS BIGINT)) = CAST(0 AS BIGINT)) THEN 1 END 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 parquet_t0 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 aed8f08804807..41204484a7dce 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_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 +SELECT CASE WHEN ((parquet_t0.`id` % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END AS `CASE WHEN ((id % CAST(2 AS BIGINT)) > CAST(0 AS BIGINT)) THEN 0 ELSE 1 END` FROM parquet_t0 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 e991ebafdc90e..a5f6af295d8f4 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_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 +SELECT CASE WHEN (parquet_t0.`id` = CAST(0 AS BIGINT)) THEN 'foo' WHEN (parquet_t0.`id` = CAST(1 AS BIGINT)) THEN 'bar' END AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar END` FROM parquet_t0 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 492777e376ecc..bd18e49d4573a 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_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 +SELECT CASE WHEN (parquet_t0.`id` = CAST(0 AS BIGINT)) THEN 'foo' WHEN (parquet_t0.`id` = CAST(1 AS BIGINT)) THEN 'bar' ELSE 'baz' END AS `CASE WHEN (id = CAST(0 AS BIGINT)) THEN foo WHEN (id = CAST(1 AS BIGINT)) THEN bar ELSE baz END` FROM parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/cluster_by.sql b/sql/hive/src/test/resources/sqlgen/cluster_by.sql index 3154791c3c5fd..96c3efdb96b70 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 CLUSTER BY parquet_t0.`id` 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 e41b645937d37..ba11f8d6a5521 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_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 +SELECT json_parquet_t0.`id` FROM 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 f5ceccde8c65b..840b9100480b5 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_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 +SELECT orc_parquet_t0.`id` FROM 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 2bccefe55e417..bacae81449bdf 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_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 +SELECT parquet_parquet_t0.`id` FROM 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 bced711caedf4..d0fcbcf8499ad 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_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 +SELECT count(DISTINCT parquet_t0.`id`) AS `count(DISTINCT id)` FROM parquet_t0 diff --git a/sql/hive/src/test/resources/sqlgen/distribute_by.sql b/sql/hive/src/test/resources/sqlgen/distribute_by.sql index 72863dcaf5c9c..969c4e596fea3 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 DISTRIBUTE BY parquet_t0.`id` 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 96b9b2dae87aa..ff58bd60b1bdb 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 CLUSTER BY parquet_t0.`id` diff --git a/sql/hive/src/test/resources/sqlgen/except.sql b/sql/hive/src/test/resources/sqlgen/except.sql index 7a7d27fcd6336..fbf23146c4a62 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_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 +(SELECT t0.`id` FROM `default`.`t0`) EXCEPT (SELECT t0.`id` FROM `default`.`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 9cd6514d771ff..e363876e8456f 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_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 +SELECT t.`a` FROM (SELECT (parquet_t1.`key` + CAST(1 AS BIGINT)) AS `a` FROM parquet_t1) AS t WHERE (t.`a` > CAST(5 AS BIGINT)) 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 805197a4ea11b..c3ab3cbaf4a68 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_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 +SELECT `val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) AS `val` WHERE (parquet_t3.`id` > CAST(2 AS BIGINT)) ORDER BY `val` ASC, parquet_t3.`id` ASC LIMIT 5 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 ef9a596197b8b..84f4960015606 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_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 +SELECT exp1.`val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr2`) exp1 AS `nested_array` LATERAL VIEW explode(exp1.`nested_array`) exp1 AS `val` WHERE (exp1.`val` > CAST(2 AS BIGINT)) ORDER BY exp1.`val` ASC, parquet_t3.`id` ASC LIMIT 5 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 2f6596ef422b0..a1339de1aa5e7 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_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 +SELECT exp.`val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) exp AS `val` 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 239980dd80bda..b43d2ba12f608 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_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 +SELECT exp.`val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW OUTER explode(parquet_t3.`arr`) exp AS `val` 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 7fe0298c8e171..2ff0cfe8f8333 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_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 +SELECT `col` FROM `default`.`t0` LATERAL VIEW explode(array(1, 2, 3)) AS `col` 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 8db834acc73a1..c5ad07b7aae3b 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_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 +SELECT `val` FROM `default`.`t0` LATERAL VIEW explode(array(1, 2, 3)) AS `val` 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 fef65e006867c..d464a19582053 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_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 +SELECT `col`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) AS `col` 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 e0e310888f11f..e888f09e7e157 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_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 +SELECT `val`, parquet_t3.`id` AS `a` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) AS `val` 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 ea5db850bef8a..dd03171413c89 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_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 +SELECT `col` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) AS `col` 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 8f75b825476e0..36b483c9403ef 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_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 +SELECT `val` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) AS `val` 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 984cce8a2ca83..dfccce5a3d0ae 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_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 +SELECT exp.`id`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr`) exp AS `id` 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 5c55b164c7feb..7f9811081ce2d 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_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 +SELECT exp.`id`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW OUTER explode(parquet_t3.`arr`) exp AS `id` 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 ee22fe8728995..d9a01a4e794dd 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_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 +SELECT explode(array(1, 2, 3)) AS `col` 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 0acded74b3eee..3a95be4f836eb 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_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 +SELECT explode(array(1, 2, 3)) AS `val` 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 db2b2cc732889..f7b497c162171 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_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 +SELECT count(1) AS `cnt`, (t.`key` % CAST(5 AS BIGINT)) AS `k1`, (t.`key` - CAST(5 AS BIGINT)) AS `k2`, grouping_id() AS `k3` FROM (SELECT t.`key`, t.`(key % CAST(2 AS BIGINT))`, t.`(key - CAST(5 AS BIGINT))`, (t.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))`, (t.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM (SELECT parquet_t1.`key`, (parquet_t1.`key` % CAST(2 AS BIGINT)) AS `(key % CAST(2 AS BIGINT))`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `(key - CAST(5 AS BIGINT))` FROM parquet_t1) AS t) GROUP BY (t.`key` % CAST(5 AS BIGINT)), (t.`key` - CAST(5 AS BIGINT)) GROUPING SETS(((t.`key` % CAST(5 AS BIGINT))), ((t.`key` - CAST(5 AS BIGINT)))) 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 b2c426c660d80..111c29fdb870e 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`), (parquet_t2.`b`)) ORDER BY `a` ASC, `b` ASC 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 96ee8e85951e8..a397507674ab1 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`)) ORDER BY `a` ASC, `b` ASC 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 9b8b230c879c2..d3833ed448662 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`b`)) ORDER BY `a` ASC, `b` ASC 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 c35db74a5c5b5..bacab743565d8 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS(()) ORDER BY `a` ASC, `b` ASC 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 e47f6d5dcf465..d4dbf34ff0266 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((), (parquet_t2.`a`), (parquet_t2.`a`, parquet_t2.`b`)) ORDER BY `a` ASC, `b` ASC diff --git a/sql/hive/src/test/resources/sqlgen/in.sql b/sql/hive/src/test/resources/sqlgen/in.sql index 7cff62b1af7df..f19630f5d166d 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 WHERE (CAST(parquet_t0.`id` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT))) diff --git a/sql/hive/src/test/resources/sqlgen/inline_tables.sql b/sql/hive/src/test/resources/sqlgen/inline_tables.sql index 18803a3ee59b9..de6dedfc44918 100644 --- a/sql/hive/src/test/resources/sqlgen/inline_tables.sql +++ b/sql/hive/src/test/resources/sqlgen/inline_tables.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) where b > 1 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (VALUES ('one', 1), ('two', 2), ('three', CAST(NULL AS INT)) AS gen_subquery_0(gen_attr_0, gen_attr_1)) AS data WHERE (`gen_attr_1` > 1)) AS data +SELECT data.`a`, data.`b` FROM VALUES ('one', 1), ('two', 2), ('three', CAST(NULL AS INT)) AS data(a, b) WHERE (data.`b` > 1) diff --git a/sql/hive/src/test/resources/sqlgen/intersect.sql b/sql/hive/src/test/resources/sqlgen/intersect.sql index 4143a6208d4b5..7354feb46dec9 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_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 +(SELECT t0.`id` FROM `default`.`t0`) INTERSECT (SELECT t0.`id` FROM `default`.`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 9dd200c3c0cfa..1b2b1bf57b88f 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_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 +SELECT `count(value)`, b.`KEY`, a.`KEY` FROM (SELECT count(a.`value`) AS `count(value)`, b.`KEY`, a.`KEY`, max(a.`KEY`) AS `max(KEY)` FROM (parquet_t1) AS a INNER JOIN (parquet_t1) AS b GROUP BY a.`KEY`, b.`KEY` HAVING (`max(KEY)` > CAST(0 AS BIGINT))) 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 11e45a48f1b89..d403cc8e8c4e8 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_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 +SELECT jt.`c0`, jt.`c1`, jt.`c2` FROM parquet_t3 LATERAL VIEW json_tuple(parquet_t3.`json`, 'f1', 'f2', 'f3') jt AS `c0`, `c1`, `c2` 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 d86b39df57442..239c004dca51f 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_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 +SELECT jt.`a`, jt.`b`, jt.`c` FROM parquet_t3 LATERAL VIEW json_tuple(parquet_t3.`json`, 'f1', 'f2', 'f3') jt AS `a`, `b`, `c` diff --git a/sql/hive/src/test/resources/sqlgen/multi_distinct.sql b/sql/hive/src/test/resources/sqlgen/multi_distinct.sql index 3ca526fcc4415..9eb042cc8834c 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_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 +SELECT parquet_t2.`a`, count(DISTINCT parquet_t2.`b`) AS `count(DISTINCT b)`, count(DISTINCT parquet_t2.`c`) AS `count(DISTINCT c)`, sum(parquet_t2.`d`) AS `sum(d)` FROM parquet_t2 GROUP BY parquet_t2.`a` 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 e681c2b6354c0..5d6a93657d515 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_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 +SELECT exp1.`val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr2`) exp1 AS `nested_array` LATERAL VIEW explode(exp1.`nested_array`) exp1 AS `val` 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 e9d6522c91680..2380fcf28aadc 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_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 +SELECT exp1.`val`, parquet_t3.`id` FROM parquet_t3 LATERAL VIEW explode(parquet_t3.`arr2`) exp1 AS `nested_array` LATERAL VIEW OUTER explode(exp1.`nested_array`) exp1 AS `val` diff --git a/sql/hive/src/test/resources/sqlgen/not_in.sql b/sql/hive/src/test/resources/sqlgen/not_in.sql index 797d22e8e9154..bade12f549ba4 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_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 +SELECT t0.`id` FROM `default`.`t0` WHERE (NOT (CAST(t0.`id` AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2 AS BIGINT), CAST(3 AS BIGINT)))) diff --git a/sql/hive/src/test/resources/sqlgen/not_like.sql b/sql/hive/src/test/resources/sqlgen/not_like.sql index 22485045e212e..107a657cd00bb 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_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 +SELECT t0.`id` FROM `default`.`t0` WHERE (NOT CAST((t0.`id` + CAST(5 AS BIGINT)) AS STRING) LIKE '1%') diff --git a/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql b/sql/hive/src/test/resources/sqlgen/predicate_subquery.sql index 6e5bd9860008c..f10051adbc2d9 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_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 +SELECT b.`a` FROM (`default`.`t1`) AS b WHERE EXISTS (SELECT a.`a` FROM (`default`.`t1`) AS a) diff --git a/sql/hive/src/test/resources/sqlgen/range.sql b/sql/hive/src/test/resources/sqlgen/range.sql index 53c72ea71e6ac..3445f8db8c7b8 100644 --- a/sql/hive/src/test/resources/sqlgen/range.sql +++ b/sql/hive/src/test/resources/sqlgen/range.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from range(100) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT id AS `gen_attr_0` FROM range(0, 100, 1)) AS gen_subquery_0) AS gen_subquery_1 +SELECT `id` FROM range(0, 100, 1) diff --git a/sql/hive/src/test/resources/sqlgen/range_with_splits.sql b/sql/hive/src/test/resources/sqlgen/range_with_splits.sql index 83d637d54a302..853d4e39efca5 100644 --- a/sql/hive/src/test/resources/sqlgen/range_with_splits.sql +++ b/sql/hive/src/test/resources/sqlgen/range_with_splits.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. select * from range(1, 100, 20, 10) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM (SELECT id AS `gen_attr_0` FROM range(1, 100, 20, 10)) AS gen_subquery_0) AS gen_subquery_1 +SELECT `id` FROM range(1, 100, 20, 10) 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 37cd5568baa7f..9e4572e7cc88d 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_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 +SELECT `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT parquet_t1.`key`, `_w1`, `_we0`, (`_we0` + parquet_t1.`key`) AS `(max(key) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) + key)` FROM (SELECT parquet_t1.`key`, `_w1`, max(parquet_t1.`key`) OVER (PARTITION BY `_w1` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `_we0` FROM (SELECT parquet_t1.`key`, (parquet_t1.`key` % CAST(3 AS BIGINT)) AS `_w1` FROM parquet_t1))) 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 c54963ab5c550..3214234c00447 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_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 +SELECT count(1) AS `cnt`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % CAST(5 AS BIGINT))`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))` FROM parquet_t1) GROUP BY (parquet_t1.`key` % CAST(5 AS BIGINT)) GROUPING SETS(((parquet_t1.`key` % CAST(5 AS BIGINT))), ()) 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 6c869063c3bec..c4bac3c9f4f81 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_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 +SELECT count(1) AS `cnt`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % CAST(5 AS BIGINT))`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))` FROM parquet_t1) GROUP BY (parquet_t1.`key` % CAST(5 AS BIGINT)) GROUPING SETS(((parquet_t1.`key` % CAST(5 AS BIGINT))), ()) 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 9628e38572940..7e52aa009c44c 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_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 +SELECT parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value`, count(parquet_t1.`value`) AS `count(value)` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value` FROM parquet_t1) GROUP BY parquet_t1.`key`, parquet_t1.`value` GROUPING SETS((parquet_t1.`key`, parquet_t1.`value`), (parquet_t1.`key`), ()) 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 d6b61929df0ad..b84d02a3ff77f 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_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 +SELECT parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value`, count(parquet_t1.`value`) AS `count(value)` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value` FROM parquet_t1) GROUP BY parquet_t1.`key`, parquet_t1.`value` GROUPING SETS((parquet_t1.`key`, parquet_t1.`value`), (parquet_t1.`key`), (parquet_t1.`value`), ()) 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 d04b6578fc1ce..f17cb92fbfd60 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_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 +SELECT parquet_t1.`key` AS `key`, count(parquet_t1.`value`) AS `count(value)`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value` FROM parquet_t1) GROUP BY parquet_t1.`key`, parquet_t1.`value` GROUPING SETS((parquet_t1.`key`, parquet_t1.`value`), (parquet_t1.`key`), ()) 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 80a5d93438f2a..c103f4ba56fb7 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_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 +SELECT parquet_t1.`key` AS `key`, count(parquet_t1.`value`) AS `count(value)`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, parquet_t1.`key` AS `key`, parquet_t1.`value` AS `value` FROM parquet_t1) GROUP BY parquet_t1.`key`, parquet_t1.`value` GROUPING SETS((parquet_t1.`key`, parquet_t1.`value`), (parquet_t1.`key`), (parquet_t1.`value`), ()) 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 619a554875ff0..75418aca8ccd8 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_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 +SELECT count(1) AS `cnt`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `k1`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `k2`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM parquet_t1) GROUP BY (parquet_t1.`key` % CAST(5 AS BIGINT)), (parquet_t1.`key` - CAST(5 AS BIGINT)) GROUPING SETS(((parquet_t1.`key` % CAST(5 AS BIGINT)), (parquet_t1.`key` - CAST(5 AS BIGINT))), ((parquet_t1.`key` % CAST(5 AS BIGINT))), ()) 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 8bf164519165c..b73761c62c76e 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_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 +SELECT count(1) AS `cnt`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `k1`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `k2`, grouping_id() AS `grouping_id()` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM parquet_t1) GROUP BY (parquet_t1.`key` % CAST(5 AS BIGINT)), (parquet_t1.`key` - CAST(5 AS BIGINT)) GROUPING SETS(((parquet_t1.`key` % CAST(5 AS BIGINT)), (parquet_t1.`key` - CAST(5 AS BIGINT))), ((parquet_t1.`key` % CAST(5 AS BIGINT))), ((parquet_t1.`key` - CAST(5 AS BIGINT))), ()) 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 17e78a0a706a5..20ebce36a9542 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_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 +SELECT count(1) AS `cnt`, (t.`key` % CAST(5 AS BIGINT)) AS `k1`, (t.`key` - CAST(5 AS BIGINT)) AS `k2`, grouping_id() AS `k3` FROM (SELECT t.`key`, t.`(key % CAST(2 AS BIGINT))`, t.`(key - CAST(5 AS BIGINT))`, (t.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))`, (t.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM (SELECT parquet_t1.`key`, (parquet_t1.`key` % CAST(2 AS BIGINT)) AS `(key % CAST(2 AS BIGINT))`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `(key - CAST(5 AS BIGINT))` FROM parquet_t1) AS t) GROUP BY (t.`key` % CAST(5 AS BIGINT)), (t.`key` - CAST(5 AS BIGINT)) GROUPING SETS(((t.`key` % CAST(5 AS BIGINT)), (t.`key` - CAST(5 AS BIGINT))), ((t.`key` % CAST(5 AS BIGINT))), ()) 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 72506ef72aecd..e664c446ee4ec 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_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 +SELECT count(1) AS `cnt`, (t.`key` % CAST(5 AS BIGINT)) AS `k1`, (t.`key` - CAST(5 AS BIGINT)) AS `k2`, grouping_id() AS `k3` FROM (SELECT t.`key`, t.`(key % CAST(2 AS BIGINT))`, t.`(key - CAST(5 AS BIGINT))`, (t.`key` % CAST(5 AS BIGINT)) AS `(key % cast(5 as bigint))`, (t.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM (SELECT parquet_t1.`key`, (parquet_t1.`key` % CAST(2 AS BIGINT)) AS `(key % CAST(2 AS BIGINT))`, (parquet_t1.`key` - CAST(5 AS BIGINT)) AS `(key - CAST(5 AS BIGINT))` FROM parquet_t1) AS t) GROUP BY (t.`key` % CAST(5 AS BIGINT)), (t.`key` - CAST(5 AS BIGINT)) GROUPING SETS(((t.`key` % CAST(5 AS BIGINT)), (t.`key` - CAST(5 AS BIGINT))), ((t.`key` % CAST(5 AS BIGINT))), ((t.`key` - CAST(5 AS BIGINT))), ()) 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 22df578518ef3..d8eb62915265e 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), ()) ORDER BY `a` ASC, `b` ASC 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 f44b652343acb..0d50c4061199e 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`c`) AS `sum(c)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), (parquet_t2.`b`), ()) ORDER BY `a` ASC, `b` ASC 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 40f6924913765..abdd8c4567256 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`a`) AS `sum(a)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), ()) ORDER BY `a` ASC, `b` ASC 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 608e644dee6d0..fb23f34d02d1a 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, sum(parquet_t2.`a`) AS `sum(a)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), (parquet_t2.`b`), ()) ORDER BY `a` ASC, `b` ASC 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 26885a26e2b96..0fe9aef9494db 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_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 +SELECT (parquet_t2.`a` + parquet_t2.`b`) AS `(a + b)`, parquet_t2.`b` AS `b`, sum((parquet_t2.`a` - parquet_t2.`b`)) AS `sum((a - b))` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, (parquet_t2.`a` + parquet_t2.`b`) AS `(a + b)`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY (parquet_t2.`a` + parquet_t2.`b`), parquet_t2.`b` GROUPING SETS(((parquet_t2.`a` + parquet_t2.`b`), parquet_t2.`b`), ((parquet_t2.`a` + parquet_t2.`b`)), ()) 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 dd97c976afe61..0b0061d9790e0 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_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 +SELECT (parquet_t2.`a` + parquet_t2.`b`) AS `(a + b)`, parquet_t2.`b` AS `b`, sum((parquet_t2.`a` - parquet_t2.`b`)) AS `sum((a - b))` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, (parquet_t2.`a` + parquet_t2.`b`) AS `(a + b)`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY (parquet_t2.`a` + parquet_t2.`b`), parquet_t2.`b` GROUPING SETS(((parquet_t2.`a` + parquet_t2.`b`), parquet_t2.`b`), ((parquet_t2.`a` + parquet_t2.`b`)), (parquet_t2.`b`), ()) 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 aae2d75d794be..9b03072592171 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, grouping_id() AS `grouping_id(a, b)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), (parquet_t2.`b`), ()) 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 9958c8f38bc87..901259ffd77ad 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, grouping(parquet_t2.`b`) AS `grouping(b)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), (parquet_t2.`b`), ()) 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 fd012043cf6cb..f17ed75e7c1d1 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_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 +SELECT parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b`, grouping(parquet_t2.`a`) AS `grouping(a)` FROM (SELECT parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`, parquet_t2.`a` AS `a`, parquet_t2.`b` AS `b` FROM parquet_t2) GROUP BY parquet_t2.`a`, parquet_t2.`b` GROUPING SETS((parquet_t2.`a`, parquet_t2.`b`), (parquet_t2.`a`), (parquet_t2.`b`), ()) 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 61c27067e1521..a27f16614fbcc 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_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 +SELECT t.`hkey` AS `k1`, (t.`value` - CAST(5 AS BIGINT)) AS `k2`, hash(grouping_id()) AS `hgid` FROM (SELECT t.`hkey`, t.`value`, t.`hkey` AS `hkey`, (t.`value` - CAST(5 AS BIGINT)) AS `(value - cast(5 as bigint))` FROM (SELECT hash(parquet_t1.`key`) AS `hkey`, parquet_t1.`key` AS `value` FROM parquet_t1) AS t) GROUP BY t.`hkey`, (t.`value` - CAST(5 AS BIGINT)) GROUPING SETS((t.`hkey`, (t.`value` - CAST(5 AS BIGINT))), (t.`hkey`), ()) 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 16f254fa41f78..b5533fa9798f4 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_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 +SELECT t.`hkey` AS `k1`, (t.`value` - CAST(5 AS BIGINT)) AS `k2`, hash(grouping_id()) AS `hgid` FROM (SELECT t.`hkey`, t.`value`, t.`hkey` AS `hkey`, (t.`value` - CAST(5 AS BIGINT)) AS `(value - cast(5 as bigint))` FROM (SELECT hash(parquet_t1.`key`) AS `hkey`, parquet_t1.`key` AS `value` FROM parquet_t1) AS t) GROUP BY t.`hkey`, (t.`value` - CAST(5 AS BIGINT)) GROUPING SETS((t.`hkey`, (t.`value` - CAST(5 AS BIGINT))), (t.`hkey`), ((t.`value` - CAST(5 AS BIGINT))), ()) 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 cfce1758434de..4fcc497de13a7 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_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 +SELECT (t.`key` - CAST(5 AS BIGINT)) AS `(key - CAST(5 AS BIGINT))`, t.`cnt` AS `cnt`, sum(t.`cnt`) AS `sum(cnt)` FROM (SELECT t.`key`, t.`cnt`, t.`cnt` AS `cnt`, (t.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM (SELECT x.`key`, count(1) AS `cnt` FROM (parquet_t1) AS x INNER JOIN (parquet_t1) AS y ON (x.`key` = y.`key`) GROUP BY x.`key`) AS t) GROUP BY t.`cnt`, (t.`key` - CAST(5 AS BIGINT)) GROUPING SETS((t.`cnt`, (t.`key` - CAST(5 AS BIGINT))), (t.`cnt`), ()) 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 d950674b74c19..668663cec02c2 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_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 +SELECT (t.`key` - CAST(5 AS BIGINT)) AS `(key - CAST(5 AS BIGINT))`, t.`cnt` AS `cnt`, sum(t.`cnt`) AS `sum(cnt)` FROM (SELECT t.`key`, t.`cnt`, t.`cnt` AS `cnt`, (t.`key` - CAST(5 AS BIGINT)) AS `(key - cast(5 as bigint))` FROM (SELECT x.`key`, count(1) AS `cnt` FROM (parquet_t1) AS x INNER JOIN (parquet_t1) AS y ON (x.`key` = y.`key`) GROUP BY x.`key`) AS t) GROUP BY t.`cnt`, (t.`key` - CAST(5 AS BIGINT)) GROUPING SETS((t.`cnt`, (t.`key` - CAST(5 AS BIGINT))), (t.`cnt`), ((t.`key` - CAST(5 AS BIGINT))), ()) 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 1736d74b0cfa9..7e126bd3a81d0 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_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 +SELECT TRANSFORM (parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`) USING 'cat' AS (`key` string, `value` string) FROM parquet_t2 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 07f59d6bffddc..64ed35271304e 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_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 +SELECT TRANSFORM (parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`) USING 'cat' AS (`key` string, `value` string) FROM parquet_t2 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 fc0cabec237bc..ae4338e37512e 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_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 +SELECT TRANSFORM (parquet_t2.`a`, parquet_t2.`b`, parquet_t2.`c`, parquet_t2.`d`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`d1` string, `d2` string, `d3` string, `d4` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM parquet_t2 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 a45f9a2c625f6..8a20005626cce 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_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 +SELECT (t.`thing1` + 1) AS `(thing1 + 1)` FROM (SELECT TRANSFORM (parquet_t1.`key`, parquet_t1.`value`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') USING 'cat' AS (`thing1` int, `thing2` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = ' ') FROM parquet_t1) AS t 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 30d37c78b58e1..80652d0f06c40 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_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 +SELECT TRANSFORM (parquet_t1.`key`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' USING 'cat' AS (`tKey` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LINES TERMINATED BY '\t' FROM parquet_t1 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 0b694e0d6dafa..8bf28cd70d4ae 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_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 +SELECT TRANSFORM (parquet_t1.`key`) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' USING 'cat' AS (`tKey` string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' FROM parquet_t1 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 14cff373852dd..aec1725542c83 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_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 +SELECT TRANSFORM (parquet_t1.`key`, parquet_t1.`value`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') USING 'cat' AS (`tKey` string, `tValue` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES('field.delim' = '|') FROM parquet_t1 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 d20caf7afcf0f..ee550817c86c5 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_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 +SELECT TRANSFORM (parquet_t1.`key`, parquet_t1.`value`) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' USING 'cat' AS (`tKey` string, `tValue` string) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/select_distinct.sql b/sql/hive/src/test/resources/sqlgen/select_distinct.sql index 09d93cac8e5fd..7a130e46eea9c 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_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 +SELECT DISTINCT parquet_t0.`id` FROM 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 18ff021798972..bf59b3eaf1138 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_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 +SELECT orc_t.`c1`, orc_t.`c2` FROM `default`.`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 d2eac9c08f56c..98a72fd88bbc2 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_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 +SELECT parquet_t.`c1`, parquet_t.`c2` FROM 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 d6dcee2f67dbd..8f99f2ef21e31 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_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 +SELECT x.`key` FROM (parquet_t1) AS x INNER JOIN (parquet_t1) AS y ON (x.`key` = y.`key`) 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 1dedb44dbff65..6915e87e63a67 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_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 +SELECT x.`key`, count(1) AS `count(1)` FROM (parquet_t1) AS x INNER JOIN (parquet_t1) AS y ON (x.`key` = y.`key`) GROUP BY x.`key` 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 da60204297a21..e8f720fb22a03 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_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 +SELECT `count(value)` FROM (SELECT `count(value)`, parquet_t1.`key` FROM (SELECT count(parquet_t1.`value`) AS `count(value)`, max(parquet_t1.`key`) AS `max(key)`, parquet_t1.`key` FROM parquet_t1 GROUP BY parquet_t1.`key` HAVING (`max(key)` > CAST(0 AS BIGINT))) SORT BY parquet_t1.`key` ASC) diff --git a/sql/hive/src/test/resources/sqlgen/subq2.sql b/sql/hive/src/test/resources/sqlgen/subq2.sql index ee7e80c1fc9e2..178741bb752fa 100644 --- a/sql/hive/src/test/resources/sqlgen/subq2.sql +++ b/sql/hive/src/test/resources/sqlgen/subq2.sql @@ -5,4 +5,4 @@ FROM (SELECT b.key as k, count(1) as c GROUP BY b.key) a WHERE a.k >= 90 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `k`, `gen_attr_1` AS `c` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_2` AS `gen_attr_0`, count(1) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`) AS a WHERE (`gen_attr_0` >= 90)) AS a +SELECT a.`k`, a.`c` FROM (SELECT b.`key` AS `k`, count(1) AS `c` FROM (`default`.`src`) AS b GROUP BY b.`key`) AS a WHERE (a.`k` >= 90) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql index bd28d8dca94c2..2f1b844cea601 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_1.sql @@ -5,4 +5,4 @@ where exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_9') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS b +SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b WHERE EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9') AND (b.`value` = a.`value`) AND (a.`key` = b.`key`)) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql index d2965fc0b9b77..a31a04de78bdd 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_2.sql @@ -6,4 +6,4 @@ from (select * from src a where b.value = a.value and a.key = b.key and a.value > 'val_9')) a -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3)) AS a) AS a +SELECT a.`key`, a.`value` FROM (SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b WHERE EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9') AND (b.`value` = a.`value`) AND (a.`key` = b.`key`))) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql index 93ce902b75994..5d98f16812dc4 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_1.sql @@ -6,4 +6,4 @@ having exists (select a.key from src a where a.key = b.key and a.value > 'val_9') -------------------------------------------------------------------------------- -SELECT `gen_attr_1` AS `key`, `gen_attr_2` AS `count(1)` FROM (SELECT `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_0` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > 'val_9')) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3)) AS b +SELECT b.`key`, count(1) AS `count(1)` FROM (`default`.`src`) AS b GROUP BY b.`key` HAVING EXISTS (SELECT a.`key` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9') AND (a.`key` = b.`key`)) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql index 411e073f0d280..d3d0d26a841f0 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_2.sql @@ -7,4 +7,4 @@ from (select b.key, count(*) from src a where a.key = b.key and a.value > 'val_9')) a -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_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`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_5` > 'val_9')) AS gen_subquery_1 WHERE (`gen_attr_2` = `gen_attr_0`)) AS gen_subquery_3)) AS a) AS a +SELECT a.`key`, a.`count(1)` FROM (SELECT b.`key`, count(1) AS `count(1)` FROM (`default`.`src`) AS b GROUP BY b.`key` HAVING EXISTS (SELECT a.`key` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9') AND (a.`key` = b.`key`))) AS a diff --git a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql index b2ed0b0557aff..255d00473c0fa 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_exists_having_3.sql @@ -6,4 +6,4 @@ having exists (select a.key from src a where a.value > 'val_9' and a.value = min(b.value)) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_4`) AS `gen_attr_1`, min(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING EXISTS(SELECT `gen_attr_5` AS `1` FROM (SELECT 1 AS `gen_attr_5` FROM (SELECT `gen_attr_6`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_2` > 'val_9')) AS gen_subquery_2 WHERE (`gen_attr_2` = `gen_attr_3`)) AS gen_subquery_4)) AS gen_subquery_1) AS b +SELECT b.`key`, `min(value)` FROM (SELECT b.`key`, min(b.`value`) AS `min(value)`, min(b.`value`) AS `min(value)` FROM (`default`.`src`) AS b GROUP BY b.`key` HAVING EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9') AND (a.`value` = `min(value)`))) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in.sql b/sql/hive/src/test/resources/sqlgen/subquery_in.sql index 0fe62248dbfec..a14e2f9a97f70 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in.sql @@ -3,4 +3,4 @@ SELECT key FROM src WHERE key in (SELECT max(key) FROM src) -------------------------------------------------------------------------------- -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`.`src`) AS gen_subquery_0 WHERE (`gen_attr_0` IN (SELECT `gen_attr_3` AS `_c0` FROM (SELECT `gen_attr_1` AS `gen_attr_3` FROM (SELECT max(`gen_attr_4`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_2) AS gen_subquery_1) AS gen_subquery_3))) AS src +SELECT src.`key` FROM `default`.`src` WHERE src.`key` IN (SELECT max(src.`key`) AS `max(key)` FROM `default`.`src`) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql index 9894f5ab39c76..8636fdc362493 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql @@ -5,4 +5,4 @@ group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) order by key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS src +SELECT src.`key`, `count(1)` FROM (SELECT src.`key`, count(1) AS `count(1)` FROM `default`.`src` GROUP BY src.`key` HAVING `count(1)` IN (SELECT count(1) AS `count(1)` FROM (`default`.`src`) AS s1 WHERE (CAST(s1.`key` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY s1.`key`)) ORDER BY src.`key` ASC diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index c3a122aa889b9..22e58197ca348 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS b +SELECT b.`key`, `min(value)` FROM (SELECT b.`key`, min(b.`value`) AS `min(value)`, min(b.`value`) AS `min(value)` FROM (`default`.`src`) AS b GROUP BY b.`key` HAVING struct(b.`key`, `min(value)`) IN (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_9'))) ORDER BY b.`key` ASC diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql index eed20a5d311f3..39e0121221bf4 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_1.sql @@ -5,4 +5,4 @@ where not exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_2') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_3`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_2')) AS gen_subquery_1 WHERE ((`gen_attr_1` = `gen_attr_2`) AND (`gen_attr_3` = `gen_attr_0`))) AS gen_subquery_3))) AS b +SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b WHERE (NOT EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_2') AND (b.`value` = a.`value`) AND (a.`key` = b.`key`))) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql index 7040e106e7ba2..bd671fc758797 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_2.sql @@ -5,4 +5,4 @@ where not exists (select a.key from src a where b.value = a.value and a.value > 'val_2') -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_0 WHERE (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT `gen_attr_4`, `gen_attr_2` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_2` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_2` > 'val_2')) AS gen_subquery_1 WHERE (`gen_attr_1` = `gen_attr_2`)) AS gen_subquery_3))) AS b +SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b WHERE (NOT EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_2') AND (b.`value` = a.`value`))) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql index 3c0e90ed42223..654998ef63493 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_1.sql @@ -6,4 +6,4 @@ having not exists (select a.key from src a where b.value = a.value and a.key = b.key and a.value > 'val_12') -------------------------------------------------------------------------------- -SELECT `gen_attr_3` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_3`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_3`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4` FROM (SELECT `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > 'val_12')) AS gen_subquery_1 WHERE ((`gen_attr_0` = `gen_attr_1`) AND (`gen_attr_2` = `gen_attr_3`))) AS gen_subquery_3))) AS b +SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b GROUP BY b.`key`, b.`value` HAVING (NOT EXISTS (SELECT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_12') AND (b.`value` = a.`value`) AND (a.`key` = b.`key`))) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql index 0c16f9e58b9b9..e39f380d1164f 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_not_exists_having_2.sql @@ -6,4 +6,4 @@ having not exists (select distinct a.key from src a where b.value = a.value and a.value > 'val_12') -------------------------------------------------------------------------------- -SELECT `gen_attr_2` AS `key`, `gen_attr_0` AS `value` FROM (SELECT `gen_attr_2`, `gen_attr_0` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_0` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_2`, `gen_attr_0` HAVING (NOT EXISTS(SELECT `gen_attr_3` AS `1` FROM (SELECT 1 AS `gen_attr_3` FROM (SELECT DISTINCT `gen_attr_4`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_4`, `value` AS `gen_attr_1` FROM `default`.`src`) AS gen_subquery_2 WHERE (`gen_attr_1` > 'val_12')) AS gen_subquery_1 WHERE (`gen_attr_0` = `gen_attr_1`)) AS gen_subquery_3))) AS b +SELECT b.`key`, b.`value` FROM (`default`.`src`) AS b GROUP BY b.`key`, b.`value` HAVING (NOT EXISTS (SELECT DISTINCT a.`key`, a.`value` FROM (`default`.`src`) AS a WHERE (a.`value` > 'val_12') AND (b.`value` = a.`value`))) diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_1.sql b/sql/hive/src/test/resources/sqlgen/tablesample_1.sql index 291f2f59d7378..bb9c5146dc5f1 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_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 +SELECT s.`id` FROM parquet_t0 TABLESAMPLE(100.0 PERCENT) s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_2.sql b/sql/hive/src/test/resources/sqlgen/tablesample_2.sql index 6a92d7aef72f1..c6c4edab42810 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 TABLESAMPLE(100.0 PERCENT) 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 4a17d7105eec6..8194e9974880f 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_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 +SELECT s.`id` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT) s diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_4.sql b/sql/hive/src/test/resources/sqlgen/tablesample_4.sql index 873de051a6bd5..6e5fa6479d37c 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_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 +SELECT t0.`id` FROM `default`.`t0` TABLESAMPLE(100.0 PERCENT) diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_5.sql b/sql/hive/src/test/resources/sqlgen/tablesample_5.sql index f958b2f111ba2..e30c7e75c02aa 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_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 +SELECT s.`id` FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) s WHERE (1 = 0) diff --git a/sql/hive/src/test/resources/sqlgen/tablesample_6.sql b/sql/hive/src/test/resources/sqlgen/tablesample_6.sql index 688a102d1da4e..cf69971a09355 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_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 +SELECT parquet_t0.`id` FROM parquet_t0 TABLESAMPLE(0.1 PERCENT) parquet_t0 WHERE (1 = 0) 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 713c7502f5a1a..0ca4512b9af58 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_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 +((SELECT parquet_t0.`id` FROM parquet_t0) UNION ALL (SELECT parquet_t0.`id` FROM parquet_t0)) UNION ALL (SELECT parquet_t0.`id` FROM 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 ebb8a92afd345..8143da7db6e55 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_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 +(SELECT parquet_t0.`id` FROM (SELECT parquet_t0.`id` FROM parquet_t0)) UNION ALL (SELECT CAST(`id` AS BIGINT) AS `id` FROM (SELECT CAST(parquet_t0.`id` AS INT) AS `id` FROM 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 46644b89ebb04..3180c1c581fd4 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_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 +(SELECT t0.`id` FROM `default`.`t0`) UNION DISTINCT (SELECT t0.`id` FROM `default`.`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 000c4e735ac6e..c3076d28ecd5f 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_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 +SELECT `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT parquet_t1.`value`, `_w1`, `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)`, `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT parquet_t1.`value`, `_w1`, max(parquet_t1.`value`) OVER (PARTITION BY `_w1` ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `max(value) OVER (PARTITION BY (key % CAST(3 AS BIGINT)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)` FROM (SELECT parquet_t1.`value`, (parquet_t1.`key` % CAST(3 AS BIGINT)) AS `_w1` FROM parquet_t1))) 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 ec55d4b7146f2..58eef396ab56e 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_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 +SELECT parquet_t1.`key`, parquet_t1.`value`, `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `_we0`, round(`_we0`, 2) AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, avg(parquet_t1.`key`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `_we0` FROM (SELECT parquet_t1.`key`, parquet_t1.`value` FROM parquet_t1))) ORDER BY parquet_t1.`key` ASC 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 c0ac9541e67ee..afd14ceb407a5 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_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 +SELECT parquet_t1.`value`, `max` FROM (SELECT parquet_t1.`value`, `_w0`, `_w1`, `_w2`, `max`, `max` FROM (SELECT parquet_t1.`value`, `_w0`, `_w1`, `_w2`, max(`_w0`) OVER (PARTITION BY `_w1` ORDER BY `_w2` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `max` FROM (SELECT parquet_t1.`value`, (parquet_t1.`key` + CAST(1 AS BIGINT)) AS `_w0`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `_w1`, (parquet_t1.`key` % CAST(7 AS BIGINT)) AS `_w2` FROM 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 030a4c0907a1c..9de13d6bf73a8 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_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 +SELECT x.`key`, `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT x.`key`, y.`key`, `_w1`, `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)`, `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT x.`key`, y.`key`, `_w1`, max(y.`key`) OVER (PARTITION BY `_w1` ORDER BY x.`key` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT x.`key`, y.`key`, (x.`key` % CAST(5 AS BIGINT)) AS `_w1` FROM (parquet_t1) AS x INNER JOIN (parquet_t1) AS y ON (x.`key` = y.`key`)))) 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 7b99539a05480..9ea3f57c26da8 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_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 +SELECT parquet_t1.`key`, parquet_t1.`value`, `dr`, `count(key)` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `count(key)`, `dr`, `dr` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `count(key)`, DENSE_RANK() OVER (PARTITION BY parquet_t1.`key` ORDER BY parquet_t1.`key` ASC, parquet_t1.`value` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `dr` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, count(parquet_t1.`key`) AS `count(key)` FROM parquet_t1 GROUP BY parquet_t1.`key`, parquet_t1.`value`))) 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 591a654a3888e..1eee9c3aba286 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_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 +SELECT parquet_t1.`key`, parquet_t1.`value`, `dr`, `ca` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `dr`, `ca`, `dr`, `ca` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, DENSE_RANK() OVER (PARTITION BY parquet_t1.`key` ORDER BY parquet_t1.`key` ASC, parquet_t1.`value` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `dr`, count(parquet_t1.`key`) OVER (PARTITION BY parquet_t1.`key` ORDER BY parquet_t1.`key` ASC, parquet_t1.`value` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `ca` FROM (SELECT parquet_t1.`key`, parquet_t1.`value` FROM 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 d9169eab6e46a..af0f0cbf77cb9 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_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 +SELECT parquet_t1.`key`, parquet_t1.`value`, `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `_w0`, `max`, `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `_w0`, max(parquet_t1.`value`) OVER (PARTITION BY `_w0` ORDER BY parquet_t1.`key` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `_w0` FROM parquet_t1 GROUP BY parquet_t1.`key`, parquet_t1.`value`))) 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 f0a820811ee0a..4fc96885897fc 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_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 +SELECT parquet_t1.`key`, parquet_t1.`value`, `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `_w0`, `max`, `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, `_w0`, max(parquet_t1.`value`) OVER (PARTITION BY `_w0` ORDER BY parquet_t1.`key` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `max` FROM (SELECT parquet_t1.`key`, parquet_t1.`value`, (parquet_t1.`key` % CAST(5 AS BIGINT)) AS `_w0` FROM parquet_t1 GROUP BY parquet_t1.`key`, parquet_t1.`value` HAVING (parquet_t1.`key` > CAST(5 AS BIGINT))))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala index b4eb50e331cf9..37999a903af5b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionToSQLSuite.scala @@ -56,7 +56,7 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { private def checkSqlGeneration(hiveQl: String): Unit = { val df = sql(hiveQl) - val convertedSQL = try new SQLBuilder(df).toSQL catch { + val convertedSQL = try df.logicalPlan.sql catch { case NonFatal(e) => fail( s"""Cannot convert the following HiveQL query plan back to SQL query string: 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 9c6da6a628dcf..bc304a9ac4a6f 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, NonSQLPlan} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -101,12 +101,13 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { * Compare the generated SQL with the expected answer string. */ private def checkSQLStructure(originalSQL: String, convertedSQL: String, answerFile: String) = { + val normalizedSQL = convertedSQL.replaceAll("#\\d+L?", "") 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$convertedSQL\n" + val answerText = s"$header\n${originalSQL.trim()}\n${separator}\n$normalizedSQL\n" Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" @@ -119,7 +120,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") val expectedSQL = sqls(1).trim() - assert(convertedSQL == expectedSQL) + assert(normalizedSQL == expectedSQL) } } } @@ -133,7 +134,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { private def checkSQL(sqlString: String, answerFile: String = null): Unit = { val df = sql(sqlString) - val convertedSQL = try new SQLBuilder(df).toSQL catch { + val convertedSQL = try df.logicalPlan.sql catch { case NonFatal(e) => fail( s"""Cannot convert the following SQL query plan back to SQL query string: @@ -183,7 +184,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { } test("Test should fail if the SQL query cannot be regenerated") { - case class Unsupported() extends LeafNode with MultiInstanceRelation { + case class Unsupported() extends LeafNode with MultiInstanceRelation with NonSQLPlan { override def newInstance(): Unsupported = copy() override def output: Seq[Attribute] = Nil } @@ -621,7 +622,7 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { test("plans with non-SQL expressions") { spark.udf.register("foo", (_: Int) * 2) - intercept[UnsupportedOperationException](new SQLBuilder(sql("SELECT foo(id) FROM t0")).toSQL) + intercept[UnsupportedOperationException](sql("SELECT foo(id) FROM t0").logicalPlan.sql) } test("named expression in column names shouldn't be quoted") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 80e75aa898c38..2cfb01ddcf0d1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -26,7 +26,6 @@ import scala.util.control.NonFatal import org.scalatest.{BeforeAndAfterAll, GivenWhenThen} import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.SQLBuilder import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ @@ -354,7 +353,7 @@ abstract class HiveComparisonTest originalQuery } else { val convertedSQL = try { - new SQLBuilder(originalQuery.analyzed).toSQL + originalQuery.analyzed.sql } catch { case NonFatal(e) => fail( s"""Cannot convert the following HiveQL query plan back to SQL query string: