/
AstBuilder.scala
3714 lines (3398 loc) · 131 KB
/
AstBuilder.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.catalyst.parser
import java.util.Locale
import javax.xml.bind.DatatypeConverter
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import org.antlr.v4.runtime.{ParserRuleContext, Token}
import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, FunctionResource, FunctionResourceType}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last}
import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp}
import org.apache.spark.sql.catalyst.util.IntervalUtils
import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit
import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog}
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
import org.apache.spark.util.random.RandomSampler
/**
* The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
* TableIdentifier.
*/
class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
import ParserUtils._
protected def conf: SQLConf = SQLConf.get
protected def typedVisit[T](ctx: ParseTree): T = {
ctx.accept(this).asInstanceOf[T]
}
/**
* Override the default behavior for all visit methods. This will only return a non-null result
* when the context has only one child. This is done because there is no generic method to
* combine the results of the context children. In all other cases null is returned.
*/
override def visitChildren(node: RuleNode): AnyRef = {
if (node.getChildCount == 1) {
node.getChild(0).accept(this)
} else {
null
}
}
override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
visit(ctx.statement).asInstanceOf[LogicalPlan]
}
override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
visitNamedExpression(ctx.namedExpression)
}
override def visitSingleTableIdentifier(
ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
visitTableIdentifier(ctx.tableIdentifier)
}
override def visitSingleFunctionIdentifier(
ctx: SingleFunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) {
visitFunctionIdentifier(ctx.functionIdentifier)
}
override def visitSingleMultipartIdentifier(
ctx: SingleMultipartIdentifierContext): Seq[String] = withOrigin(ctx) {
visitMultipartIdentifier(ctx.multipartIdentifier)
}
override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
visitSparkDataType(ctx.dataType)
}
override def visitSingleTableSchema(ctx: SingleTableSchemaContext): StructType = {
withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList)))
}
def parseRawDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
typedVisit[DataType](ctx.dataType())
}
/* ********************************************************************************************
* Plan parsing
* ******************************************************************************************** */
protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
/**
* Create a top-level plan with Common Table Expressions.
*/
override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
val query = plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses)
// Apply CTEs
query.optionalMap(ctx.ctes)(withCTE)
}
override def visitDmlStatement(ctx: DmlStatementContext): AnyRef = withOrigin(ctx) {
val dmlStmt = plan(ctx.dmlStatementNoWith)
// Apply CTEs
dmlStmt.optionalMap(ctx.ctes)(withCTE)
}
private def withCTE(ctx: CtesContext, plan: LogicalPlan): LogicalPlan = {
val ctes = ctx.namedQuery.asScala.map { nCtx =>
val namedQuery = visitNamedQuery(nCtx)
(namedQuery.alias, namedQuery)
}
// Check for duplicate names.
val duplicates = ctes.groupBy(_._1).filter(_._2.size > 1).keys
if (duplicates.nonEmpty) {
throw new ParseException(
s"CTE definition can't have duplicate names: ${duplicates.mkString("'", "', '", "'")}.",
ctx)
}
With(plan, ctes.toSeq)
}
/**
* Create a logical query plan for a hive-style FROM statement body.
*/
private def withFromStatementBody(
ctx: FromStatementBodyContext, plan: LogicalPlan): LogicalPlan = withOrigin(ctx) {
// two cases for transforms and selects
if (ctx.transformClause != null) {
withTransformQuerySpecification(
ctx,
ctx.transformClause,
ctx.whereClause,
plan
)
} else {
withSelectQuerySpecification(
ctx,
ctx.selectClause,
ctx.lateralView,
ctx.whereClause,
ctx.aggregationClause,
ctx.havingClause,
ctx.windowClause,
plan
)
}
}
override def visitFromStatement(ctx: FromStatementContext): LogicalPlan = withOrigin(ctx) {
val from = visitFromClause(ctx.fromClause)
val selects = ctx.fromStatementBody.asScala.map { body =>
withFromStatementBody(body, from).
// Add organization statements.
optionalMap(body.queryOrganization)(withQueryResultClauses)
}
// If there are multiple SELECT just UNION them together into one query.
if (selects.length == 1) {
selects.head
} else {
Union(selects.toSeq)
}
}
/**
* Create a named logical plan.
*
* This is only used for Common Table Expressions.
*/
override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
val subQuery: LogicalPlan = plan(ctx.query).optionalMap(ctx.columnAliases)(
(columnAliases, plan) =>
UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan)
)
SubqueryAlias(ctx.name.getText, subQuery)
}
/**
* Create a logical plan which allows for multiple inserts using one 'from' statement. These
* queries have the following SQL form:
* {{{
* [WITH cte...]?
* FROM src
* [INSERT INTO tbl1 SELECT *]+
* }}}
* For example:
* {{{
* FROM db.tbl1 A
* INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
* INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
* }}}
* This (Hive) feature cannot be combined with set-operators.
*/
override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
val from = visitFromClause(ctx.fromClause)
// Build the insert clauses.
val inserts = ctx.multiInsertQueryBody.asScala.map { body =>
withInsertInto(body.insertInto,
withFromStatementBody(body.fromStatementBody, from).
optionalMap(body.fromStatementBody.queryOrganization)(withQueryResultClauses))
}
// If there are multiple INSERTS just UNION them together into one query.
if (inserts.length == 1) {
inserts.head
} else {
Union(inserts.toSeq)
}
}
/**
* Create a logical plan for a regular (single-insert) query.
*/
override def visitSingleInsertQuery(
ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
withInsertInto(
ctx.insertInto(),
plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses))
}
/**
* Parameters used for writing query to a table:
* (multipartIdentifier, partitionKeys, ifPartitionNotExists).
*/
type InsertTableParams = (Seq[String], Map[String, Option[String]], Boolean)
/**
* Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider).
*/
type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String])
/**
* Add an
* {{{
* INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]?
* INSERT INTO [TABLE] tableIdentifier [partitionSpec]
* INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat]
* INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList]
* }}}
* operation to logical plan
*/
private def withInsertInto(
ctx: InsertIntoContext,
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
ctx match {
case table: InsertIntoTableContext =>
val (tableIdent, partition, ifPartitionNotExists) = visitInsertIntoTable(table)
InsertIntoStatement(
UnresolvedRelation(tableIdent),
partition,
query,
overwrite = false,
ifPartitionNotExists)
case table: InsertOverwriteTableContext =>
val (tableIdent, partition, ifPartitionNotExists) = visitInsertOverwriteTable(table)
InsertIntoStatement(
UnresolvedRelation(tableIdent),
partition,
query,
overwrite = true,
ifPartitionNotExists)
case dir: InsertOverwriteDirContext =>
val (isLocal, storage, provider) = visitInsertOverwriteDir(dir)
InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
case hiveDir: InsertOverwriteHiveDirContext =>
val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir)
InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
case _ =>
throw new ParseException("Invalid InsertIntoContext", ctx)
}
}
/**
* Add an INSERT INTO TABLE operation to the logical plan.
*/
override def visitInsertIntoTable(
ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) {
val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
if (ctx.EXISTS != null) {
operationNotAllowed("INSERT INTO ... IF NOT EXISTS", ctx)
}
(tableIdent, partitionKeys, false)
}
/**
* Add an INSERT OVERWRITE TABLE operation to the logical plan.
*/
override def visitInsertOverwriteTable(
ctx: InsertOverwriteTableContext): InsertTableParams = withOrigin(ctx) {
assert(ctx.OVERWRITE() != null)
val tableIdent = visitMultipartIdentifier(ctx.multipartIdentifier)
val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
val dynamicPartitionKeys: Map[String, Option[String]] = partitionKeys.filter(_._2.isEmpty)
if (ctx.EXISTS != null && dynamicPartitionKeys.nonEmpty) {
operationNotAllowed("IF NOT EXISTS with dynamic partitions: " +
dynamicPartitionKeys.keys.mkString(", "), ctx)
}
(tableIdent, partitionKeys, ctx.EXISTS() != null)
}
/**
* Write to a directory, returning a [[InsertIntoDir]] logical plan.
*/
override def visitInsertOverwriteDir(
ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) {
throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
}
/**
* Write to a directory, returning a [[InsertIntoDir]] logical plan.
*/
override def visitInsertOverwriteHiveDir(
ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) {
throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
}
private def getTableAliasWithoutColumnAlias(
ctx: TableAliasContext, op: String): Option[String] = {
if (ctx == null) {
None
} else {
val ident = ctx.strictIdentifier()
if (ctx.identifierList() != null) {
throw new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList())
}
if (ident != null) Some(ident.getText) else None
}
}
override def visitDeleteFromTable(
ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) {
val table = UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier()))
val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE")
val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table)
val predicate = if (ctx.whereClause() != null) {
Some(expression(ctx.whereClause().booleanExpression()))
} else {
None
}
DeleteFromTable(aliasedTable, predicate)
}
override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) {
val table = UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier()))
val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE")
val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table)
val assignments = withAssignments(ctx.setClause().assignmentList())
val predicate = if (ctx.whereClause() != null) {
Some(expression(ctx.whereClause().booleanExpression()))
} else {
None
}
UpdateTable(aliasedTable, assignments, predicate)
}
private def withAssignments(assignCtx: SqlBaseParser.AssignmentListContext): Seq[Assignment] =
withOrigin(assignCtx) {
assignCtx.assignment().asScala.map { assign =>
Assignment(UnresolvedAttribute(visitMultipartIdentifier(assign.key)),
expression(assign.value))
}.toSeq
}
override def visitMergeIntoTable(ctx: MergeIntoTableContext): LogicalPlan = withOrigin(ctx) {
val targetTable = UnresolvedRelation(visitMultipartIdentifier(ctx.target))
val targetTableAlias = getTableAliasWithoutColumnAlias(ctx.targetAlias, "MERGE")
val aliasedTarget = targetTableAlias.map(SubqueryAlias(_, targetTable)).getOrElse(targetTable)
val sourceTableOrQuery = if (ctx.source != null) {
UnresolvedRelation(visitMultipartIdentifier(ctx.source))
} else if (ctx.sourceQuery != null) {
visitQuery(ctx.sourceQuery)
} else {
throw new ParseException("Empty source for merge: you should specify a source" +
" table/subquery in merge.", ctx.source)
}
val sourceTableAlias = getTableAliasWithoutColumnAlias(ctx.sourceAlias, "MERGE")
val aliasedSource =
sourceTableAlias.map(SubqueryAlias(_, sourceTableOrQuery)).getOrElse(sourceTableOrQuery)
val mergeCondition = expression(ctx.mergeCondition)
val matchedActions = ctx.matchedClause().asScala.map {
clause => {
if (clause.matchedAction().DELETE() != null) {
DeleteAction(Option(clause.matchedCond).map(expression))
} else if (clause.matchedAction().UPDATE() != null) {
val condition = Option(clause.matchedCond).map(expression)
if (clause.matchedAction().ASTERISK() != null) {
UpdateAction(condition, Seq())
} else {
UpdateAction(condition, withAssignments(clause.matchedAction().assignmentList()))
}
} else {
// It should not be here.
throw new ParseException(
s"Unrecognized matched action: ${clause.matchedAction().getText}",
clause.matchedAction())
}
}
}
val notMatchedActions = ctx.notMatchedClause().asScala.map {
clause => {
if (clause.notMatchedAction().INSERT() != null) {
val condition = Option(clause.notMatchedCond).map(expression)
if (clause.notMatchedAction().ASTERISK() != null) {
InsertAction(condition, Seq())
} else {
val columns = clause.notMatchedAction().columns.multipartIdentifier()
.asScala.map(attr => UnresolvedAttribute(visitMultipartIdentifier(attr)))
val values = clause.notMatchedAction().expression().asScala.map(expression)
if (columns.size != values.size) {
throw new ParseException("The number of inserted values cannot match the fields.",
clause.notMatchedAction())
}
InsertAction(condition, columns.zip(values).map(kv => Assignment(kv._1, kv._2)).toSeq)
}
} else {
// It should not be here.
throw new ParseException(
s"Unrecognized not matched action: ${clause.notMatchedAction().getText}",
clause.notMatchedAction())
}
}
}
if (matchedActions.isEmpty && notMatchedActions.isEmpty) {
throw new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx)
}
// children being empty means that the condition is not set
val matchedActionSize = matchedActions.length
if (matchedActionSize >= 2 && !matchedActions.init.forall(_.condition.nonEmpty)) {
throw new ParseException("When there are more than one MATCHED clauses in a MERGE " +
"statement, only the last MATCHED clause can omit the condition.", ctx)
}
val notMatchedActionSize = notMatchedActions.length
if (notMatchedActionSize >= 2 && !notMatchedActions.init.forall(_.condition.nonEmpty)) {
throw new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " +
"statement, only the last NOT MATCHED clause can omit the condition.", ctx)
}
MergeIntoTable(
aliasedTarget,
aliasedSource,
mergeCondition,
matchedActions.toSeq,
notMatchedActions.toSeq)
}
/**
* Create a partition specification map.
*/
override def visitPartitionSpec(
ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
val parts = ctx.partitionVal.asScala.map { pVal =>
val name = pVal.identifier.getText
val value = Option(pVal.constant).map(visitStringConstant)
name -> value
}
// Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values
// in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for
// partition columns will be done in analyzer.
checkDuplicateKeys(parts.toSeq, ctx)
parts.toMap
}
/**
* Create a partition specification map without optional values.
*/
protected def visitNonOptionalPartitionSpec(
ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
visitPartitionSpec(ctx).map {
case (key, None) => throw new ParseException(s"Found an empty partition key '$key'.", ctx)
case (key, Some(value)) => key -> value
}
}
/**
* Convert a constant of any type into a string. This is typically used in DDL commands, and its
* main purpose is to prevent slight differences due to back to back conversions i.e.:
* String -> Literal -> String.
*/
protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
ctx match {
case s: StringLiteralContext => createString(s)
case o => o.getText
}
}
/**
* Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
* clauses determine the shape (ordering/partitioning/rows) of the query result.
*/
private def withQueryResultClauses(
ctx: QueryOrganizationContext,
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
import ctx._
// Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
val withOrder = if (
!order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
// ORDER BY ...
Sort(order.asScala.map(visitSortItem).toSeq, global = true, query)
} else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
// SORT BY ...
Sort(sort.asScala.map(visitSortItem).toSeq, global = false, query)
} else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
// DISTRIBUTE BY ...
withRepartitionByExpression(ctx, expressionList(distributeBy), query)
} else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
// SORT BY ... DISTRIBUTE BY ...
Sort(
sort.asScala.map(visitSortItem).toSeq,
global = false,
withRepartitionByExpression(ctx, expressionList(distributeBy), query))
} else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
// CLUSTER BY ...
val expressions = expressionList(clusterBy)
Sort(
expressions.map(SortOrder(_, Ascending)),
global = false,
withRepartitionByExpression(ctx, expressions, query))
} else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
// [EMPTY]
query
} else {
throw new ParseException(
"Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
}
// WINDOWS
val withWindow = withOrder.optionalMap(windowClause)(withWindowClause)
// LIMIT
// - LIMIT ALL is the same as omitting the LIMIT clause
withWindow.optional(limit) {
Limit(typedVisit(limit), withWindow)
}
}
/**
* Create a clause for DISTRIBUTE BY.
*/
protected def withRepartitionByExpression(
ctx: QueryOrganizationContext,
expressions: Seq[Expression],
query: LogicalPlan): LogicalPlan = {
throw new ParseException("DISTRIBUTE BY is not supported", ctx)
}
override def visitTransformQuerySpecification(
ctx: TransformQuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
val from = OneRowRelation().optional(ctx.fromClause) {
visitFromClause(ctx.fromClause)
}
withTransformQuerySpecification(ctx, ctx.transformClause, ctx.whereClause, from)
}
override def visitRegularQuerySpecification(
ctx: RegularQuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
val from = OneRowRelation().optional(ctx.fromClause) {
visitFromClause(ctx.fromClause)
}
withSelectQuerySpecification(
ctx,
ctx.selectClause,
ctx.lateralView,
ctx.whereClause,
ctx.aggregationClause,
ctx.havingClause,
ctx.windowClause,
from
)
}
override def visitNamedExpressionSeq(
ctx: NamedExpressionSeqContext): Seq[Expression] = {
Option(ctx).toSeq
.flatMap(_.namedExpression.asScala)
.map(typedVisit[Expression])
}
/**
* Create a logical plan using a having clause.
*/
private def withHavingClause(
ctx: HavingClauseContext, plan: LogicalPlan): LogicalPlan = {
// Note that we add a cast to non-predicate expressions. If the expression itself is
// already boolean, the optimizer will get rid of the unnecessary cast.
val predicate = expression(ctx.booleanExpression) match {
case p: Predicate => p
case e => Cast(e, BooleanType)
}
UnresolvedHaving(predicate, plan)
}
/**
* Create a logical plan using a where clause.
*/
private def withWhereClause(ctx: WhereClauseContext, plan: LogicalPlan): LogicalPlan = {
Filter(expression(ctx.booleanExpression), plan)
}
/**
* Add a hive-style transform (SELECT TRANSFORM/MAP/REDUCE) query specification to a logical plan.
*/
private def withTransformQuerySpecification(
ctx: ParserRuleContext,
transformClause: TransformClauseContext,
whereClause: WhereClauseContext,
relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
// Add where.
val withFilter = relation.optionalMap(whereClause)(withWhereClause)
// Create the transform.
val expressions = visitNamedExpressionSeq(transformClause.namedExpressionSeq)
// Create the attributes.
val (attributes, schemaLess) = if (transformClause.colTypeList != null) {
// Typed return columns.
(createSchema(transformClause.colTypeList).toAttributes, false)
} else if (transformClause.identifierSeq != null) {
// Untyped return columns.
val attrs = visitIdentifierSeq(transformClause.identifierSeq).map { name =>
AttributeReference(name, StringType, nullable = true)()
}
(attrs, false)
} else {
(Seq(AttributeReference("key", StringType)(),
AttributeReference("value", StringType)()), true)
}
// Create the transform.
ScriptTransformation(
expressions,
string(transformClause.script),
attributes,
withFilter,
withScriptIOSchema(
ctx,
transformClause.inRowFormat,
transformClause.recordWriter,
transformClause.outRowFormat,
transformClause.recordReader,
schemaLess
)
)
}
/**
* Add a regular (SELECT) query specification to a logical plan. The query specification
* is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT),
* aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
*
* Note that query hints are ignored (both by the parser and the builder).
*/
private def withSelectQuerySpecification(
ctx: ParserRuleContext,
selectClause: SelectClauseContext,
lateralView: java.util.List[LateralViewContext],
whereClause: WhereClauseContext,
aggregationClause: AggregationClauseContext,
havingClause: HavingClauseContext,
windowClause: WindowClauseContext,
relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
// Add lateral views.
val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate)
// Add where.
val withFilter = withLateralView.optionalMap(whereClause)(withWhereClause)
val expressions = visitNamedExpressionSeq(selectClause.namedExpressionSeq)
// Add aggregation or a project.
val namedExpressions = expressions.map {
case e: NamedExpression => e
case e: Expression => UnresolvedAlias(e)
}
def createProject() = if (namedExpressions.nonEmpty) {
Project(namedExpressions, withFilter)
} else {
withFilter
}
val withProject = if (aggregationClause == null && havingClause != null) {
if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) {
// If the legacy conf is set, treat HAVING without GROUP BY as WHERE.
withHavingClause(havingClause, createProject())
} else {
// According to SQL standard, HAVING without GROUP BY means global aggregate.
withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter))
}
} else if (aggregationClause != null) {
val aggregate = withAggregationClause(aggregationClause, namedExpressions, withFilter)
aggregate.optionalMap(havingClause)(withHavingClause)
} else {
// When hitting this branch, `having` must be null.
createProject()
}
// Distinct
val withDistinct = if (
selectClause.setQuantifier() != null &&
selectClause.setQuantifier().DISTINCT() != null) {
Distinct(withProject)
} else {
withProject
}
// Window
val withWindow = withDistinct.optionalMap(windowClause)(withWindowClause)
// Hint
selectClause.hints.asScala.foldRight(withWindow)(withHints)
}
/**
* Create a (Hive based) [[ScriptInputOutputSchema]].
*/
protected def withScriptIOSchema(
ctx: ParserRuleContext,
inRowFormat: RowFormatContext,
recordWriter: Token,
outRowFormat: RowFormatContext,
recordReader: Token,
schemaLess: Boolean): ScriptInputOutputSchema = {
throw new ParseException("Script Transform is not supported", ctx)
}
/**
* Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
* separated) relations here, these get converted into a single plan by condition-less inner join.
*/
override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) =>
val right = plan(relation.relationPrimary)
val join = right.optionalMap(left)(Join(_, _, Inner, None, JoinHint.NONE))
withJoinRelations(join, relation)
}
if (ctx.pivotClause() != null) {
if (!ctx.lateralView.isEmpty) {
throw new ParseException("LATERAL cannot be used together with PIVOT in FROM clause", ctx)
}
withPivot(ctx.pivotClause, from)
} else {
ctx.lateralView.asScala.foldLeft(from)(withGenerate)
}
}
/**
* Connect two queries by a Set operator.
*
* Supported Set operators are:
* - UNION [ DISTINCT | ALL ]
* - EXCEPT [ DISTINCT | ALL ]
* - MINUS [ DISTINCT | ALL ]
* - INTERSECT [DISTINCT | ALL]
*/
override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
val left = plan(ctx.left)
val right = plan(ctx.right)
val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
ctx.operator.getType match {
case SqlBaseParser.UNION if all =>
Union(left, right)
case SqlBaseParser.UNION =>
Distinct(Union(left, right))
case SqlBaseParser.INTERSECT if all =>
Intersect(left, right, isAll = true)
case SqlBaseParser.INTERSECT =>
Intersect(left, right, isAll = false)
case SqlBaseParser.EXCEPT if all =>
Except(left, right, isAll = true)
case SqlBaseParser.EXCEPT =>
Except(left, right, isAll = false)
case SqlBaseParser.SETMINUS if all =>
Except(left, right, isAll = true)
case SqlBaseParser.SETMINUS =>
Except(left, right, isAll = false)
}
}
/**
* Add a [[WithWindowDefinition]] operator to a logical plan.
*/
private def withWindowClause(
ctx: WindowClauseContext,
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
// Collect all window specifications defined in the WINDOW clause.
val baseWindowMap = ctx.namedWindow.asScala.map {
wCtx =>
(wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec))
}.toMap
// Handle cases like
// window w1 as (partition by p_mfgr order by p_name
// range between 2 preceding and 2 following),
// w2 as w1
val windowMapView = baseWindowMap.mapValues {
case WindowSpecReference(name) =>
baseWindowMap.get(name) match {
case Some(spec: WindowSpecDefinition) =>
spec
case Some(ref) =>
throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
case None =>
throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
}
case spec: WindowSpecDefinition => spec
}
// Note that mapValues creates a view instead of materialized map. We force materialization by
// mapping over identity.
WithWindowDefinition(windowMapView.map(identity).toMap, query)
}
/**
* Add an [[Aggregate]] or [[GroupingSets]] to a logical plan.
*/
private def withAggregationClause(
ctx: AggregationClauseContext,
selectExpressions: Seq[NamedExpression],
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
val groupByExpressions = expressionList(ctx.groupingExpressions)
if (ctx.GROUPING != null) {
// GROUP BY .... GROUPING SETS (...)
val selectedGroupByExprs =
ctx.groupingSet.asScala.map(_.expression.asScala.map(e => expression(e)).toSeq)
GroupingSets(selectedGroupByExprs.toSeq, groupByExpressions, query, selectExpressions)
} else {
// GROUP BY .... (WITH CUBE | WITH ROLLUP)?
val mappedGroupByExpressions = if (ctx.CUBE != null) {
Seq(Cube(groupByExpressions))
} else if (ctx.ROLLUP != null) {
Seq(Rollup(groupByExpressions))
} else {
groupByExpressions
}
Aggregate(mappedGroupByExpressions, selectExpressions, query)
}
}
/**
* Add [[UnresolvedHint]]s to a logical plan.
*/
private def withHints(
ctx: HintContext,
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
var plan = query
ctx.hintStatements.asScala.reverse.foreach { stmt =>
plan = UnresolvedHint(stmt.hintName.getText,
stmt.parameters.asScala.map(expression).toSeq, plan)
}
plan
}
/**
* Add a [[Pivot]] to a logical plan.
*/
private def withPivot(
ctx: PivotClauseContext,
query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
val aggregates = Option(ctx.aggregates).toSeq
.flatMap(_.namedExpression.asScala)
.map(typedVisit[Expression])
val pivotColumn = if (ctx.pivotColumn.identifiers.size == 1) {
UnresolvedAttribute.quoted(ctx.pivotColumn.identifier.getText)
} else {
CreateStruct(
ctx.pivotColumn.identifiers.asScala.map(
identifier => UnresolvedAttribute.quoted(identifier.getText)).toSeq)
}
val pivotValues = ctx.pivotValues.asScala.map(visitPivotValue)
Pivot(None, pivotColumn, pivotValues.toSeq, aggregates, query)
}
/**
* Create a Pivot column value with or without an alias.
*/
override def visitPivotValue(ctx: PivotValueContext): Expression = withOrigin(ctx) {
val e = expression(ctx.expression)
if (ctx.identifier != null) {
Alias(e, ctx.identifier.getText)()
} else {
e
}
}
/**
* Add a [[Generate]] (Lateral View) to a logical plan.
*/
private def withGenerate(
query: LogicalPlan,
ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
val expressions = expressionList(ctx.expression)
Generate(
UnresolvedGenerator(visitFunctionName(ctx.qualifiedName), expressions),
unrequiredChildIndex = Nil,
outer = ctx.OUTER != null,
// scalastyle:off caselocale
Some(ctx.tblName.getText.toLowerCase),
// scalastyle:on caselocale
ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply).toSeq,
query)
}
/**
* Create a single relation referenced in a FROM clause. This method is used when a part of the
* join condition is nested, for example:
* {{{
* select * from t1 join (t2 cross join t3) on col1 = col2
* }}}
*/
override def visitRelation(ctx: RelationContext): LogicalPlan = withOrigin(ctx) {
withJoinRelations(plan(ctx.relationPrimary), ctx)
}
/**
* Join one more [[LogicalPlan]]s to the current logical plan.
*/
private def withJoinRelations(base: LogicalPlan, ctx: RelationContext): LogicalPlan = {
ctx.joinRelation.asScala.foldLeft(base) { (left, join) =>
withOrigin(join) {
val baseJoinType = join.joinType match {
case null => Inner
case jt if jt.CROSS != null => Cross
case jt if jt.FULL != null => FullOuter
case jt if jt.SEMI != null => LeftSemi
case jt if jt.ANTI != null => LeftAnti
case jt if jt.LEFT != null => LeftOuter
case jt if jt.RIGHT != null => RightOuter
case _ => Inner
}
// Resolve the join type and join condition
val (joinType, condition) = Option(join.joinCriteria) match {
case Some(c) if c.USING != null =>
(UsingJoin(baseJoinType, visitIdentifierList(c.identifierList)), None)
case Some(c) if c.booleanExpression != null =>
(baseJoinType, Option(expression(c.booleanExpression)))
case None if join.NATURAL != null =>
if (baseJoinType == Cross) {
throw new ParseException("NATURAL CROSS JOIN is not supported", ctx)
}
(NaturalJoin(baseJoinType), None)
case None =>
(baseJoinType, None)
}
Join(left, plan(join.right), joinType, condition, JoinHint.NONE)
}
}
}
/**
* Add a [[Sample]] to a logical plan.
*
* This currently supports the following sampling methods:
* - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
* - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
* are defined as a number between 0 and 100.
* - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
*/
private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
// Create a sampled plan if we need one.
def sample(fraction: Double): Sample = {
// The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
// function takes X PERCENT as the input and the range of X is [0, 100], we need to
// adjust the fraction.
val eps = RandomSampler.roundingEpsilon