/
CalciteSqlParser.java
944 lines (884 loc) · 37.2 KB
/
CalciteSqlParser.java
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
/**
* 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.pinot.sql.parsers;
import com.google.common.annotations.VisibleForTesting;
import java.io.StringReader;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.calcite.avatica.util.Casing;
import org.apache.calcite.sql.SqlBasicCall;
import org.apache.calcite.sql.SqlDataTypeSpec;
import org.apache.calcite.sql.SqlExplain;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlNodeList;
import org.apache.calcite.sql.SqlNumericLiteral;
import org.apache.calcite.sql.SqlOrderBy;
import org.apache.calcite.sql.SqlSelect;
import org.apache.calcite.sql.SqlSelectKeyword;
import org.apache.calcite.sql.fun.SqlBetweenOperator;
import org.apache.calcite.sql.fun.SqlCase;
import org.apache.calcite.sql.fun.SqlLikeOperator;
import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.calcite.sql.validate.SqlConformanceEnum;
import org.apache.commons.collections.CollectionUtils;
import org.apache.pinot.common.request.DataSource;
import org.apache.pinot.common.request.Expression;
import org.apache.pinot.common.request.ExpressionType;
import org.apache.pinot.common.request.Function;
import org.apache.pinot.common.request.Identifier;
import org.apache.pinot.common.request.PinotQuery;
import org.apache.pinot.common.utils.request.RequestUtils;
import org.apache.pinot.segment.spi.AggregationFunctionType;
import org.apache.pinot.spi.utils.Pairs;
import org.apache.pinot.sql.FilterKind;
import org.apache.pinot.sql.parsers.parser.SqlInsertFromFile;
import org.apache.pinot.sql.parsers.parser.SqlParserImpl;
import org.apache.pinot.sql.parsers.rewriter.QueryRewriter;
import org.apache.pinot.sql.parsers.rewriter.QueryRewriterFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class CalciteSqlParser {
private CalciteSqlParser() {
}
public static final List<QueryRewriter> QUERY_REWRITERS = new ArrayList<>(QueryRewriterFactory.getQueryRewriters());
private static final Logger LOGGER = LoggerFactory.getLogger(CalciteSqlParser.class);
// To Keep the backward compatibility with 'OPTION' Functionality in PQL, which is used to
// provide more hints for query processing.
//
// PQL syntax is: `OPTION (<key> = <value>)`
//
// Multiple OPTIONs is also supported by:
// either
// `OPTION (<k1> = <v1>, <k2> = <v2>, <k3> = <v3>)`
// or
// `OPTION (<k1> = <v1>) OPTION (<k2> = <v2>) OPTION (<k3> = <v3>)`
// TODO: move to use parser syntax extension: `OPTION` `(` `<key>` = `<value>` [, `<key>` = `<value>`]* `)`
private static final Pattern OPTIONS_REGEX_PATTEN =
Pattern.compile("option\\s*\\(([^\\)]+)\\)", Pattern.CASE_INSENSITIVE);
/**
* Checks for the presence of semicolon in the sql query and modifies the query accordingly
*
* @param sql sql query
* @return sql query without semicolons
*
*/
private static String removeTerminatingSemicolon(String sql) {
// trim all the leading and trailing whitespaces
sql = sql.trim();
int sqlLength = sql.length();
// Terminate the semicolon only if the last character of the query is semicolon
if (sql.charAt(sqlLength - 1) == ';') {
return sql.substring(0, sqlLength - 1);
}
return sql;
}
public static SqlNodeAndOptions compileToSqlNodeAndOptions(String sql)
throws Exception {
// Remove the comments from the query
sql = removeComments(sql);
// Remove the terminating semicolon from the query
sql = removeTerminatingSemicolon(sql);
// Extract OPTION statements from sql as Calcite Parser doesn't parse it.
List<String> options = extractOptionsFromSql(sql);
if (!options.isEmpty()) {
sql = removeOptionsFromSql(sql);
}
try (StringReader inStream = new StringReader(sql)) {
SqlParserImpl sqlParser = newSqlParser(inStream);
return new SqlNodeAndOptions(sqlParser.parseSqlStmtEof(), options);
} catch (Throwable e) {
throw new SqlCompilationException("Caught exception while parsing query: " + sql, e);
}
}
public static PinotSqlType extractSqlType(SqlNode sqlNode) {
switch (sqlNode.getKind()) {
case OTHER_DDL:
if (sqlNode instanceof SqlInsertFromFile) {
return PinotSqlType.DML;
}
throw new SqlCompilationException("Unsupported SqlNode type - " + sqlNode.getKind());
default:
return PinotSqlType.DQL;
}
}
public static PinotQuery compileToPinotQuery(String sql)
throws SqlCompilationException {
// Remove the comments from the query
sql = removeComments(sql);
// Remove the terminating semicolon from the query
sql = removeTerminatingSemicolon(sql);
// Extract OPTION statements from sql as Calcite Parser doesn't parse it.
List<String> options = extractOptionsFromSql(sql);
if (!options.isEmpty()) {
sql = removeOptionsFromSql(sql);
}
SqlNode sqlNode;
try (StringReader inStream = new StringReader(sql)) {
SqlParserImpl sqlParser = newSqlParser(inStream);
sqlNode = sqlParser.parseSqlStmtEof();
} catch (Throwable e) {
throw new SqlCompilationException("Caught exception while parsing query: " + sql, e);
}
// Compile Sql without OPTION statements.
PinotQuery pinotQuery = compileSqlNodeToPinotQuery(sqlNode);
// Set Option statements to PinotQuery.
setOptions(pinotQuery, options);
return pinotQuery;
}
static void validate(PinotQuery pinotQuery)
throws SqlCompilationException {
validateGroupByClause(pinotQuery);
validateDistinctQuery(pinotQuery);
}
private static void validateGroupByClause(PinotQuery pinotQuery)
throws SqlCompilationException {
boolean hasGroupByClause = pinotQuery.getGroupByList() != null;
Set<Expression> groupByExprs = hasGroupByClause ? new HashSet<>(pinotQuery.getGroupByList()) : null;
int aggregateExprCount = 0;
for (Expression selectExpression : pinotQuery.getSelectList()) {
if (isAggregateExpression(selectExpression)) {
aggregateExprCount++;
} else if (hasGroupByClause && expressionOutsideGroupByList(selectExpression, groupByExprs)) {
throw new SqlCompilationException(
"'" + RequestUtils.prettyPrint(selectExpression) + "' should appear in GROUP BY clause.");
}
}
// block mixture of aggregate and non-aggregate expression when group by is absent
int nonAggregateExprCount = pinotQuery.getSelectListSize() - aggregateExprCount;
if (!hasGroupByClause && aggregateExprCount > 0 && nonAggregateExprCount > 0) {
throw new SqlCompilationException("Columns and Aggregate functions can't co-exist without GROUP BY clause");
}
// Sanity check on group by clause shouldn't contain aggregate expression.
if (hasGroupByClause) {
for (Expression groupByExpression : pinotQuery.getGroupByList()) {
if (isAggregateExpression(groupByExpression)) {
throw new SqlCompilationException("Aggregate expression '" + RequestUtils.prettyPrint(groupByExpression)
+ "' is not allowed in GROUP BY clause.");
}
}
}
}
/*
* Validate DISTINCT queries:
* - No GROUP-BY clause
* - LIMIT must be positive
* - ORDER-BY columns (if exist) should be included in the DISTINCT columns
*/
private static void validateDistinctQuery(PinotQuery pinotQuery)
throws SqlCompilationException {
List<Expression> selectList = pinotQuery.getSelectList();
if (selectList.size() == 1) {
Function function = selectList.get(0).getFunctionCall();
if (function != null && function.getOperator().equals("distinct")) {
if (CollectionUtils.isNotEmpty(pinotQuery.getGroupByList())) {
// TODO: Explore if DISTINCT should be supported with GROUP BY
throw new IllegalStateException("DISTINCT with GROUP BY is currently not supported");
}
if (pinotQuery.getLimit() <= 0) {
// TODO: Consider changing it to SELECTION query for LIMIT 0
throw new IllegalStateException("DISTINCT must have positive LIMIT");
}
List<Expression> orderByList = pinotQuery.getOrderByList();
if (orderByList != null) {
List<Expression> distinctExpressions = getAliasLeftExpressionsFromDistinctExpression(function);
for (Expression orderByExpression : orderByList) {
// NOTE: Order-by is always a Function with the ordering of the Expression
if (!distinctExpressions.contains(orderByExpression.getFunctionCall().getOperands().get(0))) {
throw new IllegalStateException("ORDER-BY columns should be included in the DISTINCT columns");
}
}
}
}
}
}
private static List<Expression> getAliasLeftExpressionsFromDistinctExpression(Function function) {
List<Expression> operands = function.getOperands();
List<Expression> expressions = new ArrayList<>(operands.size());
for (Expression operand : operands) {
if (isAsFunction(operand)) {
expressions.add(operand.getFunctionCall().getOperands().get(0));
} else {
expressions.add(operand);
}
}
return expressions;
}
/**
* Check recursively if an expression contains any reference not appearing in the GROUP BY clause.
*/
private static boolean expressionOutsideGroupByList(Expression expr, Set<Expression> groupByExprs) {
// return early for Literal, Aggregate and if we have an exact match
if (expr.getType() == ExpressionType.LITERAL || isAggregateExpression(expr) || groupByExprs.contains(expr)) {
return false;
}
Function function = expr.getFunctionCall();
// function expression
if (function != null) {
// for Alias function, check the actual value
if (function.getOperator().equals("as")) {
return expressionOutsideGroupByList(function.getOperands().get(0), groupByExprs);
}
// Expression is invalid if any of its children is invalid
return function.getOperands().stream().anyMatch(e -> expressionOutsideGroupByList(e, groupByExprs));
}
return true;
}
public static boolean isAggregateExpression(Expression expression) {
Function functionCall = expression.getFunctionCall();
if (functionCall != null) {
String operator = functionCall.getOperator();
if (AggregationFunctionType.isAggregationFunction(operator)) {
return true;
}
if (functionCall.getOperandsSize() > 0) {
for (Expression operand : functionCall.getOperands()) {
if (isAggregateExpression(operand)) {
return true;
}
}
}
}
return false;
}
public static boolean isAsFunction(Expression expression) {
Function function = expression.getFunctionCall();
return function != null && function.getOperator().equals("as");
}
/**
* Extract all the identifiers from given expressions.
*
* @param expressions
* @param excludeAs if true, ignores the right side identifier for AS function.
* @return all the identifier names.
*/
public static Set<String> extractIdentifiers(List<Expression> expressions, boolean excludeAs) {
Set<String> identifiers = new HashSet<>();
for (Expression expression : expressions) {
Identifier identifier = expression.getIdentifier();
if (identifier != null) {
identifiers.add(identifier.getName());
continue;
}
Function function = expression.getFunctionCall();
if (function != null) {
if (excludeAs && function.getOperator().equals("as")) {
identifiers.addAll(
extractIdentifiers(new ArrayList<>(Collections.singletonList(function.getOperands().get(0))), true));
} else {
identifiers.addAll(extractIdentifiers(function.getOperands(), excludeAs));
}
}
}
return identifiers;
}
/**
* Compiles a String expression into {@link Expression}.
*
* @param expression String expression.
* @return {@link Expression} equivalent of the string.
*
* @throws SqlCompilationException if String is not a valid expression.
*/
public static Expression compileToExpression(String expression) {
SqlNode sqlNode;
try (StringReader inStream = new StringReader(expression)) {
SqlParserImpl sqlParser = newSqlParser(inStream);
sqlNode = sqlParser.parseSqlExpressionEof();
} catch (Throwable e) {
throw new SqlCompilationException("Caught exception while parsing expression: " + expression, e);
}
return toExpression(sqlNode);
}
@VisibleForTesting
static SqlParserImpl newSqlParser(StringReader inStream) {
SqlParserImpl sqlParser = new SqlParserImpl(inStream);
sqlParser.switchTo(SqlAbstractParserImpl.LexicalState.DQID);
// TODO: convert to MySQL conformance once we retired most of the un-tested BABEL tokens
sqlParser.setConformance(SqlConformanceEnum.BABEL);
sqlParser.setTabSize(1);
sqlParser.setQuotedCasing(Casing.UNCHANGED);
sqlParser.setUnquotedCasing(Casing.UNCHANGED);
sqlParser.setIdentifierMaxLength(SqlParser.DEFAULT_IDENTIFIER_MAX_LENGTH);
return sqlParser;
}
public static Map<String, String> extractOptionsMap(List<String> optionsStatements) {
Map<String, String> options = new HashMap<>();
for (String optionsStatement : optionsStatements) {
for (String option : optionsStatement.split(",")) {
final String[] splits = option.split("=");
if (splits.length != 2) {
throw new SqlCompilationException("OPTION statement requires two parts separated by '='");
}
options.put(splits[0].trim(), splits[1].trim());
}
}
return options;
}
private static void setOptions(PinotQuery pinotQuery, List<String> optionsStatements) {
if (optionsStatements.isEmpty()) {
return;
}
pinotQuery.setQueryOptions(extractOptionsMap(optionsStatements));
}
public static PinotQuery compileSqlNodeToPinotQuery(SqlNode sqlNode) {
PinotQuery pinotQuery = new PinotQuery();
if (sqlNode instanceof SqlExplain) {
// Extract sql node for the query
sqlNode = ((SqlExplain) sqlNode).getExplicandum();
pinotQuery.setExplain(true);
}
SqlSelect selectNode;
if (sqlNode instanceof SqlOrderBy) {
// Store order-by info into the select sql node
SqlOrderBy orderByNode = (SqlOrderBy) sqlNode;
selectNode = (SqlSelect) orderByNode.query;
selectNode.setOrderBy(orderByNode.orderList);
selectNode.setFetch(orderByNode.fetch);
selectNode.setOffset(orderByNode.offset);
} else {
selectNode = (SqlSelect) sqlNode;
}
// SELECT
if (selectNode.getModifierNode(SqlSelectKeyword.DISTINCT) != null) {
// SELECT DISTINCT
if (selectNode.getGroup() != null) {
// TODO: explore support for GROUP BY with DISTINCT
throw new SqlCompilationException("DISTINCT with GROUP BY is not supported");
}
pinotQuery.setSelectList(convertDistinctSelectList(selectNode.getSelectList()));
} else {
pinotQuery.setSelectList(convertSelectList(selectNode.getSelectList()));
}
// FROM
SqlNode fromNode = selectNode.getFrom();
if (fromNode != null) {
DataSource dataSource = new DataSource();
dataSource.setTableName(fromNode.toString());
pinotQuery.setDataSource(dataSource);
if (fromNode instanceof SqlSelect || fromNode instanceof SqlOrderBy) {
dataSource.setSubquery(compileSqlNodeToPinotQuery(fromNode));
}
}
// WHERE
SqlNode whereNode = selectNode.getWhere();
if (whereNode != null) {
pinotQuery.setFilterExpression(toExpression(whereNode));
}
// GROUP-BY
SqlNodeList groupByNodeList = selectNode.getGroup();
if (groupByNodeList != null) {
pinotQuery.setGroupByList(convertSelectList(groupByNodeList));
}
// HAVING
SqlNode havingNode = selectNode.getHaving();
if (havingNode != null) {
pinotQuery.setHavingExpression(toExpression(havingNode));
}
// ORDER-BY
SqlNodeList orderByNodeList = selectNode.getOrderList();
if (orderByNodeList != null) {
pinotQuery.setOrderByList(convertOrderByList(orderByNodeList));
}
// LIMIT
SqlNode limitNode = selectNode.getFetch();
if (limitNode != null) {
pinotQuery.setLimit(((SqlNumericLiteral) limitNode).intValue(false));
}
// OFFSET
SqlNode offsetNode = selectNode.getOffset();
if (offsetNode != null) {
pinotQuery.setOffset(((SqlNumericLiteral) offsetNode).intValue(false));
}
queryRewrite(pinotQuery);
return pinotQuery;
}
private static void queryRewrite(PinotQuery pinotQuery) {
for (QueryRewriter queryRewriter : QUERY_REWRITERS) {
pinotQuery = queryRewriter.rewrite(pinotQuery);
}
// Validate
validate(pinotQuery);
}
private static List<String> extractOptionsFromSql(String sql) {
List<String> results = new ArrayList<>();
Matcher matcher = OPTIONS_REGEX_PATTEN.matcher(sql);
while (matcher.find()) {
results.add(matcher.group(1));
}
return results;
}
private static String removeOptionsFromSql(String sql) {
Matcher matcher = OPTIONS_REGEX_PATTEN.matcher(sql);
return matcher.replaceAll("");
}
/**
* Removes comments from the query.
* NOTE: Comment indicator within single quotes (literal) and double quotes (identifier) are ignored.
*/
@VisibleForTesting
static String removeComments(String sql) {
boolean openSingleQuote = false;
boolean openDoubleQuote = false;
boolean commented = false;
boolean singleLineCommented = false;
boolean multiLineCommented = false;
int commentStartIndex = -1;
List<Pairs.IntPair> commentedParts = new ArrayList<>();
int length = sql.length();
int index = 0;
while (index < length) {
switch (sql.charAt(index)) {
case '\'':
if (!commented && !openDoubleQuote) {
openSingleQuote = !openSingleQuote;
}
break;
case '"':
if (!commented && !openSingleQuote) {
openDoubleQuote = !openDoubleQuote;
}
break;
case '-':
// Single line comment start indicator: --
if (!commented && !openSingleQuote && !openDoubleQuote && index < length - 1
&& sql.charAt(index + 1) == '-') {
commented = true;
singleLineCommented = true;
commentStartIndex = index;
index++;
}
break;
case '\n':
// Single line comment end indicator: \n
if (singleLineCommented) {
commentedParts.add(new Pairs.IntPair(commentStartIndex, index + 1));
commented = false;
singleLineCommented = false;
commentStartIndex = -1;
}
break;
case '/':
// Multi-line comment start indicator: /*
if (!commented && !openSingleQuote && !openDoubleQuote && index < length - 1
&& sql.charAt(index + 1) == '*') {
commented = true;
multiLineCommented = true;
commentStartIndex = index;
index++;
}
break;
case '*':
// Multi-line comment end indicator: */
if (multiLineCommented && index < length - 1 && sql.charAt(index + 1) == '/') {
commentedParts.add(new Pairs.IntPair(commentStartIndex, index + 2));
commented = false;
multiLineCommented = false;
commentStartIndex = -1;
index++;
}
break;
default:
break;
}
index++;
}
if (commentedParts.isEmpty()) {
if (singleLineCommented) {
return sql.substring(0, commentStartIndex);
} else {
return sql;
}
} else {
StringBuilder stringBuilder = new StringBuilder();
int startIndex = 0;
for (Pairs.IntPair commentedPart : commentedParts) {
stringBuilder.append(sql, startIndex, commentedPart.getLeft()).append(' ');
startIndex = commentedPart.getRight();
}
if (startIndex < length) {
if (singleLineCommented) {
stringBuilder.append(sql, startIndex, commentStartIndex);
} else {
stringBuilder.append(sql, startIndex, length);
}
}
return stringBuilder.toString();
}
}
private static List<Expression> convertDistinctSelectList(SqlNodeList selectList) {
List<Expression> selectExpr = new ArrayList<>();
selectExpr.add(convertDistinctAndSelectListToFunctionExpression(selectList));
return selectExpr;
}
private static List<Expression> convertSelectList(SqlNodeList selectList) {
List<Expression> selectExpr = new ArrayList<>();
final Iterator<SqlNode> iterator = selectList.iterator();
while (iterator.hasNext()) {
final SqlNode next = iterator.next();
selectExpr.add(toExpression(next));
}
return selectExpr;
}
private static List<Expression> convertOrderByList(SqlNodeList orderList) {
List<Expression> orderByExpr = new ArrayList<>();
final Iterator<SqlNode> iterator = orderList.iterator();
while (iterator.hasNext()) {
final SqlNode next = iterator.next();
orderByExpr.add(convertOrderBy(next));
}
return orderByExpr;
}
private static Expression convertOrderBy(SqlNode node) {
Expression expression;
if (node.getKind() == SqlKind.DESCENDING) {
SqlBasicCall basicCall = (SqlBasicCall) node;
expression = RequestUtils.getFunctionExpression("desc");
expression.getFunctionCall().addToOperands(toExpression(basicCall.getOperandList().get(0)));
} else {
expression = RequestUtils.getFunctionExpression("asc");
expression.getFunctionCall().addToOperands(toExpression(node));
}
return expression;
}
/**
* DISTINCT is implemented as an aggregation function so need to take the select list items
* and convert them into a single function expression for handing over to execution engine
* either as a PinotQuery or BrokerRequest via conversion
* @param selectList select list items
* @return DISTINCT function expression
*/
private static Expression convertDistinctAndSelectListToFunctionExpression(SqlNodeList selectList) {
Expression functionExpression = RequestUtils.getFunctionExpression("distinct");
for (SqlNode node : selectList) {
Expression columnExpression = toExpression(node);
if (columnExpression.getType() == ExpressionType.IDENTIFIER && columnExpression.getIdentifier().getName()
.equals("*")) {
throw new SqlCompilationException(
"Syntax error: Pinot currently does not support DISTINCT with *. Please specify each column name after "
+ "DISTINCT keyword");
} else if (columnExpression.getType() == ExpressionType.FUNCTION) {
if (AggregationFunctionType.isAggregationFunction(columnExpression.getFunctionCall().getOperator())) {
throw new SqlCompilationException(
"Syntax error: Use of DISTINCT with aggregation functions is not supported");
}
}
functionExpression.getFunctionCall().addToOperands(columnExpression);
}
return functionExpression;
}
private static Expression toExpression(SqlNode node) {
LOGGER.debug("Current processing SqlNode: {}, node.getKind(): {}", node, node.getKind());
switch (node.getKind()) {
case IDENTIFIER:
if (((SqlIdentifier) node).isStar()) {
return RequestUtils.getIdentifierExpression("*");
}
if (((SqlIdentifier) node).isSimple()) {
return RequestUtils.getIdentifierExpression(((SqlIdentifier) node).getSimple());
}
return RequestUtils.getIdentifierExpression(node.toString());
case LITERAL:
return RequestUtils.getLiteralExpression((SqlLiteral) node);
case AS:
SqlBasicCall asFuncSqlNode = (SqlBasicCall) node;
List<SqlNode> operands = asFuncSqlNode.getOperandList();
Expression leftExpr = toExpression(operands.get(0));
SqlNode aliasSqlNode = operands.get(1);
String aliasName;
switch (aliasSqlNode.getKind()) {
case IDENTIFIER:
aliasName = ((SqlIdentifier) aliasSqlNode).getSimple();
break;
case LITERAL:
aliasName = ((SqlLiteral) aliasSqlNode).toValue();
break;
default:
throw new SqlCompilationException("Unsupported Alias sql node - " + aliasSqlNode);
}
Expression rightExpr = RequestUtils.getIdentifierExpression(aliasName);
// Just return left identifier if both sides are the same identifier.
if (leftExpr.isSetIdentifier() && rightExpr.isSetIdentifier()) {
if (leftExpr.getIdentifier().getName().equals(rightExpr.getIdentifier().getName())) {
return leftExpr;
}
}
Expression asFuncExpr = RequestUtils.getFunctionExpression("as");
asFuncExpr.getFunctionCall().addToOperands(leftExpr);
asFuncExpr.getFunctionCall().addToOperands(rightExpr);
return asFuncExpr;
case CASE:
// CASE WHEN Statement is model as a function with variable length parameters.
// Assume N is number of WHEN Statements, total number of parameters is (2 * N + 1).
// - N: Convert each WHEN Statement into a function Expression;
// - N: Convert each THEN Statement into an Expression;
// - 1: Convert ELSE Statement into an Expression.
SqlCase caseSqlNode = (SqlCase) node;
SqlNodeList whenOperands = caseSqlNode.getWhenOperands();
SqlNodeList thenOperands = caseSqlNode.getThenOperands();
SqlNode elseOperand = caseSqlNode.getElseOperand();
Expression caseFuncExpr = RequestUtils.getFunctionExpression("case");
for (SqlNode whenSqlNode : whenOperands.getList()) {
Expression whenExpression = toExpression(whenSqlNode);
if (isAggregateExpression(whenExpression)) {
throw new SqlCompilationException(
"Aggregation functions inside WHEN Clause is not supported - " + whenSqlNode);
}
caseFuncExpr.getFunctionCall().addToOperands(whenExpression);
}
for (SqlNode thenSqlNode : thenOperands.getList()) {
Expression thenExpression = toExpression(thenSqlNode);
if (isAggregateExpression(thenExpression)) {
throw new SqlCompilationException(
"Aggregation functions inside THEN Clause is not supported - " + thenSqlNode);
}
caseFuncExpr.getFunctionCall().addToOperands(thenExpression);
}
Expression elseExpression = toExpression(elseOperand);
if (isAggregateExpression(elseExpression)) {
throw new SqlCompilationException(
"Aggregation functions inside ELSE Clause is not supported - " + elseExpression);
}
caseFuncExpr.getFunctionCall().addToOperands(elseExpression);
return caseFuncExpr;
default:
if (node instanceof SqlDataTypeSpec) {
// This is to handle expression like: CAST(col AS INT)
return RequestUtils.getLiteralExpression(((SqlDataTypeSpec) node).getTypeName().getSimple());
} else {
return compileFunctionExpression((SqlBasicCall) node);
}
}
}
private static Expression compileFunctionExpression(SqlBasicCall functionNode) {
SqlKind functionKind = functionNode.getKind();
boolean negated = false;
String canonicalName;
switch (functionKind) {
case AND:
return compileAndExpression(functionNode);
case OR:
return compileOrExpression(functionNode);
// BETWEEN and LIKE might be negated (NOT BETWEEN, NOT LIKE)
case BETWEEN:
negated = ((SqlBetweenOperator) functionNode.getOperator()).isNegated();
canonicalName = SqlKind.BETWEEN.name();
break;
case LIKE:
negated = ((SqlLikeOperator) functionNode.getOperator()).isNegated();
canonicalName = SqlKind.LIKE.name();
break;
case OTHER:
case OTHER_FUNCTION:
case DOT:
String functionName = functionNode.getOperator().getName();
if (functionName.equals("ITEM") || functionName.equals("DOT")) {
// Calcite parses path expression such as "data[0][1].a.b[0]" into a chain of ITEM and/or DOT
// functions. Collapse this chain into an identifier.
StringBuilder pathBuilder = new StringBuilder();
compilePathExpression(functionNode, pathBuilder);
return RequestUtils.getIdentifierExpression(pathBuilder.toString());
}
canonicalName = RequestUtils.canonicalizeFunctionNamePreservingSpecialKey(functionName);
if ((functionNode.getFunctionQuantifier() != null) && ("DISTINCT".equals(
functionNode.getFunctionQuantifier().toString()))) {
if (canonicalName.equals("count")) {
canonicalName = "distinctcount";
} else if (AggregationFunctionType.isAggregationFunction(canonicalName)) {
// Aggregation function(other than COUNT) on DISTINCT is not supported, e.g. SUM(DISTINCT colA).
throw new SqlCompilationException("Function '" + functionName + "' on DISTINCT is not supported.");
}
}
break;
default:
canonicalName = RequestUtils.canonicalizeFunctionNamePreservingSpecialKey(functionKind.name());
break;
}
// When there is no argument, set an empty list as the operands
List<SqlNode> childNodes = functionNode.getOperandList();
List<Expression> operands = new ArrayList<>(childNodes.size());
for (SqlNode childNode : childNodes) {
if (childNode instanceof SqlNodeList) {
for (SqlNode node : (SqlNodeList) childNode) {
operands.add(toExpression(node));
}
} else {
operands.add(toExpression(childNode));
}
}
validateFunction(canonicalName, operands);
Expression functionExpression = RequestUtils.getFunctionExpression(canonicalName);
functionExpression.getFunctionCall().setOperands(operands);
if (negated) {
Expression negatedFunctionExpression = RequestUtils.getFunctionExpression(FilterKind.NOT.name());
// Do not use `Collections.singletonList()` because we might modify the operand later
List<Expression> negatedFunctionOperands = new ArrayList<>(1);
negatedFunctionOperands.add(functionExpression);
negatedFunctionExpression.getFunctionCall().setOperands(negatedFunctionOperands);
return negatedFunctionExpression;
} else {
return functionExpression;
}
}
/**
* Convert Calcite operator tree made up of ITEM and DOT functions to an identifier. For example, the operator tree
* shown below will be converted to IDENTIFIER "jsoncolumn.data[0][1].a.b[0]".
*
* ├── ITEM(jsoncolumn.data[0][1].a.b[0])
* ├── LITERAL (0)
* └── DOT (jsoncolumn.daa[0][1].a.b)
* ├── IDENTIFIER (b)
* └── DOT (jsoncolumn.data[0][1].a)
* ├── IDENTIFIER (a)
* └── ITEM (jsoncolumn.data[0][1])
* ├── LITERAL (1)
* └── ITEM (jsoncolumn.data[0])
* ├── LITERAL (1)
* └── IDENTIFIER (jsoncolumn.data)
*
* @param functionNode Root node of the DOT and/or ITEM operator function chain.
* @param pathBuilder StringBuilder representation of path represented by DOT and/or ITEM function chain.
*/
private static void compilePathExpression(SqlBasicCall functionNode, StringBuilder pathBuilder) {
List<SqlNode> operands = functionNode.getOperandList();
// Compile first operand of the function (either an identifier or another DOT and/or ITEM function).
SqlNode operand0 = operands.get(0);
SqlKind kind0 = operand0.getKind();
if (kind0 == SqlKind.IDENTIFIER) {
pathBuilder.append(operand0);
} else if (kind0 == SqlKind.DOT || kind0 == SqlKind.OTHER_FUNCTION) {
SqlBasicCall function0 = (SqlBasicCall) operand0;
String name0 = function0.getOperator().getName();
if (name0.equals("ITEM") || name0.equals("DOT")) {
compilePathExpression(function0, pathBuilder);
} else {
throw new SqlCompilationException("SELECT list item has bad path expression.");
}
} else {
throw new SqlCompilationException("SELECT list item has bad path expression.");
}
// Compile second operand of the function (either an identifier or literal).
SqlNode operand1 = operands.get(1);
SqlKind kind1 = operand1.getKind();
if (kind1 == SqlKind.IDENTIFIER) {
pathBuilder.append('.').append(((SqlIdentifier) operand1).getSimple());
} else if (kind1 == SqlKind.LITERAL) {
pathBuilder.append('[').append(((SqlLiteral) operand1).toValue()).append(']');
} else {
throw new SqlCompilationException("SELECT list item has bad path expression.");
}
}
private static void validateFunction(String canonicalName, List<Expression> operands) {
switch (canonicalName) {
case "jsonextractscalar":
validateJsonExtractScalarFunction(operands);
break;
case "jsonextractkey":
validateJsonExtractKeyFunction(operands);
break;
default:
break;
}
}
private static void validateJsonExtractScalarFunction(List<Expression> operands) {
int numOperands = operands.size();
// Check that there are exactly 3 or 4 arguments
if (numOperands != 3 && numOperands != 4) {
throw new SqlCompilationException(
"Expect 3 or 4 arguments for transform function: jsonExtractScalar(jsonFieldName, 'jsonPath', "
+ "'resultsType', ['defaultValue'])");
}
if (!operands.get(1).isSetLiteral() || !operands.get(2).isSetLiteral() || (numOperands == 4 && !operands.get(3)
.isSetLiteral())) {
throw new SqlCompilationException(
"Expect the 2nd/3rd/4th argument of transform function: jsonExtractScalar(jsonFieldName, 'jsonPath',"
+ " 'resultsType', ['defaultValue']) to be a single-quoted literal value.");
}
}
private static void validateJsonExtractKeyFunction(List<Expression> operands) {
// Check that there are exactly 2 arguments
if (operands.size() != 2) {
throw new SqlCompilationException(
"Expect 2 arguments are required for transform function: jsonExtractKey(jsonFieldName, 'jsonPath')");
}
if (!operands.get(1).isSetLiteral()) {
throw new SqlCompilationException(
"Expect the 2nd argument for transform function: jsonExtractKey(jsonFieldName, 'jsonPath') to be a "
+ "single-quoted literal value.");
}
}
/**
* Helper method to flatten the operands for the AND expression.
*/
private static Expression compileAndExpression(SqlBasicCall andNode) {
List<Expression> operands = new ArrayList<>();
for (SqlNode childNode : andNode.getOperandList()) {
if (childNode.getKind() == SqlKind.AND) {
Expression childAndExpression = compileAndExpression((SqlBasicCall) childNode);
operands.addAll(childAndExpression.getFunctionCall().getOperands());
} else {
operands.add(toExpression(childNode));
}
}
Expression andExpression = RequestUtils.getFunctionExpression(FilterKind.AND.name());
andExpression.getFunctionCall().setOperands(operands);
return andExpression;
}
/**
* Helper method to flatten the operands for the OR expression.
*/
private static Expression compileOrExpression(SqlBasicCall orNode) {
List<Expression> operands = new ArrayList<>();
for (SqlNode childNode : orNode.getOperandList()) {
if (childNode.getKind() == SqlKind.OR) {
Expression childAndExpression = compileOrExpression((SqlBasicCall) childNode);
operands.addAll(childAndExpression.getFunctionCall().getOperands());
} else {
operands.add(toExpression(childNode));
}
}
Expression andExpression = RequestUtils.getFunctionExpression(FilterKind.OR.name());
andExpression.getFunctionCall().setOperands(operands);
return andExpression;
}
public static boolean isLiteralOnlyExpression(Expression e) {
if (e.getType() == ExpressionType.LITERAL) {
return true;
}
if (e.getType() == ExpressionType.FUNCTION) {
Function function = e.getFunctionCall();
if (function.getOperator().equals("as")) {
return isLiteralOnlyExpression(function.getOperands().get(0));
}
return false;
}
return false;
}
}