This is an automated email from the ASF dual-hosted git repository. gengliang pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new a4fb7cceb44 [SPARK-43205][SQL][FOLLOWUP] remove unnecessary abstraction for `withIdentClause` a4fb7cceb44 is described below commit a4fb7cceb441ddd30ce6613a27ba9b62402911fd Author: Wenchen Fan <wenc...@databricks.com> AuthorDate: Wed Jun 21 10:32:36 2023 -0700 [SPARK-43205][SQL][FOLLOWUP] remove unnecessary abstraction for `withIdentClause` ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/41385 . This PR adds `withFuncIndentClause` for function identifiers, so that the related methods can be simpler. ### Why are the changes needed? code cleanup ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests Closes #41631 from cloud-fan/followup. Authored-by: Wenchen Fan <wenc...@databricks.com> Signed-off-by: Gengliang Wang <gengli...@apache.org> --- .../spark/sql/catalyst/parser/AstBuilder.scala | 70 ++++++++++++---------- .../analyzer-results/identifier-clause.sql.out | 32 +++++----- .../sql-tests/inputs/identifier-clause.sql | 14 ++--- .../sql-tests/results/identifier-clause.sql.out | 14 ++--- 4 files changed, 70 insertions(+), 60 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index abfe64f72e7..07721424a86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -66,31 +66,44 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit protected def withIdentClause( ctx: IdentifierReferenceContext, builder: Seq[String] => LogicalPlan): LogicalPlan = { - withIdentClause( - ctx.expression, - () => visitMultipartIdentifier(ctx.multipartIdentifier), - builder) + val exprCtx = ctx.expression + if (exprCtx != null) { + PlanWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder) + } else { + builder.apply(visitMultipartIdentifier(ctx.multipartIdentifier)) + } } protected def withIdentClause( - ctx: ExpressionContext, - getIdent: () => Seq[String], + ctx: IdentifierReferenceContext, + builder: Seq[String] => Expression): Expression = { + val exprCtx = ctx.expression + if (exprCtx != null) { + ExpressionWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder) + } else { + builder.apply(visitMultipartIdentifier(ctx.multipartIdentifier)) + } + } + + protected def withFuncIdentClause( + ctx: FunctionNameContext, builder: Seq[String] => LogicalPlan): LogicalPlan = { - if (ctx != null) { - PlanWithUnresolvedIdentifier(withOrigin(ctx) { expression(ctx) }, builder) + val exprCtx = ctx.expression + if (exprCtx != null) { + PlanWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder) } else { - builder.apply(getIdent()) + builder.apply(getFunctionMultiparts(ctx)) } } - protected def withIdentClause( - ctx: ExpressionContext, - getIdent: () => Seq[String], + protected def withFuncIdentClause( + ctx: FunctionNameContext, builder: Seq[String] => Expression): Expression = { - if (ctx != null) { - ExpressionWithUnresolvedIdentifier(withOrigin(ctx) { expression(ctx) }, builder) + val exprCtx = ctx.expression + if (exprCtx != null) { + ExpressionWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder) } else { - builder.apply(getIdent()) + builder.apply(getFunctionMultiparts(ctx)) } } @@ -1538,21 +1551,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit Seq.empty } - withIdentClause( - func.functionName.expression, - () => getFunctionMultiparts(func.functionName), - name => { - if (name.length > 1) { - throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx) - } + withFuncIdentClause(func.functionName, ident => { + if (ident.length > 1) { + throw QueryParsingErrors.invalidTableValuedFunctionNameError(ident, ctx) + } - val tvf = UnresolvedTableValuedFunction(name, func.expression.asScala.map(expression).toSeq) + val tvf = UnresolvedTableValuedFunction(ident, func.expression.asScala.map(expression).toSeq) - val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf + val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(ident, tvf, aliases) else tvf - tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) - } - ) + tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) + }) } /** @@ -2189,9 +2198,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit val ignoreNulls = Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false) val funcCtx = ctx.functionName - val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => { - UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) - }) + val func = withFuncIdentClause( + funcCtx, + ident => UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls) + ) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index 7fc18dfe04e..367d492905d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -69,7 +69,7 @@ CreateNamespace true -- !query -CREATE TABLE s.tab(c1 INT) USING PARQUET +CREATE TABLE s.tab(c1 INT) USING CSV -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`s`.`tab`, false @@ -83,7 +83,7 @@ SetNamespaceCommand [s] -- !query INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/s.db/tab, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/s.db/tab], Append, `spark_catalog`.`s`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/s.db/tab), [c1] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/s.db/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/s.db/tab], Append, `spark_catalog`.`s`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/s.db/tab), [c1] +- Project [cast(col1#x as int) AS c1#x] +- LocalRelation [col1#x] @@ -132,7 +132,7 @@ SELECT * FROM IDENTIFIER('tab') -- !query analysis Project [c1#x] +- SubqueryAlias spark_catalog.s.tab - +- Relation spark_catalog.s.tab[c1#x] parquet + +- Relation spark_catalog.s.tab[c1#x] csv -- !query @@ -140,7 +140,7 @@ SELECT * FROM IDENTIFIER('s.tab') -- !query analysis Project [c1#x] +- SubqueryAlias spark_catalog.s.tab - +- Relation spark_catalog.s.tab[c1#x] parquet + +- Relation spark_catalog.s.tab[c1#x] csv -- !query @@ -148,7 +148,7 @@ SELECT * FROM IDENTIFIER('`s`.`tab`') -- !query analysis Project [c1#x] +- SubqueryAlias spark_catalog.s.tab - +- Relation spark_catalog.s.tab[c1#x] parquet + +- Relation spark_catalog.s.tab[c1#x] csv -- !query @@ -156,7 +156,7 @@ SELECT * FROM IDENTIFIER('t' || 'a' || 'b') -- !query analysis Project [c1#x] +- SubqueryAlias spark_catalog.s.tab - +- Relation spark_catalog.s.tab[c1#x] parquet + +- Relation spark_catalog.s.tab[c1#x] csv -- !query @@ -201,7 +201,7 @@ Project [id#xL] -- !query -CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false @@ -228,7 +228,7 @@ SetCatalogAndNamespace -- !query -CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false @@ -241,13 +241,13 @@ DropTable true, false -- !query -CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false -- !query -REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV -- !query analysis org.apache.spark.sql.AnalysisException { @@ -265,7 +265,7 @@ CACHE TABLE IDENTIFIER('ta' || 'b') -- !query analysis CacheTable [tab], false, true +- SubqueryAlias spark_catalog.identifier_clauses.tab - +- Relation spark_catalog.identifier_clauses.tab[c1#x] parquet + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv -- !query @@ -273,7 +273,7 @@ UNCACHE TABLE IDENTIFIER('ta' || 'b') -- !query analysis UncacheTable false, true +- SubqueryAlias spark_catalog.identifier_clauses.tab - +- Relation spark_catalog.identifier_clauses.tab[c1#x] parquet + +- Relation spark_catalog.identifier_clauses.tab[c1#x] csv -- !query @@ -298,7 +298,7 @@ DropNamespace false, false -- !query -CREATE TABLE tab(c1 INT) USING parquet +CREATE TABLE tab(c1 INT) USING CSV -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false @@ -306,7 +306,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false -- !query INSERT INTO tab VALUES (1) -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/tab, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/tab], Append, `spark_catalog`.`default`.`tab`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/tab), [c1] +- Project [cast(col1#x as int) AS c1#x] +- LocalRelation [col1#x] @@ -316,7 +316,7 @@ SELECT c1 FROM tab -- !query analysis Project [c1#x] +- SubqueryAlias spark_catalog.default.tab - +- Relation spark_catalog.default.tab[c1#x] parquet + +- Relation spark_catalog.default.tab[c1#x] csv -- !query @@ -1055,7 +1055,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index 93e67411172..a1bd500455d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -17,7 +17,7 @@ SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1); -- Table references CREATE SCHEMA IF NOT EXISTS s; -CREATE TABLE s.tab(c1 INT) USING PARQUET; +CREATE TABLE s.tab(c1 INT) USING CSV; USE SCHEMA s; INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1); @@ -40,22 +40,22 @@ SELECT IDENTIFIER('abs')(-1); SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1); -- Table DDL -CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet; +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV; DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b'); CREATE SCHEMA identifier_clauses; USE identifier_clauses; -CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet; +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV; DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab'); -CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet; -REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet; +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV; +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV; CACHE TABLE IDENTIFIER('ta' || 'b'); UNCACHE TABLE IDENTIFIER('ta' || 'b'); DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b'); USE default; DROP SCHEMA identifier_clauses; -CREATE TABLE tab(c1 INT) USING parquet; +CREATE TABLE tab(c1 INT) USING CSV; INSERT INTO tab VALUES (1); SELECT c1 FROM tab; DESCRIBE IDENTIFIER('ta' || 'b'); @@ -135,7 +135,7 @@ SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v); INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1); -CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet; +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV; diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 97220606317..9410ccc5e54 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -72,7 +72,7 @@ struct<> -- !query -CREATE TABLE s.tab(c1 INT) USING PARQUET +CREATE TABLE s.tab(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -221,7 +221,7 @@ struct<id:bigint> -- !query -CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -253,7 +253,7 @@ struct<> -- !query -CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -269,7 +269,7 @@ struct<> -- !query -CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -277,7 +277,7 @@ struct<> -- !query -REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -333,7 +333,7 @@ struct<> -- !query -CREATE TABLE tab(c1 INT) USING parquet +CREATE TABLE tab(c1 INT) USING CSV -- !query schema struct<> -- !query output @@ -1209,7 +1209,7 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query -CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV -- !query schema struct<> -- !query output --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org