Skip to content

Commit a4fb7cc

Browse files
cloud-fangengliangwang
authored andcommitted
[SPARK-43205][SQL][FOLLOWUP] remove unnecessary abstraction for withIdentClause
### What changes were proposed in this pull request? This is a followup of apache#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 apache#41631 from cloud-fan/followup. Authored-by: Wenchen Fan <[email protected]> Signed-off-by: Gengliang Wang <[email protected]>
1 parent 6165f31 commit a4fb7cc

File tree

4 files changed

+70
-60
lines changed

4 files changed

+70
-60
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 40 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -66,31 +66,44 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
6666
protected def withIdentClause(
6767
ctx: IdentifierReferenceContext,
6868
builder: Seq[String] => LogicalPlan): LogicalPlan = {
69-
withIdentClause(
70-
ctx.expression,
71-
() => visitMultipartIdentifier(ctx.multipartIdentifier),
72-
builder)
69+
val exprCtx = ctx.expression
70+
if (exprCtx != null) {
71+
PlanWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder)
72+
} else {
73+
builder.apply(visitMultipartIdentifier(ctx.multipartIdentifier))
74+
}
7375
}
7476

7577
protected def withIdentClause(
76-
ctx: ExpressionContext,
77-
getIdent: () => Seq[String],
78+
ctx: IdentifierReferenceContext,
79+
builder: Seq[String] => Expression): Expression = {
80+
val exprCtx = ctx.expression
81+
if (exprCtx != null) {
82+
ExpressionWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder)
83+
} else {
84+
builder.apply(visitMultipartIdentifier(ctx.multipartIdentifier))
85+
}
86+
}
87+
88+
protected def withFuncIdentClause(
89+
ctx: FunctionNameContext,
7890
builder: Seq[String] => LogicalPlan): LogicalPlan = {
79-
if (ctx != null) {
80-
PlanWithUnresolvedIdentifier(withOrigin(ctx) { expression(ctx) }, builder)
91+
val exprCtx = ctx.expression
92+
if (exprCtx != null) {
93+
PlanWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder)
8194
} else {
82-
builder.apply(getIdent())
95+
builder.apply(getFunctionMultiparts(ctx))
8396
}
8497
}
8598

86-
protected def withIdentClause(
87-
ctx: ExpressionContext,
88-
getIdent: () => Seq[String],
99+
protected def withFuncIdentClause(
100+
ctx: FunctionNameContext,
89101
builder: Seq[String] => Expression): Expression = {
90-
if (ctx != null) {
91-
ExpressionWithUnresolvedIdentifier(withOrigin(ctx) { expression(ctx) }, builder)
102+
val exprCtx = ctx.expression
103+
if (exprCtx != null) {
104+
ExpressionWithUnresolvedIdentifier(withOrigin(exprCtx) { expression(exprCtx) }, builder)
92105
} else {
93-
builder.apply(getIdent())
106+
builder.apply(getFunctionMultiparts(ctx))
94107
}
95108
}
96109

@@ -1538,21 +1551,17 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
15381551
Seq.empty
15391552
}
15401553

1541-
withIdentClause(
1542-
func.functionName.expression,
1543-
() => getFunctionMultiparts(func.functionName),
1544-
name => {
1545-
if (name.length > 1) {
1546-
throw QueryParsingErrors.invalidTableValuedFunctionNameError(name, ctx)
1547-
}
1554+
withFuncIdentClause(func.functionName, ident => {
1555+
if (ident.length > 1) {
1556+
throw QueryParsingErrors.invalidTableValuedFunctionNameError(ident, ctx)
1557+
}
15481558

1549-
val tvf = UnresolvedTableValuedFunction(name, func.expression.asScala.map(expression).toSeq)
1559+
val tvf = UnresolvedTableValuedFunction(ident, func.expression.asScala.map(expression).toSeq)
15501560

1551-
val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(name, tvf, aliases) else tvf
1561+
val tvfAliases = if (aliases.nonEmpty) UnresolvedTVFAliases(ident, tvf, aliases) else tvf
15521562

1553-
tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan)
1554-
}
1555-
)
1563+
tvfAliases.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan)
1564+
})
15561565
}
15571566

