blob: ecce9ccf1c80a1e730f843ccaf706e8e40ed1bb8 [file] [log] [blame]
// 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.impala.analysis;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java_cup.runtime.Symbol;
import org.apache.impala.analysis.ColumnDef;
import org.apache.impala.analysis.ColumnDef.Option;
import org.apache.impala.analysis.UnionStmt.Qualifier;
import org.apache.impala.analysis.UnionStmt.UnionOperand;
import org.apache.impala.analysis.RangePartition;
import org.apache.impala.analysis.TableSampleClause;
import org.apache.impala.analysis.AlterTableAddDropRangePartitionStmt.Operation;
import org.apache.impala.catalog.ArrayType;
import org.apache.impala.catalog.MapType;
import org.apache.impala.catalog.RowFormat;
import org.apache.impala.catalog.ScalarType;
import org.apache.impala.catalog.StructField;
import org.apache.impala.catalog.StructType;
import org.apache.impala.catalog.Type;
import org.apache.impala.catalog.View;
import org.apache.impala.common.Pair;
import org.apache.impala.thrift.TCatalogObjectType;
import org.apache.impala.thrift.TDescribeOutputStyle;
import org.apache.impala.thrift.TFunctionCategory;
import org.apache.impala.thrift.THdfsFileFormat;
import org.apache.impala.thrift.TOwnerType;
import org.apache.impala.thrift.TPrivilegeLevel;
import org.apache.impala.thrift.TQueryOptions;
import org.apache.impala.thrift.TShowStatsOp;
import org.apache.impala.thrift.TTablePropertyType;
import org.apache.impala.thrift.TPrincipalType;
import org.apache.impala.thrift.TSortingOrder;
import org.apache.impala.service.BackendConfig;
import org.apache.impala.common.NotImplementedException;
parser code {:
private Symbol errorToken_;
// Set if the errorToken_ to be printed in the error message has a different name, e.g.
// when parsing identifiers instead of defined keywords. This is necessary to avoid
// conflicting keywords.
private String expectedTokenName_;
// list of expected tokens ids from current parsing state
// for generating syntax error message
private final List<Integer> expectedTokenIds_ = new ArrayList<>();
// Currently used to tell if it's decimal V1 or V2 mode.
// TODO: remove when V1 code is dropped.
private TQueryOptions queryOptions;
// to avoid reporting trivial tokens as expected tokens in error messages
private boolean reportExpectedToken(Integer tokenId, int numExpectedTokens) {
if (SqlScanner.isKeyword(tokenId) ||
tokenId.intValue() == SqlParserSymbols.COMMA ||
tokenId.intValue() == SqlParserSymbols.IDENT) {
return true;
} else {
// if this is the only valid token, always report it
return numExpectedTokens == 1;
}
}
private String getErrorTypeMessage(int lastTokenId) {
String msg = null;
switch (lastTokenId) {
case SqlParserSymbols.UNMATCHED_STRING_LITERAL:
msg = "Unmatched string literal";
break;
case SqlParserSymbols.NUMERIC_OVERFLOW:
msg = "Numeric overflow";
break;
default:
msg = "Syntax error";
break;
}
return msg;
}
public void setQueryOptions(TQueryOptions options) {
queryOptions = options;
}
public TQueryOptions getQueryOptions() {
return queryOptions;
}
// override to save error token
@Override
public void syntax_error(java_cup.runtime.Symbol token) {
errorToken_ = token;
// derive expected tokens from current parsing state
expectedTokenIds_.clear();
int state = ((Symbol)stack.peek()).parse_state;
// get row of actions table corresponding to current parsing state
// the row consists of pairs of <tokenId, actionId>
// a pair is stored as row[i] (tokenId) and row[i+1] (actionId)
// the last pair is a special error action
short[] row = action_tab[state];
short tokenId;
// the expected tokens are all the symbols with a
// corresponding action from the current parsing state
for (int i = 0; i < row.length-2; ++i) {
// get tokenId and skip actionId
tokenId = row[i++];
expectedTokenIds_.add(Integer.valueOf(tokenId));
}
}
// override to keep it from calling report_fatal_error()
@Override
public void unrecovered_syntax_error(Symbol cur_token)
throws Exception {
throw new Exception(getErrorTypeMessage(cur_token.sym));
}
/**
* Manually throw a parse error on a given symbol for special circumstances.
*
* @symbolName
* name of symbol on which to fail parsing
* @symbolId
* id of symbol from SqlParserSymbols on which to fail parsing
*/
public void parseError(String symbolName, int symbolId) throws Exception {
parseError(symbolName, symbolId, null);
}
/**
* Same as parseError() above but allows the error token to have a different
* name printed as the expected token.
*/
public void parseError(String symbolName, int symbolId, String expectedTokenName)
throws Exception {
expectedTokenName_ = expectedTokenName;
Symbol errorToken = getSymbolFactory().newSymbol(symbolName, symbolId,
((Symbol) stack.peek()), ((Symbol) stack.peek()), null);
// Call syntax error to gather information about expected tokens, etc.
// syntax_error does not throw an exception
syntax_error(errorToken);
// Unrecovered_syntax_error throws an exception and will terminate parsing
unrecovered_syntax_error(errorToken);
}
// Returns error string, consisting of a shortened offending line
// with a '^' under the offending token. Assumes
// that parse() has been called and threw an exception
public String getErrorMsg(String stmt) {
if (errorToken_ == null || stmt == null) return null;
// IMPALA-8497: Fix ArrayIndexOutOfBoundsException for queries that end with '\n'
String[] lines = stmt.split("\n", -1);
StringBuffer result = new StringBuffer();
result.append(getErrorTypeMessage(errorToken_.sym) + " in line ");
result.append(errorToken_.left);
result.append(":\n");
// errorToken_.left is the line number of error.
// errorToken_.right is the column number of the error.
String errorLine = lines[errorToken_.left - 1];
// If the error is that additional tokens are expected past the end,
// errorToken_.right will be past the end of the string.
int lastCharIndex = Math.min(errorLine.length(), errorToken_.right);
int maxPrintLength = 60;
int errorLoc = 0;
if (errorLine.length() <= maxPrintLength) {
// The line is short. Print the entire line.
result.append(errorLine);
result.append('\n');
errorLoc = errorToken_.right;
} else {
// The line is too long. Print maxPrintLength/2 characters before the error and
// after the error.
int contextLength = maxPrintLength / 2 - 3;
String leftSubStr;
if (errorToken_.right > maxPrintLength / 2) {
leftSubStr = "..." + errorLine.substring(errorToken_.right - contextLength,
lastCharIndex);
} else {
leftSubStr = errorLine.substring(0, errorToken_.right);
}
errorLoc = leftSubStr.length();
result.append(leftSubStr);
if (errorLine.length() - errorToken_.right > maxPrintLength / 2) {
result.append(errorLine.substring(errorToken_.right,
errorToken_.right + contextLength) + "...");
} else {
result.append(errorLine.substring(lastCharIndex));
}
result.append("\n");
}
// print error indicator
for (int i = 0; i < errorLoc - 1; ++i) {
result.append(' ');
}
result.append("^\n");
// only report encountered and expected tokens for syntax errors
if (errorToken_.sym == SqlParserSymbols.UNMATCHED_STRING_LITERAL ||
errorToken_.sym == SqlParserSymbols.NUMERIC_OVERFLOW) {
return result.toString();
}
// append last encountered token
result.append("Encountered: ");
String lastToken =
SqlScanner.tokenIdMap.get(Integer.valueOf(errorToken_.sym));
if (lastToken != null) {
result.append(lastToken);
} else if (SqlScanner.isReserved((String)errorToken_.value)) {
result.append("A reserved word cannot be used as an identifier: ")
.append((String)errorToken_.value);
} else {
result.append("Unknown last token with id: " + errorToken_.sym);
}
// append expected tokens
result.append('\n');
result.append("Expected: ");
if (expectedTokenName_ == null) {
String expectedToken = null;
Integer tokenId = null;
for (int i = 0; i < expectedTokenIds_.size(); ++i) {
tokenId = expectedTokenIds_.get(i);
if (reportExpectedToken(tokenId, expectedTokenIds_.size())) {
expectedToken = SqlScanner.tokenIdMap.get(tokenId);
result.append(expectedToken + ", ");
}
}
// remove trailing ", "
result.delete(result.length()-2, result.length());
} else {
result.append(expectedTokenName_);
}
result.append('\n');
return result.toString();
}
/**
* This methods checks if a given ident matches the given keyword.
*/
public void checkIdentKeyword(String keyword, String ident) throws Exception {
if (!keyword.equals(ident.toUpperCase())) {
parseError("identifier", SqlParserSymbols.IDENT, keyword);
}
}
:};
// List of keywords. Please keep them sorted alphabetically.
// ALL KEYWORDS ALSO NEED TO BE ADDED TO THE word PRODUCTION.
terminal
KW_ADD, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_ANALYTIC, KW_AND, KW_ANTI, KW_API_VERSION,
KW_ARRAY, KW_AS, KW_ASC, KW_AUTHORIZATION, KW_AVRO, KW_BETWEEN, KW_BIGINT, KW_BINARY,
KW_BLOCKSIZE, KW_BOOLEAN, KW_BY, KW_CACHED, KW_CASCADE, KW_CASE, KW_CAST, KW_CHANGE,
KW_CHAR, KW_CLASS, KW_CLOSE_FN, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMPRESSION,
KW_COMPUTE, KW_CONSTRAINT, KW_COPY, KW_CREATE, KW_CROSS, KW_CURRENT, KW_DATA,
KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DECIMAL, KW_DEFAULT, KW_DELETE,
KW_DELIMITED, KW_DESC, KW_DESCRIBE, KW_DISABLE, KW_DISTINCT, KW_DIV, KW_DOUBLE,
KW_DROP, KW_ELSE, KW_ENABLE, KW_ENCODING, KW_END, KW_ESCAPED, KW_EXISTS, KW_EXPLAIN,
KW_EXTENDED, KW_EXTERNAL, KW_FALSE, KW_FIELDS, KW_FILEFORMAT, KW_FILES, KW_FINALIZE_FN,
KW_FIRST, KW_FLOAT, KW_FOLLOWING, KW_FOR, KW_FOREIGN, KW_FORMAT, KW_FORMATTED,
KW_FROM, KW_FULL, KW_FUNCTION, KW_FUNCTIONS, KW_GRANT, KW_GROUP, KW_HASH, KW_IGNORE,
KW_HAVING, KW_IF, KW_ILIKE, KW_IN, KW_INCREMENTAL, KW_INIT_FN, KW_INNER, KW_INPATH,
KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP,
KW_IS, KW_JOIN, KW_KUDU, KW_LAST, KW_LEFT, KW_LEXICAL, KW_LIKE, KW_LIMIT, KW_LINES,
KW_LOAD, KW_LOCATION, KW_MAP, KW_MERGE_FN, KW_METADATA, KW_NORELY, KW_NOT,
KW_NOVALIDATE, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORC, KW_ORDER, KW_OUTER,
KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION, KW_PARTITIONED,
KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED, KW_PURGE,
KW_RANGE, KW_RCFILE, KW_RECOVER, KW_REFERENCES, KW_REFRESH, KW_REGEXP, KW_RELY,
KW_RENAME, KW_REPEATABLE, KW_REPLACE, KW_REPLICATION, KW_RESTRICT, KW_RETURNS,
KW_REVOKE, KW_RIGHT, KW_RLIKE, KW_ROLE, KW_ROLES, KW_ROW, KW_ROWS, KW_SCHEMA,
KW_SCHEMAS, KW_SELECT, KW_SEMI, KW_SEQUENCEFILE, KW_SERDEPROPERTIES, KW_SERIALIZE_FN,
KW_SET, KW_SHOW, KW_SMALLINT, KW_SORT, KW_STORED, KW_STRAIGHT_JOIN, KW_STRING,
KW_STRUCT, KW_SYMBOL, KW_TABLE, KW_TABLES, KW_TABLESAMPLE, KW_TBLPROPERTIES,
KW_TERMINATED, KW_TEXTFILE, KW_THEN, KW_TIMESTAMP, KW_TINYINT, KW_TRUNCATE, KW_STATS,
KW_TO, KW_TRUE, KW_UNBOUNDED, KW_UNCACHED, KW_UNION, KW_UNKNOWN, KW_UPDATE,
KW_UPDATE_FN, KW_UPSERT, KW_USE, KW_USING, KW_VALIDATE, KW_VALUES, KW_VARCHAR, KW_VIEW,
KW_WHEN, KW_WHERE, KW_WITH, KW_ZORDER;
terminal UNUSED_RESERVED_WORD;
terminal COLON, SEMICOLON, COMMA, DOT, DOTDOTDOT, STAR, LPAREN, RPAREN, LBRACKET,
RBRACKET, DIVIDE, MOD, ADD, SUBTRACT;
terminal UNARYSIGN; // Placeholder terminal for unary -/+
terminal BITAND, BITOR, BITXOR, BITNOT;
terminal EQUAL, NOT, NOTEQUAL, LESSTHAN, GREATERTHAN;
terminal FACTORIAL; // Placeholder terminal for postfix factorial operator
terminal COMMENTED_PLAN_HINT_START, COMMENTED_PLAN_HINT_END;
terminal String IDENT;
terminal String EMPTY_IDENT;
terminal String NUMERIC_OVERFLOW;
terminal BigDecimal INTEGER_LITERAL;
terminal BigDecimal DECIMAL_LITERAL;
terminal String STRING_LITERAL;
terminal String UNMATCHED_STRING_LITERAL;
terminal String UNEXPECTED_CHAR;
// IMPALA-3726 introduced the DEFAULT keyword which could break existing applications
// that use the identifier "KEYWORD" as database, column or table names. To avoid that,
// the ident_or_default non-terminal is introduced and should be used instead of IDENT.
nonterminal String ident_or_default;
// A word is an arbitrary token composed of digits and at least one letter. Reserved
// words cannot be used as identifiers but they are words and can be used in query
// options, column attributes, etc.
nonterminal String word;
nonterminal StatementBase stmt;
// Single select statement.
nonterminal SelectStmt select_stmt;
// Single values statement.
nonterminal ValuesStmt values_stmt;
// Select or union statement.
nonterminal QueryStmt query_stmt;
nonterminal QueryStmt opt_query_stmt;
// Single select_stmt or parenthesized query_stmt.
nonterminal QueryStmt union_operand;
// List of select or union blocks connected by UNION operators or a single select block.
nonterminal List<UnionOperand> union_operand_list;
// List of union operands consisting of constant selects.
nonterminal List<UnionOperand> values_operand_list;
// USE stmt
nonterminal UseStmt use_stmt;
nonterminal SetStmt set_stmt;
nonterminal ShowTablesStmt show_tables_stmt;
nonterminal ShowDbsStmt show_dbs_stmt;
nonterminal ShowStatsStmt show_stats_stmt, show_partitions_stmt,
show_range_partitions_stmt;
nonterminal String show_pattern;
nonterminal ShowFilesStmt show_files_stmt;
nonterminal DescribeDbStmt describe_db_stmt;
nonterminal DescribeTableStmt describe_table_stmt;
nonterminal ShowCreateTableStmt show_create_tbl_stmt;
nonterminal TCatalogObjectType show_create_tbl_object_type;
nonterminal ShowCreateFunctionStmt show_create_function_stmt;
nonterminal TDescribeOutputStyle describe_output_style;
nonterminal LoadDataStmt load_stmt;
nonterminal TruncateStmt truncate_stmt;
nonterminal ResetMetadataStmt reset_metadata_stmt;
// List of select blocks connected by UNION operators, with order by or limit.
nonterminal QueryStmt union_with_order_by_or_limit;
nonterminal SelectList select_clause;
nonterminal SelectList select_list;
nonterminal SelectListItem select_list_item;
nonterminal SelectListItem star_expr;
nonterminal Expr expr, non_pred_expr, arithmetic_expr, timestamp_arithmetic_expr;
nonterminal List<Expr> expr_list;
nonterminal String alias_clause;
nonterminal List<String> ident_list, primary_keys;
nonterminal List<String> opt_ident_list;
nonterminal Pair<List<String>, TSortingOrder> opt_sort_cols;
nonterminal TableName table_name;
nonterminal ColumnName column_name;
nonterminal FunctionName function_name;
nonterminal Expr where_clause;
nonterminal Expr predicate, bool_test_expr;
nonterminal Predicate between_predicate, comparison_predicate, compound_predicate,
in_predicate, like_predicate, exists_predicate;
nonterminal List<Expr> group_by_clause, opt_partition_by_clause;
nonterminal Expr having_clause;
nonterminal List<OrderByElement> order_by_elements, opt_order_by_clause;
nonterminal OrderByElement order_by_element;
nonterminal Boolean opt_order_param;
nonterminal Boolean opt_nulls_order_param;
nonterminal Expr opt_offset_param;
nonterminal LimitElement opt_limit_offset_clause;
nonterminal Expr opt_limit_clause, opt_offset_clause;
nonterminal Expr cast_expr, case_else_clause, analytic_expr;
nonterminal String cast_format_val;
nonterminal Expr function_call_expr;
nonterminal AnalyticWindow opt_window_clause;
nonterminal AnalyticWindow.Type window_type;
nonterminal AnalyticWindow.Boundary window_boundary;
nonterminal LiteralExpr literal;
nonterminal NumericLiteral numeric_literal;
nonterminal CaseExpr case_expr;
nonterminal List<CaseWhenClause> case_when_clause_list;
nonterminal FunctionParams function_params;
nonterminal List<String> dotted_path;
nonterminal SlotRef slot_ref;
nonterminal FromClause from_clause;
nonterminal List<TableRef> table_ref_list;
nonterminal TableSampleClause opt_tablesample;
nonterminal WithClause opt_with_clause;
nonterminal List<View> with_view_def_list;
nonterminal View with_view_def;
nonterminal TableRef table_ref;
nonterminal Subquery subquery;
nonterminal JoinOperator join_operator;
nonterminal opt_inner, opt_outer;
nonterminal PlanHint plan_hint;
nonterminal List<PlanHint> plan_hints, opt_plan_hints, plan_hint_list;
nonterminal TypeDef type_def;
nonterminal Type type;
nonterminal Expr sign_chain_expr;
nonterminal InsertStmt insert_stmt, upsert_stmt;
nonterminal UpdateStmt update_stmt;
nonterminal DeleteStmt delete_stmt;
nonterminal List<Pair<SlotRef, Expr>> update_set_expr_list;
nonterminal StatementBase explain_stmt;
// Optional partition spec
nonterminal PartitionSpec opt_partition_spec;
// Required partition spec
nonterminal PartitionSpec partition_spec;
// Optional partition set
nonterminal PartitionSet opt_partition_set;
// Required partition set
nonterminal PartitionSet partition_set;
nonterminal List<PartitionKeyValue> partition_clause;
nonterminal List<PartitionKeyValue> static_partition_key_value_list;
nonterminal List<PartitionKeyValue> partition_key_value_list;
nonterminal PartitionKeyValue partition_key_value;
nonterminal PartitionKeyValue static_partition_key_value;
nonterminal Qualifier union_op;
// For ALTER DATABASE.
nonterminal AlterDbStmt alter_db_stmt;
nonterminal PartitionDef partition_def;
nonterminal List<PartitionDef> partition_def_list;
nonterminal CommentOnStmt comment_on_stmt;
nonterminal AlterTableStmt alter_tbl_stmt;
nonterminal StatementBase alter_view_stmt;
nonterminal ComputeStatsStmt compute_stats_stmt;
nonterminal DropDbStmt drop_db_stmt;
nonterminal DropStatsStmt drop_stats_stmt;
nonterminal DropTableOrViewStmt drop_tbl_or_view_stmt;
nonterminal CreateDbStmt create_db_stmt;
nonterminal CreateTableAsSelectStmt create_tbl_as_select_stmt;
nonterminal CreateTableAsSelectStmt.CtasParams create_tbl_as_select_params;
nonterminal CreateTableLikeStmt create_tbl_like_stmt;
nonterminal CreateTableStmt create_tbl_stmt;
nonterminal TableDef tbl_def_without_col_defs, tbl_def_with_col_defs;
nonterminal TableDataLayout opt_tbl_data_layout, partitioned_data_layout;
nonterminal TableDef.Options tbl_options;
nonterminal List<TableDef.ForeignKey> foreign_keys_list;
nonterminal CreateViewStmt create_view_stmt;
nonterminal CreateDataSrcStmt create_data_src_stmt;
nonterminal DropDataSrcStmt drop_data_src_stmt;
nonterminal ShowDataSrcsStmt show_data_srcs_stmt;
nonterminal StructField struct_field_def;
nonterminal KuduPartitionParam hash_partition_param;
nonterminal List<RangePartition> range_params_list;
nonterminal RangePartition range_param;
nonterminal Pair<List<Expr>, Boolean> opt_lower_range_val,
opt_upper_range_val;
nonterminal List<KuduPartitionParam> hash_partition_param_list;
nonterminal List<KuduPartitionParam> partition_param_list;
nonterminal KuduPartitionParam range_partition_param;
nonterminal ColumnDef column_def, view_column_def;
nonterminal List<ColumnDef> column_def_list, partition_column_defs,
view_column_def_list, view_column_defs;
nonterminal List<StructField> struct_field_def_list;
// Options for DDL commands - CREATE/DROP/ALTER
nonterminal HdfsCachingOp cache_op_val, opt_cache_op_val;
nonterminal BigDecimal opt_cache_op_replication;
nonterminal String comment_val, opt_comment_val, nullable_comment_val;
nonterminal Boolean external_val;
nonterminal Boolean purge_val;
nonterminal String opt_init_string_val;
nonterminal THdfsFileFormat file_format_val;
nonterminal THdfsFileFormat file_format_create_table_val;
nonterminal Boolean if_exists_val;
nonterminal Boolean if_not_exists_val;
nonterminal Boolean is_primary_key_val;
nonterminal HdfsUri location_val;
nonterminal RowFormat row_format_val, opt_row_format_val;
nonterminal String field_terminator_val;
nonterminal String line_terminator_val;
nonterminal String escaped_by_val;
nonterminal String terminator_val;
nonterminal TTablePropertyType table_property_type;
nonterminal HashMap serde_properties;
nonterminal HashMap tbl_properties;
nonterminal HashMap properties_map;
// Used to simplify commands that accept either KW_DATABASE(S) or KW_SCHEMA(S)
nonterminal String db_or_schema_kw;
nonterminal String dbs_or_schemas_kw;
// Used to simplify commands where KW_COLUMN is optional
nonterminal String opt_kw_column;
// Used to simplify commands where KW_TABLE is optional
nonterminal String opt_kw_table;
nonterminal Boolean overwrite_val;
nonterminal Boolean cascade_val;
nonterminal Boolean nullability_val;
nonterminal String encoding_val;
nonterminal String compression_val;
nonterminal Expr default_val;
nonterminal LiteralExpr block_size_val;
nonterminal Pair<Option, Object> column_option;
nonterminal Map<Option, Object> column_options_map;
// Used for integrity constraints(DISABLE, NOVALIDATE, RELY)
nonterminal Boolean enable_spec, validate_spec, rely_spec;
// For GRANT/REVOKE/AUTH DDL statements
nonterminal ShowRolesStmt show_roles_stmt;
nonterminal ShowGrantPrincipalStmt show_grant_principal_stmt;
nonterminal TPrincipalType principal_type;
nonterminal CreateDropRoleStmt create_drop_role_stmt;
nonterminal GrantRevokeRoleStmt grant_role_stmt;
nonterminal GrantRevokeRoleStmt revoke_role_stmt;
nonterminal GrantRevokePrivStmt grant_privilege_stmt;
nonterminal GrantRevokePrivStmt revoke_privilege_stmt;
nonterminal PrivilegeSpec privilege_spec;
nonterminal TPrivilegeLevel privilege;
nonterminal Boolean opt_with_grantopt;
nonterminal Boolean opt_grantopt_for;
// To avoid creating common keywords such as 'SERVER' or 'SOURCES' we treat them as
// identifiers rather than keywords. Throws a parse exception if the identifier does not
// match the expected string.
nonterminal key_ident;
nonterminal system_ident;
nonterminal Boolean option_ident;
nonterminal Boolean server_ident;
nonterminal Boolean source_ident;
nonterminal Boolean sources_ident;
nonterminal Boolean uri_ident;
nonterminal testcase_ident;
// For Create/Drop/Show function ddl
nonterminal FunctionArgs function_def_args;
nonterminal FunctionArgs function_def_arg_list;
// Accepts space separated key='v' arguments.
nonterminal HashMap function_def_args_map;
nonterminal CreateFunctionStmtBase.OptArg function_def_arg_key;
nonterminal Boolean opt_is_aggregate_fn;
nonterminal Boolean opt_is_varargs;
nonterminal TypeDef opt_aggregate_fn_intermediate_type_def;
nonterminal CreateUdfStmt create_udf_stmt;
nonterminal CreateUdaStmt create_uda_stmt;
nonterminal ShowFunctionsStmt show_functions_stmt;
nonterminal DropFunctionStmt drop_function_stmt;
nonterminal TFunctionCategory opt_function_category;
// Query testcase export/load
nonterminal CopyTestCaseStmt copy_testcase_stmt;
// Admin statements.
nonterminal AdminFnStmt admin_fn_stmt;
precedence left KW_OR;
precedence left KW_AND;
precedence right KW_NOT, NOT;
precedence left KW_DEFAULT;
precedence left KW_BETWEEN, KW_IN, KW_IS, KW_EXISTS;
precedence left KW_LIKE, KW_RLIKE, KW_ILIKE, KW_REGEXP, KW_IREGEXP;
precedence left EQUAL, NOTEQUAL, LESSTHAN, GREATERTHAN, KW_FROM, KW_DISTINCT;
precedence left ADD, SUBTRACT;
precedence left STAR, DIVIDE, MOD, KW_DIV;
precedence left BITAND, BITOR, BITXOR, BITNOT;
precedence left UNARYSIGN;
precedence left FACTORIAL;
precedence left KW_ORDER, KW_BY, KW_LIMIT;
precedence left LPAREN, RPAREN;
precedence left KW_VALUES;
// Support chaining of timestamp arithmetic exprs.
precedence left KW_INTERVAL;
precedence left KW_TBLPROPERTIES;
// These tokens need to be at the end for function_def_args_map to accept
// no keys. Otherwise, the grammar has shift/reduce conflicts.
precedence left KW_COMMENT;
precedence left KW_SYMBOL;
precedence left KW_PREPARE_FN;
precedence left KW_CLOSE_FN;
precedence left KW_UPDATE_FN;
precedence left KW_FINALIZE_FN;
precedence left KW_INIT_FN;
precedence left KW_MERGE_FN;
precedence left KW_SERIALIZE_FN;
precedence left KW_OVERWRITE;
precedence left KW_INTO;
precedence left KW_OVER;
start with stmt;
stmt ::=
query_stmt:query
{: RESULT = query; :}
| insert_stmt:insert
{: RESULT = insert; :}
| update_stmt:update
{: RESULT = update; :}
| upsert_stmt:upsert
{: RESULT = upsert; :}
| delete_stmt:delete
{: RESULT = delete; :}
| use_stmt:use
{: RESULT = use; :}
| show_tables_stmt:show_tables
{: RESULT = show_tables; :}
| show_dbs_stmt:show_dbs
{: RESULT = show_dbs; :}
| show_partitions_stmt:show_partitions
{: RESULT = show_partitions; :}
| show_range_partitions_stmt:show_range_partitions
{: RESULT = show_range_partitions; :}
| show_stats_stmt:show_stats
{: RESULT = show_stats; :}
| show_functions_stmt:show_functions
{: RESULT = show_functions; :}
| show_data_srcs_stmt:show_data_srcs
{: RESULT = show_data_srcs; :}
| show_create_tbl_stmt:show_create_tbl
{: RESULT = show_create_tbl; :}
| show_create_function_stmt:show_create_function
{: RESULT = show_create_function; :}
| show_files_stmt:show_files
{: RESULT = show_files; :}
| describe_db_stmt:describe
{: RESULT = describe; :}
| describe_table_stmt:describe
{: RESULT = describe; :}
| alter_db_stmt:alter_db
{: RESULT = alter_db; :}
| alter_tbl_stmt:alter_tbl
{: RESULT = alter_tbl; :}
| alter_view_stmt:alter_view
{: RESULT = alter_view; :}
| compute_stats_stmt:compute_stats
{: RESULT = compute_stats; :}
| copy_testcase_stmt:copy_testcase
{: RESULT = copy_testcase; :}
| drop_stats_stmt:drop_stats
{: RESULT = drop_stats; :}
| create_tbl_as_select_stmt:create_tbl_as_select
{: RESULT = create_tbl_as_select; :}
| create_tbl_like_stmt:create_tbl_like
{: RESULT = create_tbl_like; :}
| create_tbl_stmt:create_tbl
{: RESULT = create_tbl; :}
| create_view_stmt:create_view
{: RESULT = create_view; :}
| create_data_src_stmt:create_data_src
{: RESULT = create_data_src; :}
| create_db_stmt:create_db
{: RESULT = create_db; :}
| create_udf_stmt:create_udf
{: RESULT = create_udf; :}
| create_uda_stmt:create_uda
{: RESULT = create_uda; :}
| drop_db_stmt:drop_db
{: RESULT = drop_db; :}
| drop_tbl_or_view_stmt:drop_tbl
{: RESULT = drop_tbl; :}
| drop_function_stmt:drop_function
{: RESULT = drop_function; :}
| drop_data_src_stmt:drop_data_src
{: RESULT = drop_data_src; :}
| explain_stmt:explain
{: RESULT = explain; :}
| load_stmt: load
{: RESULT = load; :}
| truncate_stmt: truncate
{: RESULT = truncate; :}
| reset_metadata_stmt: reset_metadata
{: RESULT = reset_metadata; :}
| set_stmt:set
{: RESULT = set; :}
| show_roles_stmt:show_roles
{: RESULT = show_roles; :}
| show_grant_principal_stmt:show_grant_principal
{: RESULT = show_grant_principal; :}
| create_drop_role_stmt:create_drop_role
{: RESULT = create_drop_role; :}
| grant_role_stmt:grant_role
{: RESULT = grant_role; :}
| revoke_role_stmt:revoke_role
{: RESULT = revoke_role; :}
| grant_privilege_stmt:grant_privilege
{: RESULT = grant_privilege; :}
| revoke_privilege_stmt:revoke_privilege
{: RESULT = revoke_privilege; :}
| comment_on_stmt:comment_on
{: RESULT = comment_on; :}
| admin_fn_stmt:shutdown
{: RESULT = shutdown; :}
| stmt:s SEMICOLON
{: RESULT = s; :}
;
load_stmt ::=
KW_LOAD KW_DATA KW_INPATH STRING_LITERAL:path overwrite_val:overwrite KW_INTO KW_TABLE
table_name:table opt_partition_spec:partition
{: RESULT = new LoadDataStmt(table, new HdfsUri(path), overwrite, partition); :}
;
truncate_stmt ::=
KW_TRUNCATE KW_TABLE if_exists_val:if_exists table_name:tbl_name
{: RESULT = new TruncateStmt(tbl_name, if_exists); :}
| KW_TRUNCATE if_exists_val:if_exists table_name:tbl_name
{: RESULT = new TruncateStmt(tbl_name, if_exists); :}
;
overwrite_val ::=
KW_OVERWRITE
{: RESULT = Boolean.TRUE; :}
| /* empty */
{: RESULT = Boolean.FALSE; :}
;
reset_metadata_stmt ::=
KW_INVALIDATE KW_METADATA
{: RESULT = ResetMetadataStmt.createInvalidateStmt(); :}
| KW_INVALIDATE KW_METADATA table_name:table
{: RESULT = ResetMetadataStmt.createInvalidateStmt(table); :}
| KW_REFRESH table_name:table
{: RESULT = ResetMetadataStmt.createRefreshTableStmt(table); :}
| KW_REFRESH table_name:table partition_spec:partition
{: RESULT = ResetMetadataStmt.createRefreshPartitionStmt(table, partition); :}
| KW_REFRESH KW_FUNCTIONS ident_or_default:db
{: RESULT = ResetMetadataStmt.createRefreshFunctionsStmt(db); :}
| KW_REFRESH KW_AUTHORIZATION
{: RESULT = ResetMetadataStmt.createRefreshAuthorizationStmt(); :}
;
explain_stmt ::=
KW_EXPLAIN query_stmt:query
{:
query.setIsExplain();
RESULT = query;
:}
| KW_EXPLAIN insert_stmt:insert
{:
insert.setIsExplain();
RESULT = insert;
:}
| KW_EXPLAIN create_tbl_as_select_stmt:ctas_stmt
{:
ctas_stmt.setIsExplain();
RESULT = ctas_stmt;
:}
| KW_EXPLAIN update_stmt:update
{:
update.setIsExplain();
RESULT = update;
:}
| KW_EXPLAIN upsert_stmt:upsert
{:
upsert.setIsExplain();
RESULT = upsert;
:}
| KW_EXPLAIN delete_stmt:delete
{:
delete.setIsExplain();
RESULT = delete;
:}
;
copy_testcase_stmt ::=
KW_COPY testcase_ident:testcase KW_TO STRING_LITERAL:path query_stmt:query
{:
RESULT = CopyTestCaseStmt.to(query, new HdfsUri(path));
:}
| KW_COPY testcase_ident:testcase KW_FROM STRING_LITERAL:path
{:
RESULT = CopyTestCaseStmt.from(new HdfsUri(path));
:}
;
// Insert statements have two optional clauses: the column permutation (INSERT into
// tbl(col1,...) etc) and the PARTITION clause. If the column permutation is present, the
// query statement clause is optional as well.
// Note: when extending INSERT/UPSERT syntax, hinting is supported at the beginning of
// the statement and before the query.
insert_stmt ::=
opt_with_clause:w KW_INSERT KW_OVERWRITE opt_kw_table table_name:table
LPAREN opt_ident_list:col_perm RPAREN partition_clause:list opt_plan_hints:hints
opt_query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, true, list, hints,
InsertStmt.HintLocation.End, query, col_perm);
:}
| opt_with_clause:w KW_INSERT KW_OVERWRITE
opt_kw_table table_name:table
partition_clause:list opt_plan_hints:hints query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, true, list, hints,
InsertStmt.HintLocation.End, query, null);
:}
| opt_with_clause:w KW_INSERT KW_INTO opt_kw_table table_name:table
LPAREN opt_ident_list:col_perm RPAREN
partition_clause:list opt_plan_hints:hints opt_query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, false, list, hints,
InsertStmt.HintLocation.End, query, col_perm);
:}
| opt_with_clause:w KW_INSERT KW_INTO opt_kw_table table_name:table
partition_clause:list opt_plan_hints:hints query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, false, list, hints,
InsertStmt.HintLocation.End, query, null);
:}
| opt_with_clause:w KW_INSERT opt_plan_hints:hints KW_OVERWRITE opt_kw_table
table_name:table LPAREN opt_ident_list:col_perm RPAREN partition_clause:list
opt_query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, true, list, hints,
InsertStmt.HintLocation.Start, query, col_perm);
:}
| opt_with_clause:w KW_INSERT opt_plan_hints:hints KW_OVERWRITE
opt_kw_table table_name:table
partition_clause:list query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, true, list, hints,
InsertStmt.HintLocation.Start, query, null);
:}
| opt_with_clause:w KW_INSERT opt_plan_hints:hints KW_INTO opt_kw_table
table_name:table LPAREN opt_ident_list:col_perm RPAREN
partition_clause:list opt_query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, false, list, hints,
InsertStmt.HintLocation.Start, query, col_perm);
:}
| opt_with_clause:w KW_INSERT opt_plan_hints:hints KW_INTO opt_kw_table
table_name:table partition_clause:list query_stmt:query
{:
RESULT = InsertStmt.createInsert(w, table, false, list, hints,
InsertStmt.HintLocation.Start, query, null);
:}
;
// Update statements have an optional WHERE and optional FROM clause.
update_stmt ::=
KW_UPDATE dotted_path:target_table KW_SET update_set_expr_list:values
where_clause:where_predicate
{:
FromClause from_clause = new FromClause(
Lists.newArrayList(new TableRef(target_table, null)));
RESULT = new UpdateStmt(target_table, from_clause, values, where_predicate);
:}
| KW_UPDATE dotted_path:target_table
KW_SET update_set_expr_list:values
from_clause:tables where_clause:where_predicate
{: RESULT = new UpdateStmt(target_table, tables, values, where_predicate); :}
;
update_set_expr_list ::=
slot_ref:slot EQUAL expr:e
{:
List<Pair<SlotRef, Expr>> tmp =
Lists.newArrayList(new Pair<SlotRef, Expr>(slot, e));
RESULT = tmp;
:}
| update_set_expr_list:list COMMA slot_ref:slot EQUAL expr:e
{:
list.add(new Pair(slot, e));
RESULT = list;
:}
;
// Upsert statements have an optional column permutation clause. If the column permutation
// is present, the query statement clause is optional as well.
// Note: when extending INSERT/UPSERT syntax, hinting is supported at the beginning of
// the statement and before the query.
upsert_stmt ::=
opt_with_clause:w KW_UPSERT KW_INTO opt_kw_table table_name:table
LPAREN opt_ident_list:col_perm RPAREN opt_plan_hints:hints opt_query_stmt:query
{: RESULT = InsertStmt.createUpsert(w, table, hints, InsertStmt.HintLocation.End,
query, col_perm); :}
| opt_with_clause:w KW_UPSERT KW_INTO opt_kw_table table_name:table
opt_plan_hints:hints query_stmt:query
{: RESULT = InsertStmt.createUpsert(w, table, hints, InsertStmt.HintLocation.End,
query, null); :}
| opt_with_clause:w KW_UPSERT opt_plan_hints:hints KW_INTO opt_kw_table table_name:table
LPAREN opt_ident_list:col_perm RPAREN opt_query_stmt:query
{: RESULT = InsertStmt.createUpsert(w, table, hints, InsertStmt.HintLocation.Start,
query, col_perm); :}
| opt_with_clause:w KW_UPSERT opt_plan_hints:hints KW_INTO opt_kw_table table_name:table
query_stmt:query
{: RESULT = InsertStmt.createUpsert(w, table, hints, InsertStmt.HintLocation.Start,
query, null); :}
;
// A DELETE statement comes in two main representations, the DELETE keyword with a path
// specification as the target table with an optional FROM keyword or the DELETE
// keyword followed by a table alias or reference and a full FROM clause. In all cases
// a WHERE clause may be present.
delete_stmt ::=
KW_DELETE dotted_path:target_table where_clause:where
{:
FromClause from_clause = new FromClause(
Lists.newArrayList(new TableRef(target_table, null)));
RESULT = new DeleteStmt(target_table, from_clause, where);
:}
| KW_DELETE KW_FROM dotted_path:target_table where_clause:where
{:
FromClause from_clause = new FromClause(
Lists.newArrayList(new TableRef(target_table, null)));
RESULT = new DeleteStmt(target_table, from_clause, where);
:}
| KW_DELETE dotted_path:target_table from_clause:from
where_clause:where
{: RESULT = new DeleteStmt(target_table, from, where); :}
;
opt_query_stmt ::=
query_stmt:query
{: RESULT = query; :}
| /* empty */
{: RESULT = null; :}
;
opt_ident_list ::=
ident_list:ident
{: RESULT = ident; :}
| /* empty */
{: RESULT = new ArrayList<>(); :}
;
opt_kw_table ::=
KW_TABLE
| /* empty */
;
show_roles_stmt ::=
KW_SHOW KW_ROLES
{: RESULT = new ShowRolesStmt(false, null); :}
| KW_SHOW KW_ROLE KW_GRANT KW_GROUP ident_or_default:group
{: RESULT = new ShowRolesStmt(false, group); :}
| KW_SHOW KW_CURRENT KW_ROLES
{: RESULT = new ShowRolesStmt(true, null); :}
;
show_grant_principal_stmt ::=
KW_SHOW KW_GRANT principal_type:type ident_or_default:name
{: RESULT = new ShowGrantPrincipalStmt(name, type, null); :}
| KW_SHOW KW_GRANT principal_type:type ident_or_default:name KW_ON
server_ident:server_kw
{:
RESULT = new ShowGrantPrincipalStmt(name, type,
PrivilegeSpec.createServerScopedPriv(TPrivilegeLevel.ALL));
:}
| KW_SHOW KW_GRANT principal_type:type ident_or_default:name KW_ON
KW_DATABASE ident_or_default:db_name
{:
RESULT = new ShowGrantPrincipalStmt(name, type,
PrivilegeSpec.createDbScopedPriv(TPrivilegeLevel.ALL, db_name));
:}
| KW_SHOW KW_GRANT principal_type:type ident_or_default:name KW_ON KW_TABLE
table_name:tbl_name
{:
RESULT = new ShowGrantPrincipalStmt(name, type,
PrivilegeSpec.createTableScopedPriv(TPrivilegeLevel.ALL, tbl_name));
:}
| KW_SHOW KW_GRANT principal_type:type ident_or_default:name KW_ON KW_COLUMN
column_name:col_name
{:
RESULT = new ShowGrantPrincipalStmt(name, type,
PrivilegeSpec.createColumnScopedPriv(TPrivilegeLevel.SELECT,
col_name.getTableName(),
Collections.singletonList(col_name.getColumnName())));
:}
| KW_SHOW KW_GRANT principal_type:type ident_or_default:name KW_ON uri_ident:uri_kw
STRING_LITERAL:uri
{:
RESULT = new ShowGrantPrincipalStmt(name, type,
PrivilegeSpec.createUriScopedPriv(TPrivilegeLevel.ALL, new HdfsUri(uri)));
:}
;
create_drop_role_stmt ::=
KW_CREATE KW_ROLE ident_or_default:role
{: RESULT = new CreateDropRoleStmt(role, false); :}
| KW_DROP KW_ROLE ident_or_default:role
{: RESULT = new CreateDropRoleStmt(role, true); :}
;
grant_role_stmt ::=
KW_GRANT KW_ROLE ident_or_default:role KW_TO KW_GROUP ident_or_default:group
{: RESULT = new GrantRevokeRoleStmt(role, group, true); :}
;
revoke_role_stmt ::=
KW_REVOKE KW_ROLE ident_or_default:role KW_FROM KW_GROUP ident_or_default:group
{: RESULT = new GrantRevokeRoleStmt(role, group, false); :}
;
// For backwards compatibility, a grant without the principal type will default to
// TPrincipalType.ROLE
grant_privilege_stmt ::=
KW_GRANT privilege_spec:priv KW_TO KW_ROLE ident_or_default:role
opt_with_grantopt:grant_opt
{: RESULT = new GrantRevokePrivStmt(role, priv, true, grant_opt, TPrincipalType.ROLE); :}
| KW_GRANT privilege_spec:priv KW_TO ident_or_default:role
opt_with_grantopt:grant_opt
{: RESULT = new GrantRevokePrivStmt(role, priv, true, grant_opt, TPrincipalType.ROLE); :}
| KW_GRANT privilege_spec:priv KW_TO IDENT:user_id ident_or_default:user
opt_with_grantopt:grant_opt
{:
parser.checkIdentKeyword("USER", user_id);
RESULT = new GrantRevokePrivStmt(user, priv, true, grant_opt, TPrincipalType.USER);
:}
| KW_GRANT privilege_spec:priv KW_TO KW_GROUP ident_or_default:group
opt_with_grantopt:grant_opt
{: RESULT = new GrantRevokePrivStmt(group, priv, true, grant_opt, TPrincipalType.GROUP); :}
;
// For backwards compatibility, a revoke without the principal type will default to
// TPrincipalType.ROLE
revoke_privilege_stmt ::=
KW_REVOKE opt_grantopt_for:grant_opt privilege_spec:priv KW_FROM
KW_ROLE ident_or_default:role
{: RESULT = new GrantRevokePrivStmt(role, priv, false, grant_opt, TPrincipalType.ROLE); :}
| KW_REVOKE opt_grantopt_for:grant_opt privilege_spec:priv KW_FROM
ident_or_default:role
{: RESULT = new GrantRevokePrivStmt(role, priv, false, grant_opt, TPrincipalType.ROLE); :}
| KW_REVOKE opt_grantopt_for:grant_opt privilege_spec:priv KW_FROM
IDENT:user_id ident_or_default:user
{:
parser.checkIdentKeyword("USER", user_id);
RESULT = new GrantRevokePrivStmt(user, priv, false, grant_opt, TPrincipalType.USER);
:}
| KW_REVOKE opt_grantopt_for:grant_opt privilege_spec:priv KW_FROM
KW_GROUP ident_or_default:group
{: RESULT = new GrantRevokePrivStmt(group, priv, false, grant_opt, TPrincipalType.GROUP); :}
;
privilege_spec ::=
privilege:priv KW_ON server_ident:server_kw
{: RESULT = PrivilegeSpec.createServerScopedPriv(priv); :}
| privilege:priv KW_ON server_ident:server_kw ident_or_default:server_name
{: RESULT = PrivilegeSpec.createServerScopedPriv(priv, server_name); :}
| privilege:priv KW_ON KW_DATABASE ident_or_default:db_name
{: RESULT = PrivilegeSpec.createDbScopedPriv(priv, db_name); :}
| privilege:priv KW_ON KW_TABLE table_name:tbl_name
{: RESULT = PrivilegeSpec.createTableScopedPriv(priv, tbl_name); :}
| privilege:priv LPAREN opt_ident_list:cols RPAREN KW_ON KW_TABLE table_name:tbl_name
{: RESULT = PrivilegeSpec.createColumnScopedPriv(priv, tbl_name, cols); :}
| privilege:priv KW_ON uri_ident:uri_kw STRING_LITERAL:uri
{: RESULT = PrivilegeSpec.createUriScopedPriv(priv, new HdfsUri(uri)); :}
;
privilege ::=
KW_SELECT
{: RESULT = TPrivilegeLevel.SELECT; :}
| KW_INSERT
{: RESULT = TPrivilegeLevel.INSERT; :}
| KW_REFRESH
{: RESULT = TPrivilegeLevel.REFRESH; :}
| KW_CREATE
{: RESULT = TPrivilegeLevel.CREATE; :}
| KW_ALTER
{: RESULT = TPrivilegeLevel.ALTER; :}
| KW_DROP
{: RESULT = TPrivilegeLevel.DROP; :}
| KW_ALL
{: RESULT = TPrivilegeLevel.ALL; :}
;
principal_type ::=
KW_ROLE
{: RESULT = TPrincipalType.ROLE; :}
| KW_GROUP
{: RESULT = TPrincipalType.GROUP; :}
| IDENT:user
{:
parser.checkIdentKeyword("USER", user);
RESULT = TPrincipalType.USER;
:}
;
opt_grantopt_for ::=
KW_GRANT option_ident:option KW_FOR
{: RESULT = true; :}
| /* empty */
{: RESULT = false; :}
;
opt_with_grantopt ::=
KW_WITH KW_GRANT option_ident:option
{: RESULT = true; :}
| /* empty */
{: RESULT = false; :}
;
partition_def ::=
partition_spec:partition location_val:location opt_cache_op_val:cache_op
{: RESULT = new PartitionDef(partition, location, cache_op); :}
;
partition_def_list ::=
partition_def:item
{:
List<PartitionDef> list = Lists.newArrayList(item);
RESULT = list;
:}
| partition_def_list:list partition_def:item
{:
list.add(item);
RESULT = list;
:}
;
comment_on_stmt ::=
KW_COMMENT KW_ON KW_DATABASE ident_or_default:db_name KW_IS nullable_comment_val:comment
{: RESULT = new CommentOnDbStmt(db_name, comment); :}
| KW_COMMENT KW_ON KW_TABLE table_name:table KW_IS nullable_comment_val:comment
{: RESULT = new CommentOnTableStmt(table, comment); :}
| KW_COMMENT KW_ON KW_VIEW table_name:table KW_IS nullable_comment_val:comment
{: RESULT = new CommentOnViewStmt(table, comment); :}
| KW_COMMENT KW_ON KW_COLUMN column_name:column KW_IS nullable_comment_val:comment
{: RESULT = new CommentOnColumnStmt(column, comment); :}
;
// Introducing OWNER and USER keywords has a potential to be a breaking change,
// such that any names that use OWNER or USER will need to be escaped. By using IDENT
// token we can make OWNER and USER to be keywords only in these statements.
alter_db_stmt ::=
KW_ALTER KW_DATABASE ident_or_default:db KW_SET IDENT:owner_id IDENT:user_id
ident_or_default:user
{:
parser.checkIdentKeyword("OWNER", owner_id);
parser.checkIdentKeyword("USER", user_id);
RESULT = new AlterDbSetOwnerStmt(db, new Owner(TOwnerType.USER, user));
:}
| KW_ALTER KW_DATABASE ident_or_default:db KW_SET IDENT:owner_id KW_ROLE
ident_or_default:role
{:
parser.checkIdentKeyword("OWNER", owner_id);
RESULT = new AlterDbSetOwnerStmt(db, new Owner(TOwnerType.ROLE, role));
:}
;
// In some places, the opt_partition_set is used to avoid conflicts even though
// a partition clause does not make sense for this stmt. If a partition
// is given, manually throw a parse error.
alter_tbl_stmt ::=
KW_ALTER KW_TABLE table_name:table KW_ADD KW_COLUMN if_not_exists_val:if_not_exists
column_def:col_def
{:
List<ColumnDef> list = new ArrayList<>();
list.add(col_def);
RESULT = new AlterTableAddColsStmt(table, if_not_exists, list);
:}
| KW_ALTER KW_TABLE table_name:table KW_ADD if_not_exists_val:if_not_exists KW_COLUMNS
LPAREN column_def_list:col_defs RPAREN
{: RESULT = new AlterTableAddColsStmt(table, if_not_exists, col_defs); :}
| KW_ALTER KW_TABLE table_name:table KW_REPLACE KW_COLUMNS
LPAREN column_def_list:col_defs RPAREN
{: RESULT = new AlterTableReplaceColsStmt(table, col_defs); :}
| KW_ALTER KW_TABLE table_name:table KW_ADD if_not_exists_val:if_not_exists
partition_def_list:partitions
{: RESULT = new AlterTableAddPartitionStmt(table, if_not_exists, partitions); :}
| KW_ALTER KW_TABLE table_name:table KW_DROP opt_kw_column ident_or_default:col_name
{: RESULT = new AlterTableDropColStmt(table, col_name); :}
| KW_ALTER KW_TABLE table_name:table KW_ADD if_not_exists_val:if_not_exists
KW_RANGE range_param:partition
{:
RESULT = new AlterTableAddDropRangePartitionStmt(table, partition, if_not_exists,
Operation.ADD);
:}
| KW_ALTER KW_TABLE table_name:table KW_CHANGE opt_kw_column ident_or_default:col_name
column_def:col_def
{: RESULT = AlterTableAlterColStmt.createChangeColStmt(table, col_name, col_def); :}
| KW_ALTER KW_TABLE table_name:table KW_DROP if_exists_val:if_exists
partition_set:partitions purge_val:purge
{: RESULT = new AlterTableDropPartitionStmt(table, partitions, if_exists, purge); :}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET KW_FILEFORMAT
file_format_val:file_format
{: RESULT = new AlterTableSetFileFormatStmt(table, partitions, file_format); :}
| KW_ALTER KW_TABLE table_name:table KW_DROP if_exists_val:if_exists
KW_RANGE range_param:partition
{:
RESULT = new AlterTableAddDropRangePartitionStmt(table, partition, if_exists,
Operation.DROP);
:}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET
KW_LOCATION STRING_LITERAL:location
{:
// Need to check in analysis that the partition set only matches a single partition.
// Avoids a reduce/reduce conflict and allows the user to select a partition without
// fully specifying all partition-key values.
RESULT = new AlterTableSetLocationStmt(table, partitions, new HdfsUri(location));
:}
| KW_ALTER KW_TABLE table_name:table KW_RENAME KW_TO table_name:new_table
{: RESULT = new AlterTableOrViewRenameStmt(table, new_table, true); :}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET
table_property_type:target LPAREN properties_map:properties RPAREN
{: RESULT = new AlterTableSetTblProperties(table, partitions, target, properties); :}
| KW_ALTER KW_TABLE table_name:table KW_SORT KW_BY LPAREN opt_ident_list:col_names
RPAREN
{: RESULT = new AlterTableSortByStmt(table, col_names, TSortingOrder.LEXICAL); :}
| KW_ALTER KW_TABLE table_name:table KW_SORT KW_BY KW_LEXICAL LPAREN opt_ident_list:col_names
RPAREN
{: RESULT = new AlterTableSortByStmt(table, col_names, TSortingOrder.LEXICAL); :}
| KW_ALTER KW_TABLE table_name:table KW_SORT KW_BY KW_ZORDER LPAREN opt_ident_list:col_names
RPAREN
{:
if (!BackendConfig.INSTANCE.isZOrderSortUnlocked()) {
throw new NotImplementedException("Z-ordering is not yet implemented");
}
RESULT = new AlterTableSortByStmt(table, col_names, TSortingOrder.ZORDER);
:}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partition KW_SET
KW_COLUMN KW_STATS ident_or_default:col LPAREN properties_map:map RPAREN
{:
// See above for special partition clause handling.
if (partition != null) parser.parseError("set", SqlParserSymbols.KW_SET);
RESULT = new AlterTableSetColumnStats(table, col, map);
:}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partition KW_SET
row_format_val:row_format
{:
RESULT = new AlterTableSetRowFormatStmt(table, partition, row_format);
:}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET
cache_op_val:cache_op
{:
RESULT = new AlterTableSetCachedStmt(table, partitions, cache_op);
:}
| KW_ALTER KW_TABLE table_name:table KW_RECOVER KW_PARTITIONS
{: RESULT = new AlterTableRecoverPartitionsStmt(table); :}
| KW_ALTER KW_TABLE table_name:table KW_ALTER opt_kw_column ident_or_default:col_name
KW_SET column_options_map:options
{:
RESULT = new AlterTableAlterColStmt(
table, col_name, new ColumnDef(col_name, null, options));
:}
| KW_ALTER KW_TABLE table_name:table KW_ALTER opt_kw_column ident_or_default:col_name
KW_DROP KW_DEFAULT
{: RESULT = AlterTableAlterColStmt.createDropDefaultStmt(table, col_name); :}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET IDENT:owner_id
IDENT:user_id ident_or_default:user
{:
// See above for special partition clause handling.
if (partitions != null) parser.parseError("set", SqlParserSymbols.KW_SET);
parser.checkIdentKeyword("OWNER", owner_id);
parser.checkIdentKeyword("USER", user_id);
RESULT = new AlterTableSetOwnerStmt(table, new Owner(TOwnerType.USER, user));
:}
| KW_ALTER KW_TABLE table_name:table opt_partition_set:partitions KW_SET IDENT:owner_id
KW_ROLE ident_or_default:role
{:
// See above for special partition clause handling.
if (partitions != null) parser.parseError("set", SqlParserSymbols.KW_SET);
parser.checkIdentKeyword("OWNER", owner_id);
RESULT = new AlterTableSetOwnerStmt(table, new Owner(TOwnerType.ROLE, role));
:}
;
table_property_type ::=
KW_TBLPROPERTIES
{: RESULT = TTablePropertyType.TBL_PROPERTY; :}
| KW_SERDEPROPERTIES
{: RESULT = TTablePropertyType.SERDE_PROPERTY; :}
;
opt_kw_column ::=
KW_COLUMN
| /* empty */
;
create_db_stmt ::=
KW_CREATE db_or_schema_kw if_not_exists_val:if_not_exists ident_or_default:db_name
opt_comment_val:comment location_val:location
{: RESULT = new CreateDbStmt(db_name, comment, location, if_not_exists); :}
;
// Merging the two cases using opt_plan_hints would lead to reduce-reduce conflict,
// because if there are no hints, CTAS cannot be distinguished from normal CREATE
// statements until the AS SELECT part, but the decision whether to reduce empty string
// to opt_plan_hints must happen before reaching that part.
create_tbl_as_select_stmt ::=
KW_CREATE create_tbl_as_select_params:ctas_params
{:
RESULT = new CreateTableAsSelectStmt(ctas_params, null);
:}
| KW_CREATE plan_hints:hints create_tbl_as_select_params:ctas_params
{:
RESULT = new CreateTableAsSelectStmt(ctas_params, hints);
:}
;
create_tbl_as_select_params ::=
tbl_def_without_col_defs:tbl_def
tbl_options:options
KW_AS query_stmt:select_stmt
{:
tbl_def.setOptions(options);
RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
select_stmt, null);
:}
| tbl_def_without_col_defs:tbl_def
// An optional clause cannot be used directly below because it would conflict with
// the first rule in "create_tbl_stmt".
primary_keys:primary_keys
partitioned_data_layout:partition_params
tbl_options:options
KW_AS query_stmt:select_stmt
{:
tbl_def.getPrimaryKeyColumnNames().addAll(primary_keys);
tbl_def.getKuduPartitionParams().addAll(partition_params.getKuduPartitionParams());
tbl_def.setOptions(options);
RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
select_stmt, null);
:}
| tbl_def_without_col_defs:tbl_def
KW_PARTITIONED KW_BY LPAREN ident_list:partition_cols RPAREN
tbl_options:options
KW_AS query_stmt:select_stmt
{:
tbl_def.setOptions(options);
RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
select_stmt, partition_cols);
:}
;
create_tbl_stmt ::=
KW_CREATE tbl_def_without_col_defs:tbl_def
tbl_options:options
{:
tbl_def.setOptions(options);
RESULT = new CreateTableStmt(tbl_def);
:}
| KW_CREATE tbl_def_without_col_defs:tbl_def
// If "opt_tbl_data_layout" were used instead so that this rule could be combined with
// the rule above, there would be a conflict with the first rule in
// "create_tbl_as_select_stmt".
partition_column_defs:partition_column_defs
tbl_options:options
{:
tbl_def.setOptions(options);
CreateTableStmt create_tbl_stmt = new CreateTableStmt(tbl_def);
create_tbl_stmt.getPartitionColumnDefs().addAll(partition_column_defs);
RESULT = create_tbl_stmt;
:}
| KW_CREATE tbl_def_with_col_defs:tbl_def
opt_tbl_data_layout:data_layout
tbl_options:options
{:
tbl_def.getPartitionColumnDefs().addAll(data_layout.getPartitionColumnDefs());
tbl_def.getKuduPartitionParams().addAll(data_layout.getKuduPartitionParams());
tbl_def.setOptions(options);
RESULT = new CreateTableStmt(tbl_def);
:}
| KW_CREATE tbl_def_with_col_defs:tbl_def
KW_PRODUCED KW_BY KW_DATA source_ident:is_source_id ident_or_default:data_src_name
opt_init_string_val:init_string
opt_comment_val:comment
{:
// Need external_val in the grammar to avoid shift/reduce conflict with other
// CREATE TABLE statements.
if (tbl_def.isExternal()) {
parser.parseError("external", SqlParserSymbols.KW_EXTERNAL);
}
tbl_def.setOptions(new TableDef.Options(comment, parser.getQueryOptions()));
RESULT = new CreateTableDataSrcStmt(new CreateTableStmt(tbl_def),
data_src_name, init_string);
:}
| KW_CREATE tbl_def_without_col_defs:tbl_def
KW_LIKE file_format_val:schema_file_format
STRING_LITERAL:schema_location
opt_tbl_data_layout:data_layout
tbl_options:options
{:
tbl_def.getPartitionColumnDefs().addAll(data_layout.getPartitionColumnDefs());
tbl_def.getKuduPartitionParams().addAll(data_layout.getKuduPartitionParams());
tbl_def.setOptions(options);
RESULT = new CreateTableLikeFileStmt(new CreateTableStmt(tbl_def),
schema_file_format, new HdfsUri(schema_location));
:}
;
// The form of CREATE TABLE below should logically be grouped with the forms above but
// 'create_tbl_stmt' must return a CreateTableStmt instance and CreateTableLikeFileStmt
// class doesn't inherit from CreateTableStmt.
// TODO: Refactor the CREATE TABLE statements to improve the grammar and the way we
// handle table options.
create_tbl_like_stmt ::=
KW_CREATE tbl_def_without_col_defs:tbl_def
KW_LIKE table_name:other_table
opt_comment_val:comment
file_format_create_table_val:file_format location_val:location
{:
RESULT = new CreateTableLikeStmt(tbl_def.getTblName(),
new Pair<>(null, TSortingOrder.LEXICAL), other_table, tbl_def.isExternal(),
comment, file_format, location, tbl_def.getIfNotExists());
:}
// This extra production is necessary since without it the parser will not be able to
// parse "CREATE TABLE A LIKE B".
| KW_CREATE tbl_def_without_col_defs:tbl_def
opt_sort_cols:sort_cols
KW_LIKE table_name:other_table
opt_comment_val:comment
file_format_create_table_val:file_format location_val:location
{:
RESULT = new CreateTableLikeStmt(tbl_def.getTblName(), sort_cols, other_table,
tbl_def.isExternal(), comment, file_format, location, tbl_def.getIfNotExists());
:}
;
// Used for creating tables where the schema is inferred externally, e.g., from an Avro
// schema, Kudu table or query statement.
tbl_def_without_col_defs ::=
external_val:external KW_TABLE
if_not_exists_val:if_not_exists table_name:table
{: RESULT = new TableDef(table, external, if_not_exists); :}
;
tbl_def_with_col_defs ::=
tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
RESULT = tbl_def;
:}
// Adding Primary Keys and Foreign Keys constraints
// PRIMARY KEY(..) DISABLE NOVALIDATE RELY
| tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA
primary_keys:primary_keys RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
tbl_def.getPrimaryKeyColumnNames().addAll(primary_keys);
TableDef.PrimaryKey pk = new TableDef.PrimaryKey(tbl_def.getTblName(),
primary_keys, null, true, false, false);
tbl_def.setPrimaryKey(pk);
RESULT = tbl_def;
:}
| tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA
primary_keys:primary_keys enable_spec:enable_spec validate_spec:validate_spec
rely_spec:rely_spec RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
TableDef.PrimaryKey pk = new TableDef.PrimaryKey(tbl_def.getTblName(),
primary_keys, null, rely_spec, validate_spec, enable_spec);
tbl_def.setPrimaryKey(pk);
RESULT = tbl_def;
:}
// PRIMARY KEY(..) DISABLE NOVALIDATE RELY FOREIGN KEY (...) REFERENCES parent_table(..)
| tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA
primary_keys:primary_keys enable_spec:enable_spec validate_spec:validate_spec
rely_spec:rely_spec COMMA foreign_keys_list:foreign_keys_list RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
TableDef.PrimaryKey pk = new TableDef.PrimaryKey(tbl_def.getTblName(),
primary_keys, null, rely_spec, validate_spec, enable_spec);
tbl_def.setPrimaryKey(pk);
tbl_def.getForeignKeysList().addAll(foreign_keys_list);
RESULT = tbl_def;
:}
// FOREIGN KEYS followed by PRIMARY KEYS
| tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA
foreign_keys_list:foreign_keys_list COMMA primary_keys:primary_keys
enable_spec:enable_spec validate_spec:validate_spec rely_spec:rely_spec RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
TableDef.PrimaryKey pk = new TableDef.PrimaryKey(tbl_def.getTblName(),
primary_keys, null, rely_spec, validate_spec, enable_spec);
tbl_def.setPrimaryKey(pk);
tbl_def.getForeignKeysList().addAll(foreign_keys_list);
RESULT = tbl_def;
:}
| tbl_def_without_col_defs:tbl_def LPAREN column_def_list:list COMMA
foreign_keys_list:foreign_keys_list RPAREN
{:
tbl_def.getColumnDefs().addAll(list);
tbl_def.getForeignKeysList().addAll(foreign_keys_list);
RESULT = tbl_def;
:}
;
foreign_keys_list ::=
KW_FOREIGN key_ident LPAREN ident_list:fk_col_names RPAREN KW_REFERENCES
table_name:parent_tbl_name LPAREN ident_list:pk_col_names RPAREN
enable_spec:fk_enable_spec validate_spec:fk_validate_spec rely_spec:fk_rely_spec
{:
List<TableDef.ForeignKey> fk_list = new ArrayList<TableDef.ForeignKey>();
fk_list.add(new TableDef.ForeignKey(parent_tbl_name, pk_col_names, fk_col_names, null,
fk_rely_spec, fk_validate_spec, fk_enable_spec));
RESULT = fk_list;
:}
| foreign_keys_list:fk_list COMMA KW_FOREIGN key_ident LPAREN ident_list:fk_col_names
RPAREN KW_REFERENCES table_name:parent_tbl_name LPAREN ident_list:pk_col_names
RPAREN enable_spec:fk_enable_spec validate_spec:fk_validate_spec
rely_spec:fk_rely_spec
{:
fk_list.add(new TableDef.ForeignKey(parent_tbl_name, pk_col_names, fk_col_names, null,
fk_rely_spec, fk_validate_spec, fk_enable_spec));
RESULT = fk_list;
:}
;
primary_keys ::=
KW_PRIMARY key_ident LPAREN ident_list:col_names RPAREN
{: RESULT = col_names; :}
;
rely_spec ::=
KW_RELY
{: RESULT = true; :}
| KW_NORELY
{: RESULT = false; :}
| // Empty
{: RESULT = false; :}
;
validate_spec ::=
KW_VALIDATE
{: RESULT = true; :}
| KW_NOVALIDATE
{: RESULT = false; :}
| //Empty
{: RESULT = false; :}
;
enable_spec ::=
KW_ENABLE
{: RESULT = true; :}
| KW_DISABLE
{: RESULT = false; :}
| //Empty
{: RESULT = false; :}
;
tbl_options ::=
opt_sort_cols:sort_cols opt_comment_val:comment opt_row_format_val:row_format
serde_properties:serde_props file_format_create_table_val:file_format
location_val:location opt_cache_op_val:cache_op
tbl_properties:tbl_props
{:
CreateTableStmt.unescapeProperties(serde_props);
CreateTableStmt.unescapeProperties(tbl_props);
RESULT = new TableDef.Options(sort_cols, comment, row_format, serde_props,
file_format, location, cache_op, tbl_props, parser.getQueryOptions());
:}
;
opt_sort_cols ::=
KW_SORT KW_BY LPAREN opt_ident_list:col_names RPAREN
{:
RESULT = new Pair<List<String>, TSortingOrder>(
col_names, TSortingOrder.LEXICAL);
:}
| KW_SORT KW_BY KW_LEXICAL LPAREN opt_ident_list:col_names RPAREN
{:
RESULT = new Pair<List<String>, TSortingOrder>(
col_names, TSortingOrder.LEXICAL);
:}
| KW_SORT KW_BY KW_ZORDER LPAREN opt_ident_list:col_names RPAREN
{:
if (!BackendConfig.INSTANCE.isZOrderSortUnlocked()) {
throw new NotImplementedException("Z-ordering is not yet implemented");
}
RESULT = new Pair<List<String>, TSortingOrder>(
col_names, TSortingOrder.ZORDER);
:}
| /* empty */
{: RESULT = new Pair<List<String>, TSortingOrder>(
null, TSortingOrder.LEXICAL); :}
;
opt_tbl_data_layout ::=
partition_column_defs:partition_column_defs
{: RESULT = TableDataLayout.createPartitionedLayout(partition_column_defs); :}
| partitioned_data_layout:data_layout
{: RESULT = data_layout; :}
;
partitioned_data_layout ::=
partition_param_list:partition_params
{: RESULT = TableDataLayout.createKuduPartitionedLayout(partition_params); :}
| /* empty */
{: RESULT = TableDataLayout.createEmptyLayout(); :}
;
partition_column_defs ::=
KW_PARTITIONED KW_BY LPAREN column_def_list:col_defs RPAREN
{: RESULT = col_defs; :}
;
// The PARTITION BY clause contains any number of HASH() clauses followed by exactly zero
// or one RANGE clauses
partition_param_list ::=
KW_PARTITION KW_BY hash_partition_param_list:list
{: RESULT = list; :}
| KW_PARTITION KW_BY range_partition_param:rng
{: RESULT = Lists.newArrayList(rng); :}
| KW_PARTITION KW_BY hash_partition_param_list:list COMMA range_partition_param:rng
{:
list.add(rng);
RESULT = list;
:}
;
// A list of HASH partitioning clauses used for flexible partitioning
hash_partition_param_list ::=
hash_partition_param:dc
{: RESULT = Lists.newArrayList(dc); :}
| hash_partition_param_list:list COMMA hash_partition_param:d
{:
list.add(d);
RESULT = list;
:}
;
// The column list for a HASH clause is optional.
hash_partition_param ::=
KW_HASH LPAREN ident_list:cols RPAREN KW_PARTITIONS INTEGER_LITERAL:numPartitions
{: RESULT = KuduPartitionParam.createHashParam(cols, numPartitions.intValue()); :}
| KW_HASH KW_PARTITIONS INTEGER_LITERAL:numPartitions
{:
RESULT = KuduPartitionParam.createHashParam(new ArrayList<>(),
numPartitions.intValue());
:}
;
// The column list for a RANGE clause is optional.
range_partition_param ::=
KW_RANGE LPAREN ident_list:cols RPAREN LPAREN range_params_list:ranges RPAREN
{:
RESULT = KuduPartitionParam.createRangeParam(cols, ranges);
:}
| KW_RANGE LPAREN range_params_list:ranges RPAREN
{:
RESULT = KuduPartitionParam.createRangeParam(Collections.<String>emptyList(), ranges);
:}
;
range_params_list ::=
range_param:param
{:
RESULT = Lists.<RangePartition>newArrayList(param);
:}
| range_params_list:list COMMA range_param:param
{:
list.add(param);
RESULT = list;
:}
;
range_param ::=
KW_PARTITION opt_lower_range_val:lower_val KW_VALUES opt_upper_range_val:upper_val
{: RESULT = RangePartition.createFromRange(lower_val, upper_val); :}
// Use dotted_path to avoid reduce/reduce conflicts with expr
| KW_PARTITION dotted_path:val EQUAL expr:l
{:
if (!val.get(0).toUpperCase().equals("VALUE")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "VALUE");
}
RESULT = RangePartition.createFromValues(Lists.newArrayList(l)); :}
| KW_PARTITION dotted_path:val EQUAL LPAREN expr_list:l RPAREN
{:
if (!val.get(0).toUpperCase().equals("VALUE")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "VALUE");
}
RESULT = RangePartition.createFromValues(l);
:}
;
opt_lower_range_val ::=
expr:l LESSTHAN
{: RESULT = new Pair<List<Expr>, Boolean>(Lists.newArrayList(l), false); :}
| expr:l LESSTHAN EQUAL
{: RESULT = new Pair<List<Expr>, Boolean>(Lists.newArrayList(l), true); :}
| LPAREN expr_list:l RPAREN LESSTHAN
{: RESULT = new Pair<List<Expr>, Boolean>(l, false); :}
| LPAREN expr_list:l RPAREN LESSTHAN EQUAL
{: RESULT = new Pair<List<Expr>, Boolean>(l, true); :}
| /* empty */
{: RESULT = null; :}
;
opt_upper_range_val ::=
LESSTHAN expr:l
{: RESULT = new Pair<List<Expr>, Boolean>(Lists.newArrayList(l), false); :}
| LESSTHAN EQUAL expr:l
{: RESULT = new Pair<List<Expr>, Boolean>(Lists.newArrayList(l), true); :}
| LESSTHAN LPAREN expr_list:l RPAREN
{: RESULT = new Pair<List<Expr>, Boolean>(l, false); :}
| LESSTHAN EQUAL LPAREN expr_list:l RPAREN
{: RESULT = new Pair<List<Expr>, Boolean>(l, true); :}
| /* empty */
{: RESULT = null; :}
;
create_udf_stmt ::=
KW_CREATE KW_FUNCTION if_not_exists_val:if_not_exists
function_name:fn_name function_def_args:fn_args
KW_RETURNS type_def:return_type
KW_LOCATION STRING_LITERAL:binary_path
function_def_args_map:arg_map
{:
RESULT = new CreateUdfStmt(fn_name, fn_args, return_type, new HdfsUri(binary_path),
if_not_exists, arg_map);
:}
| KW_CREATE KW_FUNCTION if_not_exists_val:if_not_exists
function_name:fn_name KW_LOCATION STRING_LITERAL:binary_path
function_def_args_map:arg_map
{:
RESULT = new CreateUdfStmt(fn_name, null, null, new HdfsUri(binary_path),
if_not_exists, arg_map);
:}
;
create_uda_stmt ::=
KW_CREATE KW_AGGREGATE KW_FUNCTION if_not_exists_val:if_not_exists
function_name:fn_name function_def_args:fn_args
KW_RETURNS type_def:return_type
opt_aggregate_fn_intermediate_type_def:intermediate_type
KW_LOCATION STRING_LITERAL:binary_path
function_def_args_map:arg_map
{:
RESULT = new CreateUdaStmt(fn_name, fn_args, return_type, intermediate_type,
new HdfsUri(binary_path), if_not_exists, arg_map);
:}
;
opt_cache_op_val ::=
cache_op_val:cache_op
{: RESULT = cache_op; :}
| /* empty */
{: RESULT = null; :}
;
cache_op_val ::=
KW_CACHED KW_IN STRING_LITERAL:pool_name opt_cache_op_replication:replication
{: RESULT = new HdfsCachingOp(pool_name, replication); :}
| KW_UNCACHED
{: RESULT = new HdfsCachingOp(); :}
;
opt_cache_op_replication ::=
KW_WITH KW_REPLICATION EQUAL INTEGER_LITERAL:replication
{: RESULT = replication; :}
| /* empty */
{: RESULT = null; :}
;
comment_val ::=
KW_COMMENT STRING_LITERAL:comment
{: RESULT = comment; :}
;
opt_comment_val ::=
KW_COMMENT STRING_LITERAL:comment
{: RESULT = comment; :}
| /* empty */
{: RESULT = null; :}
;
nullable_comment_val ::=
STRING_LITERAL:comment
{: RESULT = comment; :}
| KW_NULL
{: RESULT = null; :}
;
location_val ::=
KW_LOCATION STRING_LITERAL:location
{: RESULT = new HdfsUri(location); :}
| /* empty */
{: RESULT = null; :}
;
opt_init_string_val ::=
LPAREN STRING_LITERAL:init_string RPAREN
{: RESULT = init_string; :}
| /* empty */
{: RESULT = null; :}
;
external_val ::=
KW_EXTERNAL
{: RESULT = true; :}
|
{: RESULT = false; :}
;
purge_val ::=
KW_PURGE
{: RESULT = true; :}
|
{: RESULT = false; :}
;
if_not_exists_val ::=
KW_IF KW_NOT KW_EXISTS
{: RESULT = true; :}
|
{: RESULT = false; :}
;
opt_row_format_val ::=
row_format_val:row_format
{: RESULT = row_format; :}
|/* empty */
{: RESULT = null; :}
;
row_format_val ::=
KW_ROW KW_FORMAT KW_DELIMITED field_terminator_val:field_terminator
escaped_by_val:escaped_by line_terminator_val:line_terminator
{: RESULT = new RowFormat(field_terminator, line_terminator, escaped_by); :}
;
escaped_by_val ::=
KW_ESCAPED KW_BY STRING_LITERAL:escaped_by
{: RESULT = escaped_by; :}
| /* empty */
{: RESULT = null; :}
;
line_terminator_val ::=
KW_LINES terminator_val:line_terminator
{: RESULT = line_terminator; :}
| /* empty */
{: RESULT = null; :}
;
field_terminator_val ::=
KW_FIELDS terminator_val:field_terminator
{: RESULT = field_terminator; :}
| /* empty */
{: RESULT = null; :}
;
terminator_val ::=
KW_TERMINATED KW_BY STRING_LITERAL:terminator
{: RESULT = terminator; :}
;
file_format_create_table_val ::=
KW_STORED KW_AS file_format_val:file_format
{: RESULT = file_format; :}
|
{: RESULT = null; :}
;
file_format_val ::=
KW_KUDU
{: RESULT = THdfsFileFormat.KUDU; :}
| KW_PARQUET
{: RESULT = THdfsFileFormat.PARQUET; :}
| KW_PARQUETFILE
{: RESULT = THdfsFileFormat.PARQUET; :}
| KW_ORC
{: RESULT = THdfsFileFormat.ORC; :}
| KW_TEXTFILE
{: RESULT = THdfsFileFormat.TEXT; :}
| KW_SEQUENCEFILE
{: RESULT = THdfsFileFormat.SEQUENCE_FILE; :}
| KW_RCFILE
{: RESULT = THdfsFileFormat.RC_FILE; :}
| KW_AVRO
{: RESULT = THdfsFileFormat.AVRO; :}
;
tbl_properties ::=
KW_TBLPROPERTIES LPAREN properties_map:map RPAREN
{: RESULT = map; :}
| /* empty */
{: RESULT = new LinkedHashMap<String, String>(); :}
;
serde_properties ::=
KW_WITH KW_SERDEPROPERTIES LPAREN properties_map:map RPAREN
{: RESULT = map; :}
| /* empty */
{: RESULT = new LinkedHashMap<String, String>(); :}
;
properties_map ::=
STRING_LITERAL:key EQUAL STRING_LITERAL:value
{:
LinkedHashMap<String, String> properties = new LinkedHashMap<String, String>();
properties.put(key, value);
RESULT = properties;
:}
| properties_map:properties COMMA STRING_LITERAL:key EQUAL STRING_LITERAL:value
{:
properties.put(key, value);
RESULT = properties;
:}
;
column_def_list ::=
column_def:col_def
{:
List<ColumnDef> list = new ArrayList<>();
list.add(col_def);
RESULT = list;
:}
| column_def_list:list COMMA column_def:col_def
{:
list.add(col_def);
RESULT = list;
:}
;
column_def ::=
ident_or_default:col_name type_def:type column_options_map:options
{: RESULT = new ColumnDef(col_name, type, options); :}
| ident_or_default:col_name type_def:type
{: RESULT = new ColumnDef(col_name, type); :}
;
column_options_map ::=
column_options_map:map column_option:col_option
{:
if (map.put(col_option.first, col_option.second) != null) {
throw new Exception(String.format("Column option %s is specified multiple times",
col_option.first.toString()));
}
RESULT = map;
:}
| column_option:col_option
{:
LinkedHashMap<Option, Object> options = Maps.newLinkedHashMap();
options.put(col_option.first, col_option.second);
RESULT = options;
:}
;
column_option ::=
is_primary_key_val:primary_key
{: RESULT = new Pair<Option, Object>(Option.IS_PRIMARY_KEY, primary_key); :}
| nullability_val:nullability
{: RESULT = new Pair<Option, Object>(Option.IS_NULLABLE, nullability); :}
| encoding_val:encoding
{: RESULT = new Pair<Option, Object>(Option.ENCODING, encoding); :}
| compression_val:compression
{: RESULT = new Pair<Option, Object>(Option.COMPRESSION, compression); :}
| default_val:default_val
{: RESULT = new Pair<Option, Object>(Option.DEFAULT, default_val); :}
| block_size_val:block_size
{: RESULT = new Pair<Option, Object>(Option.BLOCK_SIZE, block_size); :}
| comment_val:comment
{: RESULT = new Pair<Option, Object>(Option.COMMENT, comment); :}
;
is_primary_key_val ::=
KW_PRIMARY key_ident
{: RESULT = true; :}
;
nullability_val ::=
KW_NOT KW_NULL
{: RESULT = false; :}
| KW_NULL
{: RESULT = true; :}
;
encoding_val ::=
KW_ENCODING word:value
{: RESULT = value; :}
;
compression_val ::=
KW_COMPRESSION word:value
{: RESULT = value; :}
;
default_val ::=
KW_DEFAULT expr:default_val
{: RESULT = default_val; :}
;
block_size_val ::=
KW_BLOCKSIZE literal:block_size
{: RESULT = block_size; :}
;
create_view_stmt ::=
KW_CREATE KW_VIEW if_not_exists_val:if_not_exists table_name:view_name
view_column_defs:col_defs opt_comment_val:comment KW_AS query_stmt:view_def
{:
RESULT = new CreateViewStmt(if_not_exists, view_name, col_defs, comment, view_def);
:}
;
create_data_src_stmt ::=
KW_CREATE KW_DATA source_ident:is_source_id
if_not_exists_val:if_not_exists ident_or_default:data_src_name
KW_LOCATION STRING_LITERAL:location
KW_CLASS STRING_LITERAL:class_name
KW_API_VERSION STRING_LITERAL:api_version
{:
RESULT = new CreateDataSrcStmt(data_src_name, new HdfsUri(location), class_name,
api_version, if_not_exists);
:}
;
key_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("KEY")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "KEY");
}
:}
;
system_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("SYSTEM")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "SYSTEM");
}
:}
;
source_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("SOURCE")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "SOURCE");
}
RESULT = true;
:}
;
sources_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("SOURCES")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "SOURCES");
}
RESULT = true;
:}
;
uri_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("URI")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "URI");
}
RESULT = true;
:}
;
server_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("SERVER")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "SERVER");
}
RESULT = true;
:}
;
testcase_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("TESTCASE")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "TESTCASE");
}
:}
;
option_ident ::=
IDENT:ident
{:
if (!ident.toUpperCase().equals("OPTION")) {
parser.parseError("identifier", SqlParserSymbols.IDENT, "OPTION");
}
RESULT = true;
:}
;
view_column_defs ::=
LPAREN view_column_def_list:view_col_defs RPAREN
{: RESULT = view_col_defs; :}
| /* empty */
{: RESULT = null; :}
;
view_column_def_list ::=
view_column_def:col_def
{:
List<ColumnDef> list = new ArrayList<>();
list.add(col_def);
RESULT = list;
:}
| view_column_def_list:list COMMA view_column_def:col_def
{:
list.add(col_def);
RESULT = list;
:}
;
view_column_def ::=
ident_or_default:col_name opt_comment_val:comment
{:
LinkedHashMap<Option, Object> options = Maps.newLinkedHashMap();
if (comment != null) options.put(Option.COMMENT, comment);
RESULT = new ColumnDef(col_name, null, options);
:}
;
alter_view_stmt ::=
KW_ALTER KW_VIEW table_name:table view_column_defs:col_defs KW_AS
query_stmt:view_def
{: RESULT = new AlterViewStmt(table, col_defs, view_def); :}
| KW_ALTER KW_VIEW table_name:before_table KW_RENAME KW_TO table_name:new_table
{: RESULT = new AlterTableOrViewRenameStmt(before_table, new_table, false); :}
| KW_ALTER KW_VIEW table_name:table KW_SET IDENT:owner_id IDENT:user_id
ident_or_default:user
{:
parser.checkIdentKeyword("OWNER", owner_id);
parser.checkIdentKeyword("USER", user_id);
RESULT = new AlterViewSetOwnerStmt(table, new Owner(TOwnerType.USER, user));
:}
| KW_ALTER KW_VIEW table_name:table KW_SET IDENT:owner_id KW_ROLE ident_or_default:role
{:
parser.checkIdentKeyword("OWNER", owner_id);
RESULT = new AlterViewSetOwnerStmt(table, new Owner(TOwnerType.ROLE, role));
:}
;
cascade_val ::=
KW_CASCADE
{: RESULT = true; :}
| KW_RESTRICT
{: RESULT = false; :}
|
{: RESULT = false; :}
;
compute_stats_stmt ::=
KW_COMPUTE KW_STATS table_name:table opt_tablesample:tblsmpl
{: RESULT = ComputeStatsStmt.createStatsStmt(table, tblsmpl, null); :}
| KW_COMPUTE KW_STATS table_name:table LPAREN opt_ident_list:cols RPAREN
opt_tablesample:tblsmpl
{: RESULT = ComputeStatsStmt.createStatsStmt(table, tblsmpl, cols); :}
| KW_COMPUTE KW_INCREMENTAL KW_STATS table_name:table
{: RESULT = ComputeStatsStmt.createIncrementalStatsStmt(table, null); :}
| KW_COMPUTE KW_INCREMENTAL KW_STATS table_name:table partition_set:parts
{: RESULT = ComputeStatsStmt.createIncrementalStatsStmt(table, parts); :}
;
drop_stats_stmt ::=
KW_DROP KW_STATS table_name:table
{: RESULT = new DropStatsStmt(table); :}
| KW_DROP KW_INCREMENTAL KW_STATS table_name:table partition_set:partitions
{: RESULT = new DropStatsStmt(table, partitions); :}
;
drop_db_stmt ::=
KW_DROP db_or_schema_kw if_exists_val:if_exists ident_or_default:db_name
cascade_val:cascade
{: RESULT = new DropDbStmt(db_name, if_exists, cascade); :}
;
drop_tbl_or_view_stmt ::=
KW_DROP KW_TABLE if_exists_val:if_exists table_name:table purge_val:purge
{: RESULT = new DropTableOrViewStmt(table, if_exists, true, purge); :}
| KW_DROP KW_VIEW if_exists_val:if_exists table_name:table
{: RESULT = new DropTableOrViewStmt(table, if_exists, false, false); :}
;
drop_function_stmt ::=
KW_DROP opt_is_aggregate_fn:is_aggregate KW_FUNCTION
if_exists_val:if_exists function_name:fn_name
function_def_args:fn_args
{: RESULT = new DropFunctionStmt(fn_name, fn_args, if_exists); :}
| KW_DROP opt_is_aggregate_fn:is_aggregate KW_FUNCTION
if_exists_val:if_exists function_name:fn_name
{: RESULT = new DropFunctionStmt(fn_name, null, if_exists); :}
;
drop_data_src_stmt ::=
KW_DROP KW_DATA source_ident:is_source_id if_exists_val:if_exists
ident_or_default:data_src_name
{: RESULT = new DropDataSrcStmt(data_src_name, if_exists); :}
;
db_or_schema_kw ::=
KW_DATABASE
| KW_SCHEMA
;
dbs_or_schemas_kw ::=
KW_DATABASES
| KW_SCHEMAS
;
if_exists_val ::=
KW_IF KW_EXISTS
{: RESULT = true; :}
|
{: RESULT = false; :}
;
partition_clause ::=
KW_PARTITION LPAREN partition_key_value_list:list RPAREN
{: RESULT = list; :}
|
{: RESULT = null; :}
;
partition_key_value_list ::=
partition_key_value:item
{:
List<PartitionKeyValue> list = new ArrayList<>();
list.add(item);
RESULT = list;
:}
| partition_key_value_list:list COMMA partition_key_value:item
{:
list.add(item);
RESULT = list;
:}
;
// TODO: reuse this for INSERT statement.
// A partition set is a set of expressions used to select a list of partitions
// for certain operation such as DROP PARTITION. This is different than a partition
// clause in an INSERT statement. Partition clause contains dynamic and static
// partition key/values.
partition_set ::=
KW_PARTITION LPAREN expr_list:list RPAREN
{: RESULT = new PartitionSet(list); :}
;
opt_partition_set ::=
partition_set:partition_set
{: RESULT = partition_set; :}
| /* Empty */
{: RESULT = null; :}
;
// A partition spec is a set of static partition key/value pairs. This is a bit
// different than a partition clause in an INSERT statement because that allows
// for dynamic and static partition key/values.
partition_spec ::=
KW_PARTITION LPAREN static_partition_key_value_list:list RPAREN
{: RESULT = new PartitionSpec(list); :}
;
opt_partition_spec ::=
partition_spec:partition_spec
{: RESULT = partition_spec; :}
| /* Empty */
{: RESULT = null; :}
;
static_partition_key_value_list ::=
static_partition_key_value:item
{:
List<PartitionKeyValue> list = new ArrayList<>();
list.add(item);
RESULT = list;
:}
| static_partition_key_value_list:list COMMA static_partition_key_value:item
{:
list.add(item);
RESULT = list;
:}
;
partition_key_value ::=
// Dynamic partition key values.
ident_or_default:column
{: RESULT = new PartitionKeyValue(column, null); :}
| static_partition_key_value:partition
{: RESULT = partition; :}
;
static_partition_key_value ::=
// Static partition key values.
ident_or_default:column EQUAL expr:e
{: RESULT = new PartitionKeyValue(column, e); :}
;
function_def_args ::=
LPAREN RPAREN
{: RESULT = new FunctionArgs(); :}
| LPAREN function_def_arg_list:args opt_is_varargs:var_args RPAREN
{:
args.setHasVarArgs(var_args);
RESULT = args;
:}
;
function_def_arg_list ::=
type_def:type_def
{:
FunctionArgs args = new FunctionArgs();
args.getArgTypeDefs().add(type_def);
RESULT = args;
:}
| function_def_arg_list:args COMMA type_def:type_def
{:
args.getArgTypeDefs().add(type_def);
RESULT = args;
:}
;
opt_is_aggregate_fn ::=
KW_AGGREGATE
{: RESULT = true; :}
|
{: RESULT = false; :}
;
opt_is_varargs ::=
DOTDOTDOT
{: RESULT = true; :}
|
{: RESULT = false; :}
;
opt_aggregate_fn_intermediate_type_def ::=
KW_INTERMEDIATE type_def:type_def
{: RESULT = type_def; :}
|
{: RESULT = null; :}
;
function_def_args_map ::=
function_def_arg_key:key EQUAL STRING_LITERAL:value
{:
LinkedHashMap<CreateFunctionStmtBase.OptArg, String> args =
new LinkedHashMap<CreateFunctionStmtBase.OptArg, String>();
args.put(key, value);
RESULT = args;
:}
| function_def_args_map:args function_def_arg_key:key EQUAL STRING_LITERAL:value
{:
if (args.containsKey(key)) throw new Exception("Duplicate argument key: " + key);
args.put(key, value);
RESULT = args;
:}
|
{: RESULT = new LinkedHashMap<CreateFunctionStmtBase.OptArg, String>(); :}
;
// Any keys added here must also be added to the end of the precedence list.
function_def_arg_key ::=
KW_COMMENT
{: RESULT = CreateFunctionStmtBase.OptArg.COMMENT; :}
| KW_SYMBOL
{: RESULT = CreateFunctionStmtBase.OptArg.SYMBOL; :}
| KW_PREPARE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.PREPARE_FN; :}
| KW_CLOSE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.CLOSE_FN; :}
| KW_UPDATE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.UPDATE_FN; :}
| KW_INIT_FN
{: RESULT = CreateFunctionStmtBase.OptArg.INIT_FN; :}
| KW_SERIALIZE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.SERIALIZE_FN; :}
| KW_MERGE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.MERGE_FN; :}
| KW_FINALIZE_FN
{: RESULT = CreateFunctionStmtBase.OptArg.FINALIZE_FN; :}
;
// Our parsing of UNION is slightly different from MySQL's:
// http://dev.mysql.com/doc/refman/5.5/en/union.html
//
// Imo, MySQL's parsing of union is not very clear.
// For example, MySQL cannot parse this query:
// select 3 order by 1 limit 1 union all select 1;
//
// On the other hand, MySQL does parse this query, but associates
// the order by and limit with the union, not the select:
// select 3 as g union all select 1 order by 1 limit 2;
//
// MySQL also allows some combinations of select blocks
// with and without parenthesis, but also disallows others.
//
// Our parsing:
// Select blocks may or may not be in parenthesis,
// even if the union has order by and limit.
// ORDER BY and LIMIT bind to the preceding select statement by default.
query_stmt ::=
opt_with_clause:w union_operand_list:operands
{:
QueryStmt queryStmt = null;
if (operands.size() == 1) {
queryStmt = operands.get(0).getQueryStmt();
} else {
queryStmt = new UnionStmt(operands, null, null);
}
queryStmt.setWithClause(w);
RESULT = queryStmt;
:}
| opt_with_clause:w union_with_order_by_or_limit:union
{:
union.setWithClause(w);
RESULT = union;
:}
;
opt_with_clause ::=
KW_WITH with_view_def_list:list
{: RESULT = new WithClause(list); :}
| /* empty */
{: RESULT = null; :}
;
with_view_def ::=
ident_or_default:alias KW_AS LPAREN query_stmt:query RPAREN
{: RESULT = new View(alias, query, null); :}
| STRING_LITERAL:alias KW_AS LPAREN query_stmt:query RPAREN
{: RESULT = new View(alias, query, null); :}
| ident_or_default:alias LPAREN ident_list:col_names RPAREN KW_AS LPAREN
query_stmt:query RPAREN
{: RESULT = new View(alias, query, col_names); :}
| STRING_LITERAL:alias LPAREN ident_list:col_names RPAREN
KW_AS LPAREN query_stmt:query RPAREN
{: RESULT = new View(alias, query, col_names); :}
;
with_view_def_list ::=
with_view_def:v
{:
List<View> list = new ArrayList<>();
list.add(v);
RESULT = list;
:}
| with_view_def_list:list COMMA with_view_def:v
{:
list.add(v);
RESULT = list;
:}
;
// We must have a non-empty order by or limit for them to bind to the union.
// We cannot reuse the existing opt_order_by_clause or
// limit_clause because they would introduce conflicts with EOF,
// which, unfortunately, cannot be accessed in the parser as a nonterminal
// making this issue unresolvable.
// We rely on the left precedence of KW_ORDER, KW_BY, and KW_LIMIT,
// to resolve the ambiguity with select_stmt in favor of select_stmt
// (i.e., ORDER BY and LIMIT bind to the select_stmt by default, and not the union).
// There must be at least two union operands for ORDER BY or LIMIT to bind to a union,
// and we manually throw a parse error if we reach this production
// with only a single operand.
union_with_order_by_or_limit ::=
union_operand_list:operands
KW_ORDER KW_BY order_by_elements:orderByClause
opt_offset_param:offsetExpr
{:
if (operands.size() == 1) {
parser.parseError("order", SqlParserSymbols.KW_ORDER);
}
RESULT = new UnionStmt(operands, orderByClause, new LimitElement(null, offsetExpr));
:}
|
union_operand_list:operands
KW_LIMIT expr:limitExpr
{:
if (operands.size() == 1) {
parser.parseError("limit", SqlParserSymbols.KW_LIMIT);
}
RESULT = new UnionStmt(operands, null, new LimitElement(limitExpr, null));
:}
|
union_operand_list:operands
KW_ORDER KW_BY order_by_elements:orderByClause
KW_LIMIT expr:limitExpr opt_offset_param:offsetExpr
{:
if (operands.size() == 1) {
parser.parseError("order", SqlParserSymbols.KW_ORDER);
}
RESULT = new UnionStmt(operands, orderByClause,
new LimitElement(limitExpr, offsetExpr));
:}
;
union_operand ::=
select_stmt:select
{: RESULT = select; :}
| values_stmt:values
{: RESULT = values; :}
| LPAREN query_stmt:query RPAREN
{: RESULT = query; :}
;
union_operand_list ::=
union_operand:operand
{:
List<UnionOperand> operands = new ArrayList<>();
operands.add(new UnionOperand(operand, null));
RESULT = operands;
:}
| union_operand_list:operands union_op:op union_operand:operand
{:
operands.add(new UnionOperand(operand, op));
RESULT = operands;
:}
;
union_op ::=
KW_UNION
{: RESULT = Qualifier.DISTINCT; :}
| KW_UNION KW_DISTINCT
{: RESULT = Qualifier.DISTINCT; :}
| KW_UNION KW_ALL
{: RESULT = Qualifier.ALL; :}
;
values_stmt ::=
KW_VALUES values_operand_list:operands
opt_order_by_clause:orderByClause
opt_limit_offset_clause:limitOffsetClause
{:
RESULT = new ValuesStmt(operands, orderByClause, limitOffsetClause);
:}
| KW_VALUES LPAREN values_operand_list:operands RPAREN
opt_order_by_clause:orderByClause
opt_limit_offset_clause:limitOffsetClause
{:
RESULT = new ValuesStmt(operands, orderByClause, limitOffsetClause);
:}
;
values_operand_list ::=
LPAREN select_list:selectList RPAREN
{:
List<UnionOperand> operands = new ArrayList<>();
operands.add(new UnionOperand(
new SelectStmt(selectList, null, null, null, null, null, null), null));
RESULT = operands;
:}
| values_operand_list:operands COMMA LPAREN select_list:selectList RPAREN
{:
operands.add(new UnionOperand(
new SelectStmt(selectList, null, null, null, null, null, null), Qualifier.ALL));
RESULT = operands;
:}
;
use_stmt ::=
KW_USE ident_or_default:db
{: RESULT = new UseStmt(db); :}
;
show_tables_stmt ::=
KW_SHOW KW_TABLES
{: RESULT = new ShowTablesStmt(); :}
| KW_SHOW KW_TABLES show_pattern:showPattern
{: RESULT = new ShowTablesStmt(showPattern); :}
| KW_SHOW KW_TABLES KW_IN ident_or_default:db
{: RESULT = new ShowTablesStmt(db, null); :}
| KW_SHOW KW_TABLES KW_IN ident_or_default:db show_pattern:showPattern
{: RESULT = new ShowTablesStmt(db, showPattern); :}
;
show_dbs_stmt ::=
KW_SHOW dbs_or_schemas_kw
{: RESULT = new ShowDbsStmt(); :}
| KW_SHOW dbs_or_schemas_kw show_pattern:showPattern
{: RESULT = new ShowDbsStmt(showPattern); :}
;
show_stats_stmt ::=
KW_SHOW KW_TABLE KW_STATS table_name:table
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.TABLE_STATS); :}
| KW_SHOW KW_COLUMN KW_STATS table_name:table
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.COLUMN_STATS); :}
;
show_partitions_stmt ::=
KW_SHOW KW_PARTITIONS table_name:table
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.PARTITIONS); :}
;
show_range_partitions_stmt ::=
KW_SHOW KW_RANGE KW_PARTITIONS table_name:table
{: RESULT = new ShowStatsStmt(table, TShowStatsOp.RANGE_PARTITIONS); :}
;
show_functions_stmt ::=
KW_SHOW opt_function_category:fn_type KW_FUNCTIONS
{: RESULT = new ShowFunctionsStmt(null, null, fn_type); :}
| KW_SHOW opt_function_category:fn_type KW_FUNCTIONS show_pattern:showPattern
{: RESULT = new ShowFunctionsStmt(null, showPattern, fn_type); :}
| KW_SHOW opt_function_category:fn_type KW_FUNCTIONS KW_IN ident_or_default:db
{: RESULT = new ShowFunctionsStmt(db, null, fn_type); :}
| KW_SHOW opt_function_category:fn_type KW_FUNCTIONS KW_IN ident_or_default:db
show_pattern:showPattern
{: RESULT = new ShowFunctionsStmt(db, showPattern, fn_type); :}
;
opt_function_category ::=
KW_AGGREGATE
{: RESULT = TFunctionCategory.AGGREGATE; :}
| KW_ANALYTIC
{: RESULT = TFunctionCategory.ANALYTIC; :}
| /* empty */
{: RESULT = TFunctionCategory.SCALAR; :}
;
show_data_srcs_stmt ::=
KW_SHOW KW_DATA sources_ident:is_sources_id
{: RESULT = new ShowDataSrcsStmt(); :}
| KW_SHOW KW_DATA sources_ident:is_sources_id show_pattern:showPattern
{: RESULT = new ShowDataSrcsStmt(showPattern); :}
;
show_pattern ::=
STRING_LITERAL:showPattern
{: RESULT = showPattern; :}
| KW_LIKE STRING_LITERAL:showPattern
{: RESULT = showPattern; :}
;
show_create_tbl_stmt ::=
KW_SHOW KW_CREATE show_create_tbl_object_type:object_type table_name:table
{: RESULT = new ShowCreateTableStmt(table, object_type); :}
;
show_create_tbl_object_type ::=
KW_TABLE
{: RESULT = TCatalogObjectType.TABLE; :}
| KW_VIEW
{: RESULT = TCatalogObjectType.VIEW; :}
;
show_create_function_stmt ::=
KW_SHOW KW_CREATE KW_FUNCTION function_name:fn_name
{: RESULT = new ShowCreateFunctionStmt(fn_name, TFunctionCategory.SCALAR); :}
| KW_SHOW KW_CREATE KW_AGGREGATE KW_FUNCTION function_name:fn_name
{: RESULT = new ShowCreateFunctionStmt(fn_name, TFunctionCategory.AGGREGATE); :}
;
show_files_stmt ::=
KW_SHOW KW_FILES KW_IN table_name:table opt_partition_set:partitions
{: RESULT = new ShowFilesStmt(table, partitions); :}
;
describe_db_stmt ::=
KW_DESCRIBE db_or_schema_kw describe_output_style:style ident_or_default:db
{: RESULT = new DescribeDbStmt(db, style); :}
;
describe_table_stmt ::=
KW_DESCRIBE describe_output_style:style dotted_path:path
{: RESULT = new DescribeTableStmt(path, style); :}
;
describe_output_style ::=
KW_FORMATTED
{: RESULT = TDescribeOutputStyle.FORMATTED; :}
| KW_EXTENDED
{: RESULT = TDescribeOutputStyle.EXTENDED; :}
| /* empty */
{: RESULT = TDescribeOutputStyle.MINIMAL; :}
;
select_stmt ::=
select_clause:selectList
{:
RESULT = new SelectStmt(selectList, null, null, null, null, null, null);
:}
|
select_clause:selectList
from_clause:fromClause
where_clause:wherePredicate
group_by_clause:groupingExprs
having_clause:havingPredicate
opt_order_by_clause:orderByClause
opt_limit_offset_clause:limitOffsetClause
{:
RESULT = new SelectStmt(selectList, fromClause, wherePredicate, groupingExprs,
havingPredicate, orderByClause, limitOffsetClause);
:}
;
select_clause ::=
KW_SELECT opt_plan_hints:hints select_list:l
{:
l.setPlanHints(hints);
RESULT = l;
:}
| KW_SELECT KW_ALL opt_plan_hints:hints select_list:l
{:
l.setPlanHints(hints);
RESULT = l;
:}
| KW_SELECT KW_DISTINCT opt_plan_hints:hints select_list:l
{:
l.setIsDistinct(true);
l.setPlanHints(hints);
RESULT = l;
:}
;
set_stmt ::=
KW_SET ident_or_default:key EQUAL numeric_literal:l
{: RESULT = new SetStmt(key, l.getStringValue(), false); :}
| KW_SET ident_or_default:key EQUAL STRING_LITERAL:l
{: RESULT = new SetStmt(key, l, false); :}
| KW_SET ident_or_default:key EQUAL SUBTRACT numeric_literal:l
{:
l.swapSign();
RESULT = new SetStmt(key, l.getStringValue(), false); :}
| KW_SET ident_or_default:key EQUAL word:value
{: RESULT = new SetStmt(key, value, false); :}
| KW_SET KW_ALL
{: RESULT = new SetStmt(null, null, true); :}
| KW_SET
{: RESULT = new SetStmt(null, null, false); :}
;
// Top-level function call, e.g. ": shutdown()", used for admin commands, etc.
admin_fn_stmt ::=
COLON ident_or_default:fn_name LPAREN RPAREN
{: RESULT = new AdminFnStmt(fn_name, Collections.<Expr>emptyList()); :}
| COLON ident_or_default:fn_name LPAREN expr_list:params RPAREN
{: RESULT = new AdminFnStmt(fn_name, params); :}
;
select_list ::=
select_list_item:item
{:
SelectList list = new SelectList();
list.getItems().add(item);
RESULT = list;
:}
| select_list:list COMMA select_list_item:item
{:
list.getItems().add(item);
RESULT = list;
:}
;
select_list_item ::=
expr:expr alias_clause:alias
{: RESULT = new SelectListItem(expr, alias); :}
| expr:expr
{: RESULT = new SelectListItem(expr, null); :}
| star_expr:expr
{: RESULT = expr; :}
;
alias_clause ::=
KW_AS ident_or_default:ident
{: RESULT = ident; :}
| ident_or_default:ident
{: RESULT = ident; :}
| KW_AS STRING_LITERAL:l
{: RESULT = l; :}
| STRING_LITERAL:l
{: RESULT = l; :}
;
star_expr ::=
STAR
{: RESULT = SelectListItem.createStarItem(null); :}
| dotted_path:path DOT STAR
{: RESULT = SelectListItem.createStarItem(path); :}
;
table_name ::=
ident_or_default:tbl
{: RESULT = new TableName(null, tbl); :}
| ident_or_default:db DOT ident_or_default:tbl
{: RESULT = new TableName(db, tbl); :}
;
column_name ::=
| ident_or_default:tbl DOT ident_or_default:col
{: RESULT = new ColumnName(new TableName(null, tbl), col); :}
| ident_or_default:db DOT ident_or_default:tbl DOT ident_or_default:col
{: RESULT = new ColumnName(new TableName(db, tbl), col); :}
;
function_name ::=
// Use 'dotted_path' to avoid a reduce/reduce with slot_ref.
dotted_path:path
{: RESULT = new FunctionName(path); :}
;
from_clause ::=
KW_FROM table_ref_list:l
{: RESULT = new FromClause(l); :}
;
table_ref_list ::=
table_ref:table opt_plan_hints:hints
{:
List<TableRef> list = new ArrayList<>();
table.setTableHints(hints);
list.add(table);
RESULT = list;
:}
| table_ref_list:list COMMA table_ref:table opt_plan_hints:hints
{:
table.setTableHints(hints);
list.add(table);
RESULT = list;
:}
| table_ref_list:list KW_CROSS KW_JOIN opt_plan_hints:join_hints table_ref:table
opt_plan_hints:table_hints
{:
table.setJoinOp(JoinOperator.CROSS_JOIN);
// We will throw an AnalysisException if there are join hints so that we can provide
// a better error message than a parser exception.
table.setJoinHints(join_hints);
table.setTableHints(table_hints);
list.add(table);
RESULT = list;
:}
| table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
opt_plan_hints:table_hints
{:
table.setJoinOp((JoinOperator) op);
table.setJoinHints(join_hints);
table.setTableHints(table_hints);
list.add(table);
RESULT = list;
:}
| table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
opt_plan_hints:table_hints KW_ON expr:e
{:
table.setJoinOp((JoinOperator) op);
table.setJoinHints(join_hints);
table.setTableHints(table_hints);
table.setOnClause(e);
list.add(table);
RESULT = list;
:}
| table_ref_list:list join_operator:op opt_plan_hints:join_hints table_ref:table
opt_plan_hints:table_hints KW_USING LPAREN ident_list:colNames RPAREN
{:
table.setJoinOp((JoinOperator) op);
table.setJoinHints(join_hints);
table.setTableHints(table_hints);
table.setUsingClause(colNames);
list.add(table);
RESULT = list;
:}
;
table_ref ::=
dotted_path:path opt_tablesample:tblsmpl
{: RESULT = new TableRef(path, null, tblsmpl); :}
| dotted_path:path alias_clause:alias opt_tablesample:tblsmpl
{: RESULT = new TableRef(path, alias, tblsmpl); :}
| LPAREN query_stmt:query RPAREN alias_clause:alias opt_tablesample:tblsmpl
{: RESULT = new InlineViewRef(alias, query, tblsmpl); :}
;
join_operator ::=
opt_inner KW_JOIN
{: RESULT = JoinOperator.INNER_JOIN; :}
| KW_LEFT opt_outer KW_JOIN
{: RESULT = JoinOperator.LEFT_OUTER_JOIN; :}
| KW_RIGHT opt_outer KW_JOIN
{: RESULT = JoinOperator.RIGHT_OUTER_JOIN; :}
| KW_FULL opt_outer KW_JOIN
{: RESULT = JoinOperator.FULL_OUTER_JOIN; :}
| KW_LEFT KW_SEMI KW_JOIN
{: RESULT = JoinOperator.LEFT_SEMI_JOIN; :}
| KW_RIGHT KW_SEMI KW_JOIN
{: RESULT = JoinOperator.RIGHT_SEMI_JOIN; :}
| KW_LEFT KW_ANTI KW_JOIN
{: RESULT = JoinOperator.LEFT_ANTI_JOIN; :}
| KW_RIGHT KW_ANTI KW_JOIN
{: RESULT = JoinOperator.RIGHT_ANTI_JOIN; :}
;
opt_inner ::=
KW_INNER
|
;
opt_outer ::=
KW_OUTER
|
;
opt_plan_hints ::=
plan_hints:hints
{: RESULT = hints; :}
| /* empty */
{: RESULT = new ArrayList<>(); :}
;
plan_hints ::=
COMMENTED_PLAN_HINT_START plan_hint_list:hints COMMENTED_PLAN_HINT_END
{: RESULT = hints; :}
/* legacy straight_join hint style */
| KW_STRAIGHT_JOIN
{: RESULT = Lists.newArrayList(new PlanHint("straight_join")); :}
/* legacy plan-hint style */
| LBRACKET plan_hint_list:hints RBRACKET
{: RESULT = hints; :}
;
plan_hint ::=
KW_STRAIGHT_JOIN
{: RESULT = new PlanHint("straight_join"); :}
| IDENT:name
{: RESULT = new PlanHint(name); :}
| IDENT:name LPAREN ident_list:args RPAREN
{: RESULT = new PlanHint(name, args); :}
| /* empty */
{: RESULT = null; :}
;
plan_hint_list ::=
plan_hint:hint
{:
List<PlanHint> hints = Lists.newArrayList(hint);
RESULT = hints;
:}
| plan_hint_list:hints COMMA plan_hint:hint
{:
if (hint != null) hints.add(hint);
RESULT = hints;
:}
;
opt_tablesample ::=
KW_TABLESAMPLE system_ident LPAREN INTEGER_LITERAL:p RPAREN
{: RESULT = new TableSampleClause(p.longValue(), null); :}
| KW_TABLESAMPLE system_ident LPAREN INTEGER_LITERAL:p RPAREN
KW_REPEATABLE LPAREN INTEGER_LITERAL:s RPAREN
{: RESULT = new TableSampleClause(p.longValue(), Long.valueOf(s.longValue())); :}
| /* empty */
{: RESULT = null; :}
;
ident_list ::=
ident_or_default:ident
{:
List<String> list = new ArrayList<>();
list.add(ident);
RESULT = list;
:}
| ident_list:list COMMA ident_or_default:ident
{:
list.add(ident);
RESULT = list;
:}
;
expr_list ::=
expr:e
{:
List<Expr> list = new ArrayList<>();
list.add(e);
RESULT = list;
:}
| expr_list:list COMMA expr:e
{:
list.add(e);
RESULT = list;
:}
;
where_clause ::=
KW_WHERE expr:e
{: RESULT = e; :}
| /* empty */
{: RESULT = null; :}
;
group_by_clause ::=
KW_GROUP KW_BY expr_list:l
{: RESULT = l; :}
| /* empty */
{: RESULT = null; :}
;
having_clause ::=
KW_HAVING expr:e
{: RESULT = e; :}
| /* empty */
{: RESULT = null; :}
;
opt_order_by_clause ::=
KW_ORDER KW_BY order_by_elements:l
{: RESULT = l; :}
| /* empty */
{: RESULT = null; :}
;
order_by_elements ::=
order_by_element:e
{:
List<OrderByElement> list = new ArrayList<>();
list.add(e);
RESULT = list;
:}
| order_by_elements:list COMMA order_by_element:e
{:
list.add(e);
RESULT = list;
:}
;
order_by_element ::=
expr:e opt_order_param:o opt_nulls_order_param:n
{: RESULT = new OrderByElement(e, o, n); :}
;
opt_order_param ::=
KW_ASC
{: RESULT = true; :}
| KW_DESC
{: RESULT = false; :}
| /* empty */
{: RESULT = true; :}
;
opt_nulls_order_param ::=
KW_NULLS KW_FIRST
{: RESULT = true; :}
| KW_NULLS KW_LAST
{: RESULT = false; :}
| /* empty */
{: RESULT = null; :}
;
opt_offset_param ::=
KW_OFFSET expr:e
{: RESULT = e; :}
| /* empty */
{: RESULT = null; :}
;
opt_limit_offset_clause ::=
opt_limit_clause:limitExpr opt_offset_clause:offsetExpr
{: RESULT = new LimitElement(limitExpr, offsetExpr); :}
;
opt_limit_clause ::=
KW_LIMIT expr:limitExpr
{: RESULT = limitExpr; :}
| /* empty */
{: RESULT = null; :}
;
opt_offset_clause ::=
KW_OFFSET expr:offsetExpr
{: RESULT = offsetExpr; :}
| /* empty */
{: RESULT = null; :}
;
cast_format_val ::=
KW_FORMAT STRING_LITERAL:pattern
{: RESULT = pattern; :}
| /* empty */
{: RESULT = null; :}
;
cast_expr ::=
KW_CAST LPAREN expr:e KW_AS type_def:targetType cast_format_val:format RPAREN
{: RESULT = new CastExpr(targetType, e, format); :}
;
case_expr ::=
KW_CASE expr:caseExpr
case_when_clause_list:whenClauseList
case_else_clause:elseExpr
KW_END
{: RESULT = new CaseExpr(caseExpr, whenClauseList, elseExpr); :}
| KW_CASE
case_when_clause_list:whenClauseList
case_else_clause:elseExpr
KW_END
{: RESULT = new CaseExpr(null, whenClauseList, elseExpr); :}
;
case_when_clause_list ::=
KW_WHEN expr:whenExpr KW_THEN expr:thenExpr
{:
List<CaseWhenClause> list = new ArrayList<>();
list.add(new CaseWhenClause(whenExpr, thenExpr));
RESULT = list;
:}
| case_when_clause_list:list KW_WHEN expr:whenExpr
KW_THEN expr:thenExpr
{:
list.add(new CaseWhenClause(whenExpr, thenExpr));
RESULT = list;
:}
;
case_else_clause ::=
KW_ELSE expr:e
{: RESULT = e; :}
| /* emtpy */
{: RESULT = null; :}
;
sign_chain_expr ::=
SUBTRACT expr:e
{:
// integrate signs into literals
// integer literals require analysis to set their type, so the instance check below
// is not equivalent to e.getType().isNumericType()
if (e instanceof NumericLiteral) {
((LiteralExpr)e).swapSign();
RESULT = e;
} else {
RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.MULTIPLY,
new NumericLiteral(BigDecimal.valueOf(-1)), e);
}
:}
%prec UNARYSIGN
| ADD expr:e
{: RESULT = e; :}
%prec UNARYSIGN
;
expr ::=
non_pred_expr:e
{: RESULT = e; :}
| predicate:p
{: RESULT = p; :}
;
exists_predicate ::=
KW_EXISTS subquery:s
{: RESULT = new ExistsPredicate(s, false); :}
;
non_pred_expr ::=
sign_chain_expr:e
{: RESULT = e; :}
| literal:l
{: RESULT = l; :}
| function_call_expr:e
{: RESULT = e; :}
| analytic_expr:e
{: RESULT = e; :}
/* Additional rules for function names that are also keywords */
| KW_IF LPAREN expr_list:exprs RPAREN
{: RESULT = new FunctionCallExpr("if", exprs); :}
| KW_REPLACE LPAREN expr_list:exprs RPAREN
{: RESULT = new FunctionCallExpr("replace", exprs); :}
| KW_TRUNCATE LPAREN expr_list:exprs RPAREN
{: RESULT = new FunctionCallExpr("truncate", exprs); :}
| KW_LEFT LPAREN expr_list:exprs RPAREN
{: RESULT = new FunctionCallExpr("left", exprs); :}
| KW_RIGHT LPAREN expr_list:exprs RPAREN
{: RESULT = new FunctionCallExpr("right", exprs); :}
| cast_expr:c
{: RESULT = c; :}
| case_expr:c
{: RESULT = c; :}
| slot_ref:c
{: RESULT = c; :}
| timestamp_arithmetic_expr:e
{: RESULT = e; :}
| arithmetic_expr:e
{: RESULT = e; :}
| LPAREN non_pred_expr:e RPAREN
{:
e.setPrintSqlInParens(true);
RESULT = e;
:}
| subquery:s
{: RESULT = s; :}
;
function_call_expr ::=
function_name:fn_name LPAREN RPAREN
{:
RESULT = FunctionCallExpr.createExpr(
fn_name, new FunctionParams(new ArrayList<>()), parser.getQueryOptions());
:}
| function_name:fn_name LPAREN function_params:params RPAREN
{: RESULT = FunctionCallExpr.createExpr(fn_name, params, parser.getQueryOptions()); :}
// Below is a special case for EXTRACT. Idents are used to avoid adding new keywords.
| function_name:fn_name LPAREN ident_or_default:u KW_FROM expr:t RPAREN
{: RESULT = new ExtractFromExpr(fn_name, u, t); :}
;
// TODO: allow an arbitrary expr here instead of agg/fn call, and check during analysis?
// The parser errors aren't particularly easy to parse.
analytic_expr ::=
function_call_expr:e KW_OVER
LPAREN opt_partition_by_clause:p opt_order_by_clause:o opt_window_clause:w RPAREN
{:
// Handle cases where function_call_expr resulted in a plain Expr
if (!(e instanceof FunctionCallExpr)) {
parser.parseError("over", SqlParserSymbols.KW_OVER);
}
FunctionCallExpr f = (FunctionCallExpr)e;
f.setIsAnalyticFnCall(true);
RESULT = new AnalyticExpr(f, p, o, w);
:}
%prec KW_OVER
;
opt_partition_by_clause ::=
KW_PARTITION KW_BY expr_list:l
{: RESULT = l; :}
| /* empty */
{: RESULT = null; :}
;
opt_window_clause ::=
window_type:t window_boundary:b
{: RESULT = new AnalyticWindow(t, b); :}
| window_type:t KW_BETWEEN window_boundary:l KW_AND window_boundary:r
{: RESULT = new AnalyticWindow(t, l, r); :}
| /* empty */
{: RESULT = null; :}
;
window_type ::=
KW_ROWS
{: RESULT = AnalyticWindow.Type.ROWS; :}
| KW_RANGE
{: RESULT = AnalyticWindow.Type.RANGE; :}
;
window_boundary ::=
KW_UNBOUNDED KW_PRECEDING
{:
RESULT = new AnalyticWindow.Boundary(
AnalyticWindow.BoundaryType.UNBOUNDED_PRECEDING, null);
:}
| KW_UNBOUNDED KW_FOLLOWING
{:
RESULT = new AnalyticWindow.Boundary(
AnalyticWindow.BoundaryType.UNBOUNDED_FOLLOWING, null);
:}
| KW_CURRENT KW_ROW
{:
RESULT = new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.CURRENT_ROW, null);
:}
| expr:e KW_PRECEDING
{: RESULT = new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.PRECEDING, e); :}
| expr:e KW_FOLLOWING
{: RESULT = new AnalyticWindow.Boundary(AnalyticWindow.BoundaryType.FOLLOWING, e); :}
;
arithmetic_expr ::=
expr:e1 STAR expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.MULTIPLY, e1, e2); :}
| expr:e1 DIVIDE expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.DIVIDE, e1, e2); :}
| expr:e1 MOD expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.MOD, e1, e2); :}
| expr:e1 KW_DIV expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.INT_DIVIDE, e1, e2); :}
| expr:e1 ADD expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.ADD, e1, e2); :}
| expr:e1 SUBTRACT expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.SUBTRACT, e1, e2); :}
| expr:e1 BITAND expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.BITAND, e1, e2); :}
| expr:e1 BITOR expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.BITOR, e1, e2); :}
| expr:e1 BITXOR expr:e2
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.BITXOR, e1, e2); :}
| BITNOT expr:e
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.BITNOT, e, null); :}
| expr:e NOT
{: RESULT = new ArithmeticExpr(ArithmeticExpr.Operator.FACTORIAL, e, null); :}
%prec FACTORIAL
;
// We use IDENT for the temporal unit to avoid making DAY, YEAR, etc. keywords.
// This way we do not need to change existing uses of IDENT.
// We chose not to make DATE_ADD and DATE_SUB keywords for the same reason.
timestamp_arithmetic_expr ::=
KW_INTERVAL expr:v IDENT:u ADD expr:t
{: RESULT = new TimestampArithmeticExpr(ArithmeticExpr.Operator.ADD, t, v, u, true); :}
| expr:t ADD KW_INTERVAL expr:v IDENT:u
{:
RESULT = new TimestampArithmeticExpr(ArithmeticExpr.Operator.ADD, t, v, u, false);
:}
// Set precedence to KW_INTERVAL (which is higher than ADD) for chaining.
%prec KW_INTERVAL
| expr:t SUBTRACT KW_INTERVAL expr:v IDENT:u
{:
RESULT =
new TimestampArithmeticExpr(ArithmeticExpr.Operator.SUBTRACT, t, v, u, false);
:}
// Set precedence to KW_INTERVAL (which is higher than ADD) for chaining.
%prec KW_INTERVAL
// Timestamp arithmetic expr that looks like a function call.
// We use expr_list instead of expr to avoid a shift/reduce conflict with
// expr_list on COMMA, and report an error if the list contains more than one expr.
// Although we don't want to accept function names as the expr, we can't parse it
// as just an IDENT due to the precedence conflict with function_name.
| function_name:functionName LPAREN expr_list:l COMMA
KW_INTERVAL expr:v IDENT:u RPAREN
{:
if (l.size() > 1) {
// Report parsing failure on keyword interval.
parser.parseError("interval", SqlParserSymbols.KW_INTERVAL);
}
List<String> fnNamePath = functionName.getFnNamePath();
if (fnNamePath.size() > 1) {
// This production should not accept fully qualified function names
throw new Exception("interval should not be qualified by database name");
}
RESULT = new TimestampArithmeticExpr(fnNamePath.get(0), l.get(0), v, u);
:}
;
numeric_literal ::=
INTEGER_LITERAL:l
{: RESULT = new NumericLiteral(l); :}
| DECIMAL_LITERAL:l
{: RESULT = new NumericLiteral(l); :}
;
literal ::=
numeric_literal:l
{: RESULT = l; :}
| STRING_LITERAL:l
{: RESULT = new StringLiteral(l); :}
| KW_DATE STRING_LITERAL:l
{: RESULT = new DateLiteral(l); :}
| KW_TRUE
{: RESULT = new BoolLiteral(true); :}
| KW_FALSE
{: RESULT = new BoolLiteral(false); :}
| KW_NULL
{: RESULT = new NullLiteral(); :}
| UNMATCHED_STRING_LITERAL:l expr:e
{:
// we have an unmatched string literal.
// to correctly report the root cause of this syntax error
// we must force parsing to fail at this point,
// and generate an unmatched string literal symbol
// to be passed as the last seen token in the
// error handling routine (otherwise some other token could be reported)
parser.parseError("literal", SqlParserSymbols.UNMATCHED_STRING_LITERAL);
:}
| NUMERIC_OVERFLOW:l
{:
// similar to the unmatched string literal case
// we must terminate parsing at this point
// and generate a corresponding symbol to be reported
parser.parseError("literal", SqlParserSymbols.NUMERIC_OVERFLOW);
:}
;
function_params ::=
STAR
{: RESULT = FunctionParams.createStarParam(); :}
| KW_ALL STAR
{: RESULT = FunctionParams.createStarParam(); :}
| expr_list:exprs
{: RESULT = new FunctionParams(false, exprs); :}
| KW_ALL expr_list:exprs
{: RESULT = new FunctionParams(false, exprs); :}
| KW_DISTINCT:distinct expr_list:exprs
{: RESULT = new FunctionParams(true, exprs); :}
| expr_list:exprs KW_IGNORE KW_NULLS
{: RESULT = new FunctionParams(false, true, exprs); :}
;
predicate ::=
expr:e KW_IS KW_NULL
{: RESULT = new IsNullPredicate(e, false); :}
| expr:e KW_IS KW_NOT KW_NULL
{: RESULT = new IsNullPredicate(e, true); :}
| between_predicate:p
{: RESULT = p; :}
| comparison_predicate:p
{: RESULT = p; :}
| compound_predicate:p
{: RESULT = p; :}
| in_predicate:p
{: RESULT = p; :}
| exists_predicate:p
{: RESULT = p; :}
| like_predicate:p
{: RESULT = p; :}
| bool_test_expr:e
{: RESULT = e; :}
| LPAREN predicate:p RPAREN
{:
p.setPrintSqlInParens(true);
RESULT = p;
:}
;
comparison_predicate ::=
expr:e1 EQUAL expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.EQ, e1, e2); :}
| expr:e1 NOTEQUAL expr:e2 // single != token
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.NE, e1, e2); :}
| expr:e1 NOT EQUAL expr:e2 // separate ! and = tokens
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.NE, e1, e2); :}
| expr:e1 LESSTHAN GREATERTHAN expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.NE, e1, e2); :}
| expr:e1 LESSTHAN EQUAL expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.LE, e1, e2); :}
| expr:e1 GREATERTHAN EQUAL expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.GE, e1, e2); :}
| expr:e1 LESSTHAN expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.LT, e1, e2); :}
| expr:e1 GREATERTHAN expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.GT, e1, e2); :}
| expr:e1 LESSTHAN EQUAL GREATERTHAN expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.NOT_DISTINCT, e1, e2); :}
| expr:e1 KW_IS KW_DISTINCT KW_FROM expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.DISTINCT_FROM, e1, e2); :}
| expr:e1 KW_IS KW_NOT KW_DISTINCT KW_FROM expr:e2
{: RESULT = new BinaryPredicate(BinaryPredicate.Operator.NOT_DISTINCT, e1, e2); :}
;
like_predicate ::=
expr:e1 KW_LIKE expr:e2
{: RESULT = new LikePredicate(LikePredicate.Operator.LIKE, e1, e2); :}
| expr:e1 KW_ILIKE expr:e2
{: RESULT = new LikePredicate(LikePredicate.Operator.ILIKE, e1, e2); :}
| expr:e1 KW_RLIKE expr:e2
{: RESULT = new LikePredicate(LikePredicate.Operator.RLIKE, e1, e2); :}
| expr:e1 KW_REGEXP expr:e2
{: RESULT = new LikePredicate(LikePredicate.Operator.REGEXP, e1, e2); :}
| expr:e1 KW_IREGEXP expr:e2
{: RESULT = new LikePredicate(LikePredicate.Operator.IREGEXP, e1, e2); :}
| expr:e1 KW_NOT KW_LIKE expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT,
new LikePredicate(LikePredicate.Operator.LIKE, e1, e2), null); :}
| expr:e1 KW_NOT KW_ILIKE expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT,
new LikePredicate(LikePredicate.Operator.ILIKE, e1, e2), null); :}
| expr:e1 KW_NOT KW_RLIKE expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT,
new LikePredicate(LikePredicate.Operator.RLIKE, e1, e2), null); :}
| expr:e1 KW_NOT KW_REGEXP expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT,
new LikePredicate(LikePredicate.Operator.REGEXP, e1, e2), null); :}
| expr:e1 KW_NOT KW_IREGEXP expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT,
new LikePredicate(LikePredicate.Operator.IREGEXP, e1, e2), null); :}
;
// Avoid a reduce/reduce conflict with compound_predicate by explicitly
// using non_pred_expr and predicate separately instead of expr.
between_predicate ::=
expr:e1 KW_BETWEEN non_pred_expr:e2 KW_AND expr:e3
{: RESULT = new BetweenPredicate(e1, e2, e3, false); :}
| expr:e1 KW_BETWEEN predicate:e2 KW_AND expr:e3
{: RESULT = new BetweenPredicate(e1, e2, e3, false); :}
| expr:e1 KW_NOT KW_BETWEEN non_pred_expr:e2 KW_AND expr:e3
{: RESULT = new BetweenPredicate(e1, e2, e3, true); :}
| expr:e1 KW_NOT KW_BETWEEN predicate:e2 KW_AND expr:e3
{: RESULT = new BetweenPredicate(e1, e2, e3, true); :}
;
in_predicate ::=
expr:e KW_IN LPAREN expr_list:l RPAREN
{: RESULT = new InPredicate(e, l, false); :}
| expr:e KW_NOT KW_IN LPAREN expr_list:l RPAREN
{: RESULT = new InPredicate(e, l, true); :}
| expr:e KW_IN subquery:s
{: RESULT = new InPredicate(e, s, false); :}
| expr:e KW_NOT KW_IN subquery:s
{: RESULT = new InPredicate(e, s, true); :}
;
// Boolean test expression: <expr> IS [NOT] (TRUE | FALSE | UNKNOWN)
bool_test_expr ::=
expr:e KW_IS KW_TRUE
{: RESULT = new FunctionCallExpr("istrue", Lists.newArrayList(e)); :}
| expr:e KW_IS KW_NOT KW_TRUE
{: RESULT = new FunctionCallExpr("isnottrue", Lists.newArrayList(e)); :}
| expr:e KW_IS KW_FALSE
{: RESULT = new FunctionCallExpr("isfalse", Lists.newArrayList(e)); :}
| expr:e KW_IS KW_NOT KW_FALSE
{: RESULT = new FunctionCallExpr("isnotfalse", Lists.newArrayList(e)); :}
| expr:e KW_IS KW_UNKNOWN
{: RESULT = new IsNullPredicate(e, false); :}
| expr:e KW_IS KW_NOT KW_UNKNOWN
{: RESULT = new IsNullPredicate(e, true); :}
;
subquery ::=
LPAREN subquery:s RPAREN
{: RESULT = s; :}
| LPAREN query_stmt:s RPAREN
{: RESULT = new Subquery(s); :}
;
compound_predicate ::=
expr:e1 KW_AND expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.AND, e1, e2); :}
| expr:e1 KW_OR expr:e2
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.OR, e1, e2); :}
| KW_NOT expr:e
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT, e, null); :}
| NOT expr:e
{: RESULT = new CompoundPredicate(CompoundPredicate.Operator.NOT, e, null); :}
;
slot_ref ::=
dotted_path:path
{: RESULT = new SlotRef(path); :}
;
dotted_path ::=
ident_or_default:ident
{:
List<String> list = new ArrayList<>();
list.add(ident);
RESULT = list;
:}
| dotted_path:list DOT ident_or_default:ident
{:
list.add(ident);
RESULT = list;
:}
;
type_def ::=
type:t
{: RESULT = new TypeDef(t); :}
;
type ::=
KW_TINYINT
{: RESULT = Type.TINYINT; :}
| KW_SMALLINT
{: RESULT = Type.SMALLINT; :}
| KW_INT
{: RESULT = Type.INT; :}
| KW_BIGINT
{: RESULT = Type.BIGINT; :}
| KW_BOOLEAN
{: RESULT = Type.BOOLEAN; :}
| KW_FLOAT
{: RESULT = Type.FLOAT; :}
| KW_DOUBLE
{: RESULT = Type.DOUBLE; :}
| KW_DATE
{: RESULT = Type.DATE; :}
| KW_DATETIME
{: RESULT = Type.DATETIME; :}
| KW_TIMESTAMP
{: RESULT = Type.TIMESTAMP; :}
| KW_STRING
{: RESULT = Type.STRING; :}
| KW_VARCHAR LPAREN INTEGER_LITERAL:len RPAREN
{: RESULT = ScalarType.createVarcharType(len.intValue()); :}
| KW_VARCHAR
{: RESULT = Type.STRING; :}
| KW_BINARY
{: RESULT = Type.BINARY; :}
| KW_CHAR LPAREN INTEGER_LITERAL:len RPAREN
{: RESULT = ScalarType.createCharType(len.intValue()); :}
| KW_DECIMAL LPAREN INTEGER_LITERAL:precision RPAREN
{: RESULT = ScalarType.createDecimalType(precision.intValue()); :}
| KW_DECIMAL LPAREN INTEGER_LITERAL:precision COMMA INTEGER_LITERAL:scale RPAREN
{: RESULT = ScalarType.createDecimalType(precision.intValue(), scale.intValue()); :}
| KW_DECIMAL
{: RESULT = ScalarType.createDecimalType(); :}
| KW_ARRAY LESSTHAN type:value_type GREATERTHAN
{: RESULT = new ArrayType(value_type); :}
| KW_MAP LESSTHAN type:key_type COMMA type:value_type GREATERTHAN
{: RESULT = new MapType(key_type, value_type); :}
| KW_STRUCT LESSTHAN struct_field_def_list:fields GREATERTHAN
{: RESULT = new StructType(fields); :}
;
// Recognize identifiers and keywords as struct-field names such
// that we can parse type strings from the Hive Metastore which
// may have unquoted identifiers corresponding to keywords.
struct_field_def ::=
word:name COLON type:t opt_comment_val:comment
{: RESULT = new StructField(name, t, comment); :}
;
struct_field_def_list ::=
struct_field_def:field_def
{:
List<StructField> list = new ArrayList<>();
list.add(field_def);
RESULT = list;
:}
| struct_field_def_list:list COMMA struct_field_def:field_def
{:
list.add(field_def);
RESULT = list;
:}
;
ident_or_default ::=
IDENT:name
{: RESULT = name.toString(); :}
| KW_DEFAULT:name
{: RESULT = name.toString(); :}
;
word ::=
IDENT:r
{: RESULT = r.toString(); :}
| KW_ADD:r
{: RESULT = r.toString(); :}
| KW_AGGREGATE:r
{: RESULT = r.toString(); :}
| KW_ALL:r
{: RESULT = r.toString(); :}
| KW_ALTER:r
{: RESULT = r.toString(); :}
| KW_ANALYTIC:r
{: RESULT = r.toString(); :}
| KW_AND:r
{: RESULT = r.toString(); :}
| KW_ANTI:r
{: RESULT = r.toString(); :}
| KW_API_VERSION:r
{: RESULT = r.toString(); :}
| KW_ARRAY:r
{: RESULT = r.toString(); :}
| KW_AS:r
{: RESULT = r.toString(); :}
| KW_ASC:r
{: RESULT = r.toString(); :}
| KW_AUTHORIZATION:r
{: RESULT = r.toString(); :}
| KW_AVRO:r
{: RESULT = r.toString(); :}
| KW_BETWEEN:r
{: RESULT = r.toString(); :}
| KW_BIGINT:r
{: RESULT = r.toString(); :}
| KW_BINARY:r
{: RESULT = r.toString(); :}
| KW_BLOCKSIZE:r
{: RESULT = r.toString(); :}
| KW_BOOLEAN:r
{: RESULT = r.toString(); :}
| KW_BY:r
{: RESULT = r.toString(); :}
| KW_CACHED:r
{: RESULT = r.toString(); :}
| KW_CASCADE:r
{: RESULT = r.toString(); :}
| KW_CASE:r
{: RESULT = r.toString(); :}
| KW_CAST:r
{: RESULT = r.toString(); :}
| KW_CHANGE:r
{: RESULT = r.toString(); :}
| KW_CHAR:r
{: RESULT = r.toString(); :}
| KW_CLASS:r
{: RESULT = r.toString(); :}
| KW_CLOSE_FN:r
{: RESULT = r.toString(); :}
| KW_COLUMN:r
{: RESULT = r.toString(); :}
| KW_COLUMNS:r
{: RESULT = r.toString(); :}
| KW_COMMENT:r
{: RESULT = r.toString(); :}
| KW_COMPRESSION:r
{: RESULT = r.toString(); :}
| KW_COMPUTE:r
{: RESULT = r.toString(); :}
| KW_CONSTRAINT:r
{: RESULT = r.toString(); :}
| KW_COPY:r
{: RESULT = r.toString(); :}
| KW_CREATE:r
{: RESULT = r.toString(); :}
| KW_CROSS:r
{: RESULT = r.toString(); :}
| KW_CURRENT:r
{: RESULT = r.toString(); :}
| KW_DATA:r
{: RESULT = r.toString(); :}
| KW_DATABASE:r
{: RESULT = r.toString(); :}
| KW_DATABASES:r
{: RESULT = r.toString(); :}
| KW_DATE:r
{: RESULT = r.toString(); :}
| KW_DATETIME:r
{: RESULT = r.toString(); :}
| KW_DECIMAL:r
{: RESULT = r.toString(); :}
| KW_DEFAULT:r
{: RESULT = r.toString(); :}
| KW_DELETE:r
{: RESULT = r.toString(); :}
| KW_DELIMITED:r
{: RESULT = r.toString(); :}
| KW_DESC:r
{: RESULT = r.toString(); :}
| KW_DESCRIBE:r
{: RESULT = r.toString(); :}
| KW_DISABLE:r
{: RESULT = r.toString(); :}
| KW_DISTINCT:r
{: RESULT = r.toString(); :}
| KW_DIV:r
{: RESULT = r.toString(); :}
| KW_DOUBLE:r
{: RESULT = r.toString(); :}
| KW_DROP:r
{: RESULT = r.toString(); :}
| KW_ELSE:r
{: RESULT = r.toString(); :}
| KW_ENABLE:r
{: RESULT = r.toString(); :}
| KW_ENCODING:r
{: RESULT = r.toString(); :}
| KW_END:r
{: RESULT = r.toString(); :}
| KW_ESCAPED:r
{: RESULT = r.toString(); :}
| KW_EXISTS:r
{: RESULT = r.toString(); :}
| KW_EXPLAIN:r
{: RESULT = r.toString(); :}
| KW_EXTENDED:r
{: RESULT = r.toString(); :}
| KW_EXTERNAL:r
{: RESULT = r.toString(); :}
| KW_FALSE:r
{: RESULT = r.toString(); :}
| KW_FIELDS:r
{: RESULT = r.toString(); :}
| KW_FILEFORMAT:r
{: RESULT = r.toString(); :}
| KW_FILES:r
{: RESULT = r.toString(); :}
| KW_FINALIZE_FN:r
{: RESULT = r.toString(); :}
| KW_FIRST:r
{: RESULT = r.toString(); :}
| KW_FLOAT:r
{: RESULT = r.toString(); :}
| KW_FOLLOWING:r
{: RESULT = r.toString(); :}
| KW_FOR:r
{: RESULT = r.toString(); :}
| KW_FOREIGN:r
{: RESULT = r.toString(); :}
| KW_FORMAT:r
{: RESULT = r.toString(); :}
| KW_FORMATTED:r
{: RESULT = r.toString(); :}
| KW_FROM:r
{: RESULT = r.toString(); :}
| KW_FULL:r
{: RESULT = r.toString(); :}
| KW_FUNCTION:r
{: RESULT = r.toString(); :}
| KW_FUNCTIONS:r
{: RESULT = r.toString(); :}
| KW_GRANT:r
{: RESULT = r.toString(); :}
| KW_GROUP:r
{: RESULT = r.toString(); :}
| KW_HAVING:r
{: RESULT = r.toString(); :}
| KW_HASH:r
{: RESULT = r.toString(); :}
| KW_IF:r
{: RESULT = r.toString(); :}
| KW_IGNORE:r
{: RESULT = r.toString(); :}
| KW_ILIKE:r
{: RESULT = r.toString(); :}
| KW_IN:r
{: RESULT = r.toString(); :}
| KW_INCREMENTAL:r
{: RESULT = r.toString(); :}
| KW_INIT_FN:r
{: RESULT = r.toString(); :}
| KW_INNER:r
{: RESULT = r.toString(); :}
| KW_INPATH:r
{: RESULT = r.toString(); :}
| KW_INSERT:r
{: RESULT = r.toString(); :}
| KW_INT:r
{: RESULT = r.toString(); :}
| KW_INTERMEDIATE:r
{: RESULT = r.toString(); :}
| KW_INTERVAL:r
{: RESULT = r.toString(); :}
| KW_INTO:r
{: RESULT = r.toString(); :}
| KW_INVALIDATE:r
{: RESULT = r.toString(); :}
| KW_IREGEXP:r
{: RESULT = r.toString(); :}
| KW_IS:r
{: RESULT = r.toString(); :}
| KW_JOIN:r
{: RESULT = r.toString(); :}
| KW_KUDU:r
{: RESULT = r.toString(); :}
| KW_LAST:r
{: RESULT = r.toString(); :}
| KW_LEFT:r
{: RESULT = r.toString(); :}
| KW_LEXICAL:r
{: RESULT = r.toString(); :}
| KW_LIKE:r
{: RESULT = r.toString(); :}
| KW_LIMIT:r
{: RESULT = r.toString(); :}
| KW_LINES:r
{: RESULT = r.toString(); :}
| KW_LOAD:r
{: RESULT = r.toString(); :}
| KW_LOCATION:r
{: RESULT = r.toString(); :}
| KW_MAP:r
{: RESULT = r.toString(); :}
| KW_MERGE_FN:r
{: RESULT = r.toString(); :}
| KW_METADATA:r
{: RESULT = r.toString(); :}
| KW_NORELY:r
{: RESULT = r.toString(); :}
| KW_NOT:r
{: RESULT = r.toString(); :}
| KW_NOVALIDATE:r
{: RESULT = r.toString(); :}
| KW_NULL:r
{: RESULT = r.toString(); :}
| KW_NULLS:r
{: RESULT = r.toString(); :}
| KW_OFFSET:r
{: RESULT = r.toString(); :}
| KW_ON:r
{: RESULT = r.toString(); :}
| KW_OR:r
{: RESULT = r.toString(); :}
| KW_ORC:r
{: RESULT = r.toString(); :}
| KW_ORDER:r
{: RESULT = r.toString(); :}
| KW_OUTER:r
{: RESULT = r.toString(); :}
| KW_OVER:r
{: RESULT = r.toString(); :}
| KW_OVERWRITE:r
{: RESULT = r.toString(); :}
| KW_PARQUET:r
{: RESULT = r.toString(); :}
| KW_PARQUETFILE:r
{: RESULT = r.toString(); :}
| KW_PARTITION:r
{: RESULT = r.toString(); :}
| KW_PARTITIONED:r
{: RESULT = r.toString(); :}
| KW_PARTITIONS:r
{: RESULT = r.toString(); :}
| KW_PRECEDING:r
{: RESULT = r.toString(); :}
| KW_PREPARE_FN:r
{: RESULT = r.toString(); :}
| KW_PRIMARY:r
{: RESULT = r.toString(); :}
| KW_PRODUCED:r
{: RESULT = r.toString(); :}
| KW_PURGE:r
{: RESULT = r.toString(); :}
| KW_RANGE:r
{: RESULT = r.toString(); :}
| KW_RCFILE:r
{: RESULT = r.toString(); :}
| KW_RECOVER:r
{: RESULT = r.toString(); :}
| KW_REFERENCES:r
{: RESULT = r.toString(); :}
| KW_REFRESH:r
{: RESULT = r.toString(); :}
| KW_REGEXP:r
{: RESULT = r.toString(); :}
| KW_RELY:r
{: RESULT = r.toString(); :}
| KW_RENAME:r
{: RESULT = r.toString(); :}
| KW_REPEATABLE:r
{: RESULT = r.toString(); :}
| KW_REPLACE:r
{: RESULT = r.toString(); :}
| KW_REPLICATION:r
{: RESULT = r.toString(); :}
| KW_RESTRICT:r
{: RESULT = r.toString(); :}
| KW_RETURNS:r
{: RESULT = r.toString(); :}
| KW_REVOKE:r
{: RESULT = r.toString(); :}
| KW_RIGHT:r
{: RESULT = r.toString(); :}
| KW_RLIKE:r
{: RESULT = r.toString(); :}
| KW_ROLE:r
{: RESULT = r.toString(); :}
| KW_ROLES:r
{: RESULT = r.toString(); :}
| KW_ROW:r
{: RESULT = r.toString(); :}
| KW_ROWS:r
{: RESULT = r.toString(); :}
| KW_SCHEMA:r
{: RESULT = r.toString(); :}
| KW_SCHEMAS:r
{: RESULT = r.toString(); :}
| KW_SELECT:r
{: RESULT = r.toString(); :}
| KW_SEMI:r
{: RESULT = r.toString(); :}
| KW_SEQUENCEFILE:r
{: RESULT = r.toString(); :}
| KW_SERDEPROPERTIES:r
{: RESULT = r.toString(); :}
| KW_SERIALIZE_FN:r
{: RESULT = r.toString(); :}
| KW_SET:r
{: RESULT = r.toString(); :}
| KW_SHOW:r
{: RESULT = r.toString(); :}
| KW_SMALLINT:r
{: RESULT = r.toString(); :}
| KW_SORT:r
{: RESULT = r.toString(); :}
| KW_STORED:r
{: RESULT = r.toString(); :}
| KW_STRAIGHT_JOIN:r
{: RESULT = r.toString(); :}
| KW_STRING:r
{: RESULT = r.toString(); :}
| KW_STRUCT:r
{: RESULT = r.toString(); :}
| KW_SYMBOL:r
{: RESULT = r.toString(); :}
| KW_TABLE:r
{: RESULT = r.toString(); :}
| KW_TABLES:r
{: RESULT = r.toString(); :}
| KW_TABLESAMPLE:r
{: RESULT = r.toString(); :}
| KW_TBLPROPERTIES:r
{: RESULT = r.toString(); :}
| KW_TERMINATED:r
{: RESULT = r.toString(); :}
| KW_TEXTFILE:r
{: RESULT = r.toString(); :}
| KW_THEN:r
{: RESULT = r.toString(); :}
| KW_TIMESTAMP:r
{: RESULT = r.toString(); :}
| KW_TINYINT:r
{: RESULT = r.toString(); :}
| KW_TRUNCATE:r
{: RESULT = r.toString(); :}
| KW_STATS:r
{: RESULT = r.toString(); :}
| KW_TO:r
{: RESULT = r.toString(); :}
| KW_TRUE:r
{: RESULT = r.toString(); :}
| KW_UNBOUNDED:r
{: RESULT = r.toString(); :}
| KW_UNCACHED:r
{: RESULT = r.toString(); :}
| KW_UNION:r
{: RESULT = r.toString(); :}
| KW_UNKNOWN:r
{: RESULT = r.toString(); :}
| KW_UPDATE:r
{: RESULT = r.toString(); :}
| KW_UPDATE_FN:r
{: RESULT = r.toString(); :}
| KW_UPSERT:r
{: RESULT = r.toString(); :}
| KW_USE:r
{: RESULT = r.toString(); :}
| KW_USING:r
{: RESULT = r.toString(); :}
| KW_VALIDATE:r
{: RESULT = r.toString(); :}
| KW_VALUES:r
{: RESULT = r.toString(); :}
| KW_VARCHAR:r
{: RESULT = r.toString(); :}
| KW_VIEW:r
{: RESULT = r.toString(); :}
| KW_WHEN:r
{: RESULT = r.toString(); :}
| KW_WHERE:r
{: RESULT = r.toString(); :}
| KW_WITH:r
{: RESULT = r.toString(); :}
| KW_ZORDER:r
{: RESULT = r.toString(); :}
| UNUSED_RESERVED_WORD:r
{: RESULT = r.toString(); :}
;