diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index bd617bf7e3df6..e50489dd2077f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.catalyst.parser +import java.util.Locale + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -285,255 +287,67 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "where", "with") - // All the keywords in `docs/sql-keywords.md` are listed below: - val allCandidateKeywords = Set( - "add", - "after", - "all", - "alter", - "analyze", - "and", - "anti", - "any", - "archive", - "array", - "as", - "asc", - "at", - "authorization", - "between", - "both", - "bucket", - "buckets", - "by", - "cache", - "cascade", - "case", - "cast", - "change", - "check", - "clear", - "cluster", - "clustered", - "codegen", - "collate", - "collection", - "column", - "columns", - "comment", - "commit", - "compact", - "compactions", - "compute", - "concatenate", - "constraint", - "cost", - "create", - "cross", - "cube", - "current", - "current_date", - "current_time", - "current_timestamp", - "current_user", - "data", - "database", - "databases", - "day", - "dbproperties", - "defined", - "delete", - "delimited", - "desc", - "describe", - "dfs", - "directories", - "directory", - "distinct", - "distribute", - "div", - "drop", - "else", - "end", - "escape", - "escaped", - "except", - "exchange", - "exists", - "explain", - "export", - "extended", - "external", - "extract", - "false", - "fetch", - "fields", - "fileformat", - "first", - "following", - "for", - "foreign", - "format", - "formatted", - "from", - "full", - "function", - "functions", - "global", - "grant", - "group", - "grouping", - "having", - "hour", - "if", - "ignore", - "import", - "in", - "index", - "indexes", - "inner", - "inpath", - "inputformat", - "insert", - "intersect", - "interval", - "into", - "is", - "items", - "join", - "keys", - "last", - "lateral", - "lazy", - "leading", - "left", - "like", - "limit", - "lines", - "list", - "load", - "local", - "location", - "lock", - "locks", - "logical", - "macro", - "map", - "minus", - "minute", - "month", - "msck", - "namespaces", - "natural", - "no", - "not", - "null", - "nulls", - "of", - "on", - "only", - "option", - "options", - "or", - "order", - "out", - "outer", - "outputformat", - "over", - "overlaps", - "overlay", - "overwrite", - "partition", - "partitioned", - "partitions", - "percent", - "pivot", - "placing", - "position", - "preceding", - "primary", - "principals", - "purge", - "query", - "range", - "recordreader", - "recordwriter", - "recover", - "reduce", - "references", - "refresh", - "rename", - "repair", - "replace", - "reset", - "restrict", - "revoke", - "right", - "rlike", - "role", - "roles", - "rollback", - "rollup", - "row", - "rows", - "schema", - "second", - "select", - "semi", - "separated", - "serde", - "serdeproperties", - "session_user", - "set", - "sets", - "show", - "skewed", - "some", - "sort", - "sorted", - "start", - "statistics", - "stored", - "stratify", - "struct", - "substr", - "substring", - "table", - "tables", - "tablesample", - "tblproperties", - "temporary", - "terminated", - "then", - "to", - "touch", - "trailing", - "transaction", - "transactions", - "transform", - "true", - "truncate", - "type", - "unarchive", - "unbounded", - "uncache", - "union", - "unique", - "unknown", - "unlock", - "unset", - "use", - "user", - "using", - "values", - "view", - "views", - "when", - "where", - "window", - "with", - "year") + // All the SQL keywords defined in `SqlBase.g4` + val allCandidateKeywords = { + // Symbols we don't care + val blacklistSymbols = Seq( + "EQ", + "LTE", + "GTE", + "STRING", + "BIGINT_LITERAL", + "SMALLINT_LITERAL", + "TINYINT_LITERAL", + "INTEGER_VALUE", + "EXPONENT_VALUE", + "DECIMAL_VALUE", + "DECIMAL_VALUE", + "DOUBLE_LITERAL", + "BIGDECIMAL_LITERAL", + "IDENTIFIER", + "BACKQUOTED_IDENTIFIER", + "SIMPLE_COMMENT", + "BRACKETED_COMMENT", + "WS", + "UNRECOGNIZED" + ) + + // If a symbol has multiple literal tokens (e.g., `RLIKE: 'RLIKE' | 'REGEXP';`), + // we can not extract the literals from the generated parser class (`SqlBaseParser`). + // So, we need to manually add them in the candidate keyword set. + val missingKeywordMap = Map( + "DATABASES" -> Seq("DATABASES", "SCHEMAS"), + "NOT" -> Seq("NOT"), + "RLIKE" -> Seq("RLIKE", "REGEXP"), + "TEMPORARY" -> Seq("TEMPORARY", "TEMP") + ) ++ blacklistSymbols.map { symbol => + symbol -> Nil + } + + val sqlVocabs = SqlBaseParser.VOCABULARY + val literals = (0 until sqlVocabs.getMaxTokenType).flatMap { i => + val litNamesOrNull = { + val s = sqlVocabs.getLiteralName(i) + if (s != null) { + s.replaceAll("'", "") :: Nil + } else { + val symNameOrNull = sqlVocabs.getSymbolicName(i) + if (symNameOrNull != null) { + missingKeywordMap.getOrElse(symNameOrNull, { + fail(s"Cannot get keyword literals from a symbol `$symNameOrNull`, " + + "so please add them in `missingKeywordMap` manually") + }) + } else { + null + } + } + } + Option(litNamesOrNull).map(_.map(_.toLowerCase(Locale.ROOT))) + } + literals.flatten.filter(_.matches("[a-z_]+")).toSet + } + // Reserved keywords Spark has based on SQL:2011 val reservedKeywordsInAnsiMode = Set( "all", "and", @@ -562,6 +376,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "except", "false", "fetch", + "filter", "for", "foreign", "from",