15581567
/**
@@ -2189,9 +2198,10 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit
21892198
val ignoreNulls =
21902199
Option(ctx.nullsOption).map(_.getType == SqlBaseParser.IGNORE).getOrElse(false)
21912200
val funcCtx = ctx.functionName
2192-
val func = withIdentClause(funcCtx.expression, () => getFunctionMultiparts(funcCtx), ident => {
2193-
UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls)
2194-
})
2201+
val func = withFuncIdentClause(
2202+
funcCtx,
2203+
ident => UnresolvedFunction(ident, arguments, isDistinct, filter, ignoreNulls)
2204+
)
21952205

21962206
// Check if the function is evaluated in a windowed context.
21972207
ctx.windowSpec match {

sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out

Lines changed: 16 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@ CreateNamespace true
6969

7070

7171
-- !query
72-
CREATE TABLE s.tab(c1 INT) USING PARQUET
72+
CREATE TABLE s.tab(c1 INT) USING CSV
7373
-- !query analysis
7474
CreateDataSourceTableCommand `spark_catalog`.`s`.`tab`, false
7575

@@ -83,7 +83,7 @@ SetNamespaceCommand [s]
8383
-- !query
8484
INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1)
8585
-- !query analysis
86-
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]
86+
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]
8787
+- Project [cast(col1#x as int) AS c1#x]
8888
+- LocalRelation [col1#x]
8989

@@ -132,31 +132,31 @@ SELECT * FROM IDENTIFIER('tab')
132132
-- !query analysis
133133
Project [c1#x]
134134
+- SubqueryAlias spark_catalog.s.tab
135-
+- Relation spark_catalog.s.tab[c1#x] parquet
135+
+- Relation spark_catalog.s.tab[c1#x] csv
136136

137137

138138
-- !query
139139
SELECT * FROM IDENTIFIER('s.tab')
140140
-- !query analysis
141141
Project [c1#x]
142142
+- SubqueryAlias spark_catalog.s.tab
143-
+- Relation spark_catalog.s.tab[c1#x] parquet
143+
+- Relation spark_catalog.s.tab[c1#x] csv
144144

145145

146146
-- !query
147147
SELECT * FROM IDENTIFIER('`s`.`tab`')
148148
-- !query analysis
149149
Project [c1#x]
150150
+- SubqueryAlias spark_catalog.s.tab
151-
+- Relation spark_catalog.s.tab[c1#x] parquet
151+
+- Relation spark_catalog.s.tab[c1#x] csv
152152

153153

154154
-- !query
155155
SELECT * FROM IDENTIFIER('t' || 'a' || 'b')
156156
-- !query analysis
157157
Project [c1#x]
158158
+- SubqueryAlias spark_catalog.s.tab
159-
+- Relation spark_catalog.s.tab[c1#x] parquet
159+
+- Relation spark_catalog.s.tab[c1#x] csv
160160

161161

162162
-- !query
@@ -201,7 +201,7 @@ Project [id#xL]
201201

202202

203203
-- !query
204-
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet
204+
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV
205205
-- !query analysis
206206
CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false
207207

@@ -228,7 +228,7 @@ SetCatalogAndNamespace
228228

229229

230230
-- !query
231-
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet
231+
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV
232232
-- !query analysis
233233
CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false
234234

@@ -241,13 +241,13 @@ DropTable true, false
241241

242242

243243
-- !query
244-
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet
244+
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV
245245
-- !query analysis
246246
CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false
247247

248248

249249
-- !query
250-
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet
250+
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV
251251
-- !query analysis
252252
org.apache.spark.sql.AnalysisException
253253
{
@@ -265,15 +265,15 @@ CACHE TABLE IDENTIFIER('ta' || 'b')
265265
-- !query analysis
266266
CacheTable [tab], false, true
267267
+- SubqueryAlias spark_catalog.identifier_clauses.tab
268-
+- Relation spark_catalog.identifier_clauses.tab[c1#x] parquet
268+
+- Relation spark_catalog.identifier_clauses.tab[c1#x] csv
269269

270270

271271
-- !query
272272
UNCACHE TABLE IDENTIFIER('ta' || 'b')
273273
-- !query analysis
274274
UncacheTable false, true
275275
+- SubqueryAlias spark_catalog.identifier_clauses.tab
276-
+- Relation spark_catalog.identifier_clauses.tab[c1#x] parquet
276+
+- Relation spark_catalog.identifier_clauses.tab[c1#x] csv
277277

278278

279279
-- !query
@@ -298,15 +298,15 @@ DropNamespace false, false
298298

299299

300300
-- !query
301-
CREATE TABLE tab(c1 INT) USING parquet
301+
CREATE TABLE tab(c1 INT) USING CSV
302302
-- !query analysis
303303
CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false
304304

305305

306306
-- !query
307307
INSERT INTO tab VALUES (1)
308308
-- !query analysis
309-
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]
309+
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]
310310
+- Project [cast(col1#x as int) AS c1#x]
311311
+- LocalRelation [col1#x]
312312

@@ -316,7 +316,7 @@ SELECT c1 FROM tab
316316
-- !query analysis
317317
Project [c1#x]
318318
+- SubqueryAlias spark_catalog.default.tab
319-
+- Relation spark_catalog.default.tab[c1#x] parquet
319+
+- Relation spark_catalog.default.tab[c1#x] csv
320320

321321

322322
-- !query
@@ -1055,7 +1055,7 @@ org.apache.spark.sql.catalyst.parser.ParseException
10551055

10561056

10571057
-- !query
1058-
CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet
1058+
CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV
10591059
-- !query analysis
10601060
org.apache.spark.sql.catalyst.parser.ParseException
10611061
{

sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@ SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1);
1717

1818
-- Table references
1919
CREATE SCHEMA IF NOT EXISTS s;
20-
CREATE TABLE s.tab(c1 INT) USING PARQUET;
20+
CREATE TABLE s.tab(c1 INT) USING CSV;
2121
USE SCHEMA s;
2222

2323
INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1);
@@ -40,22 +40,22 @@ SELECT IDENTIFIER('abs')(-1);
4040
SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1);
4141

4242
-- Table DDL
43-
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet;
43+
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV;
4444
DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b');
4545

4646
CREATE SCHEMA identifier_clauses;
4747
USE identifier_clauses;
48-
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet;
48+
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV;
4949
DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab');
50-
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet;
51-
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet;
50+
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV;
51+
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV;
5252
CACHE TABLE IDENTIFIER('ta' || 'b');
5353
UNCACHE TABLE IDENTIFIER('ta' || 'b');
5454
DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b');
5555
USE default;
5656
DROP SCHEMA identifier_clauses;
5757

58-
CREATE TABLE tab(c1 INT) USING parquet;
58+
CREATE TABLE tab(c1 INT) USING CSV;
5959
INSERT INTO tab VALUES (1);
6060
SELECT c1 FROM tab;
6161
DESCRIBE IDENTIFIER('ta' || 'b');
@@ -135,7 +135,7 @@ SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS
135135
WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v);
136136
INSERT INTO tab(IDENTIFIER('c1')) VALUES(1);
137137
CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1);
138-
CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet;
138+
CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV;
139139

140140

141141

sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ struct<>
7272

7373

7474
-- !query
75-
CREATE TABLE s.tab(c1 INT) USING PARQUET
75+
CREATE TABLE s.tab(c1 INT) USING CSV
7676
-- !query schema
7777
struct<>
7878
-- !query output
@@ -221,7 +221,7 @@ struct<id:bigint>
221221

222222

223223
-- !query
224-
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING parquet
224+
CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV
225225
-- !query schema
226226
struct<>
227227
-- !query output
@@ -253,7 +253,7 @@ struct<>
253253

254254

255255
-- !query
256-
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING parquet
256+
CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV
257257
-- !query schema
258258
struct<>
259259
-- !query output
@@ -269,15 +269,15 @@ struct<>
269269

270270

271271
-- !query
272-
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet
272+
CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV
273273
-- !query schema
274274
struct<>
275275
-- !query output
276276

277277

278278

279279
-- !query
280-
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING parquet
280+
REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV
281281
-- !query schema
282282
struct<>
283283
-- !query output
@@ -333,7 +333,7 @@ struct<>
333333

334334

335335
-- !query
336-
CREATE TABLE tab(c1 INT) USING parquet
336+
CREATE TABLE tab(c1 INT) USING CSV
337337
-- !query schema
338338
struct<>
339339
-- !query output
@@ -1209,7 +1209,7 @@ org.apache.spark.sql.catalyst.parser.ParseException
12091209

12101210

12111211
-- !query
1212-
CREATE TABLE tab(IDENTIFIER('c1') INT) USING parquet
1212+
CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV
12131213
-- !query schema
12141214
struct<>
12151215
-- !query output

0 commit comments

Comments
 (0)