Add table system.keywords [Part 1] (#51808)

This commit is contained in:
Nikita Mikhaylov 2024-03-18 16:58:57 +01:00 committed by GitHub
parent d4895c2e52
commit 15896ecaf6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
102 changed files with 1848 additions and 1077 deletions

View File

@ -2,6 +2,7 @@
#include "Commands.h"
#include <queue>
#include "KeeperClient.h"
#include "Parsers/CommonParsers.h"
namespace DB
@ -106,13 +107,13 @@ bool CreateCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> &
int mode = zkutil::CreateMode::Persistent;
if (ParserKeyword{"PERSISTENT"}.ignore(pos, expected))
if (ParserKeyword(Keyword::PERSISTENT).ignore(pos, expected))
mode = zkutil::CreateMode::Persistent;
else if (ParserKeyword{"EPHEMERAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::EPHEMERAL).ignore(pos, expected))
mode = zkutil::CreateMode::Ephemeral;
else if (ParserKeyword{"EPHEMERAL SEQUENTIAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::EPHEMERAL_SEQUENTIAL).ignore(pos, expected))
mode = zkutil::CreateMode::EphemeralSequential;
else if (ParserKeyword{"PERSISTENT SEQUENTIAL"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::PERSISTENT_SEQUENTIAL).ignore(pos, expected))
mode = zkutil::CreateMode::PersistentSequential;
node->args.push_back(std::move(mode));
@ -382,12 +383,16 @@ void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) co
bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr<ASTKeeperQuery> & node, DB::Expected & expected) const
{
ParserKeyword s_add(Keyword::ADD);
ParserKeyword s_remove(Keyword::REMOVE);
ParserKeyword s_set(Keyword::SET);
ReconfigCommand::Operation operation;
if (ParserKeyword{"ADD"}.ignore(pos, expected))
if (s_add.ignore(pos, expected))
operation = ReconfigCommand::Operation::ADD;
else if (ParserKeyword{"REMOVE"}.ignore(pos, expected))
else if (s_remove.ignore(pos, expected))
operation = ReconfigCommand::Operation::REMOVE;
else if (ParserKeyword{"SET"}.ignore(pos, expected))
else if (s_set.ignore(pos, expected))
operation = ReconfigCommand::Operation::SET;
else
return false;

View File

@ -13,63 +13,63 @@ namespace ErrorCodes
const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType type_)
{
static constexpr auto make_info = [](const char * raw_name_, bool is_password_ = false)
static constexpr auto make_info = [](Keyword keyword_, bool is_password_ = false)
{
String init_name = raw_name_;
String init_name = String(toStringView(keyword_));
boost::to_lower(init_name);
return AuthenticationTypeInfo{raw_name_, std::move(init_name), is_password_};
return AuthenticationTypeInfo{keyword_, std::move(init_name), is_password_};
};
switch (type_)
{
case AuthenticationType::NO_PASSWORD:
{
static const auto info = make_info("NO_PASSWORD");
static const auto info = make_info(Keyword::NO_PASSWORD);
return info;
}
case AuthenticationType::PLAINTEXT_PASSWORD:
{
static const auto info = make_info("PLAINTEXT_PASSWORD", true);
static const auto info = make_info(Keyword::PLAINTEXT_PASSWORD, true);
return info;
}
case AuthenticationType::SHA256_PASSWORD:
{
static const auto info = make_info("SHA256_PASSWORD", true);
static const auto info = make_info(Keyword::SHA256_PASSWORD, true);
return info;
}
case AuthenticationType::DOUBLE_SHA1_PASSWORD:
{
static const auto info = make_info("DOUBLE_SHA1_PASSWORD", true);
static const auto info = make_info(Keyword::DOUBLE_SHA1_PASSWORD, true);
return info;
}
case AuthenticationType::LDAP:
{
static const auto info = make_info("LDAP");
static const auto info = make_info(Keyword::LDAP);
return info;
}
case AuthenticationType::KERBEROS:
{
static const auto info = make_info("KERBEROS");
static const auto info = make_info(Keyword::KERBEROS);
return info;
}
case AuthenticationType::SSL_CERTIFICATE:
{
static const auto info = make_info("SSL_CERTIFICATE");
static const auto info = make_info(Keyword::SSL_CERTIFICATE);
return info;
}
case AuthenticationType::BCRYPT_PASSWORD:
{
static const auto info = make_info("BCRYPT_PASSWORD", true);
static const auto info = make_info(Keyword::BCRYPT_PASSWORD, true);
return info;
}
case AuthenticationType::SSH_KEY:
{
static const auto info = make_info("SSH_KEY");
static const auto info = make_info(Keyword::SSH_KEY);
return info;
}
case AuthenticationType::HTTP:
{
static const auto info = make_info("HTTP");
static const auto info = make_info(Keyword::HTTP);
return info;
}
case AuthenticationType::MAX:

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/types.h>
#include <Parsers/CommonParsers.h>
namespace DB
{
@ -45,7 +46,7 @@ enum class AuthenticationType
struct AuthenticationTypeInfo
{
const char * const raw_name;
Keyword keyword; // Keyword used in parser
const String name; /// Lowercased with underscores, e.g. "sha256_password".
bool is_password;
static const AuthenticationTypeInfo & get(AuthenticationType type_);
@ -53,7 +54,7 @@ struct AuthenticationTypeInfo
inline String toString(AuthenticationType type_)
{
return AuthenticationTypeInfo::get(type_).raw_name;
return String(toStringView(AuthenticationTypeInfo::get(type_).keyword));
}
}

View File

@ -12,7 +12,7 @@ struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash,
using Base::Base;
static constexpr bool need_zero_value_storage = false;
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const Key & getKey(const value_type & value_) { return value_.first; }
};
@ -32,7 +32,7 @@ struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16,
void setZero() { this->value.first.items[1] = 0; }
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const StringKey16 & getKey(const value_type & value_) { return value_.first; }
};
@ -53,7 +53,7 @@ struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24,
void setZero() { this->value.first.c = 0; }
// external
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringView(this->value.first); } /// NOLINT
// internal
static const StringKey24 & getKey(const value_type & value_) { return value_.first; }
};

View File

@ -19,7 +19,7 @@ struct StringKey24
bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
};
inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey8 & n)
{
assert(n != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
@ -28,7 +28,7 @@ inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
return {reinterpret_cast<const char *>(&n), 8ul - (std::countl_zero(n) >> 3)};
#endif
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey16 & n)
{
assert(n.items[1] != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
@ -37,7 +37,7 @@ inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
return {reinterpret_cast<const char *>(&n), 16ul - (std::countl_zero(n.items[1]) >> 3)};
#endif
}
inline StringRef ALWAYS_INLINE toStringRef(const StringKey24 & n)
inline StringRef ALWAYS_INLINE toStringView(const StringKey24 & n)
{
assert(n.c != 0);
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__

View File

@ -240,7 +240,7 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
return "toDayOfMonth";
case IntervalKind::Kind::Week:
// TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15')))
// else if (ParserKeyword("WEEK").ignore(pos, expected))
// else if (ParserKeyword(Keyword::WEEK).ignore(pos, expected))
// function_name = "toRelativeWeekNum";
throw Exception(ErrorCodes::SYNTAX_ERROR, "The syntax 'EXTRACT(WEEK FROM date)' is not supported, cannot extract the number of a week");
case IntervalKind::Kind::Month:

View File

@ -12,23 +12,23 @@ StorageID tryParseTableIDFromDDL(const String & query, const String & default_da
Tokens tokens(query.data(), query.data() + query.size());
IParser::Pos pos(tokens, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
Expected expected;
if (ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos, expected) || ParserKeyword("CREATE TABLE").ignore(pos, expected))
if (ParserKeyword(Keyword::CREATE_TEMPORARY_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::CREATE_TABLE).ignore(pos, expected))
{
ParserKeyword("IF NOT EXISTS").ignore(pos, expected);
ParserKeyword(Keyword::IF_NOT_EXISTS).ignore(pos, expected);
is_ddl = true;
}
else if (ParserKeyword("ALTER TABLE").ignore(pos, expected) || ParserKeyword("RENAME TABLE").ignore(pos, expected))
else if (ParserKeyword(Keyword::ALTER_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::RENAME_TABLE).ignore(pos, expected))
{
is_ddl = true;
}
else if (ParserKeyword("DROP TABLE").ignore(pos, expected) || ParserKeyword("DROP TEMPORARY TABLE").ignore(pos, expected))
else if (ParserKeyword(Keyword::DROP_TABLE).ignore(pos, expected) || ParserKeyword(Keyword::DROP_TEMPORARY_TABLE).ignore(pos, expected))
{
ParserKeyword("IF EXISTS").ignore(pos, expected);
ParserKeyword(Keyword::IF_EXISTS).ignore(pos, expected);
is_ddl = true;
}
else if (ParserKeyword("TRUNCATE").ignore(pos, expected))
else if (ParserKeyword(Keyword::TRUNCATE).ignore(pos, expected))
{
ParserKeyword("TABLE").ignore(pos, expected);
ParserKeyword(Keyword::TABLE).ignore(pos, expected);
is_ddl = true;
}

View File

@ -2094,7 +2094,6 @@ struct Transformer
|| std::is_same_v<Additions, DateTimeAccurateOrNullConvertStrategyAdditions>)
{
bool is_valid_input = vec_from[i] >= 0 && vec_from[i] <= 0xFFFFFFFFL;
if (!is_valid_input)
{
if constexpr (std::is_same_v<Additions, DateTimeAccurateOrNullConvertStrategyAdditions>)

View File

@ -55,7 +55,7 @@ bool ParserJSONPathRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
}
else if (pos->type == TokenType::BareWord)
{
if (!ParserKeyword("TO").ignore(pos, expected))
if (!ParserKeyword(Keyword::TO).ignore(pos, expected))
{
return false;
}

View File

@ -77,7 +77,7 @@ ColumnsDescription SessionLogElement::getColumnsDescription()
{"Logout", static_cast<Int8>(SESSION_LOGOUT)}
});
#define AUTH_TYPE_NAME_AND_VALUE(v) std::make_pair(AuthenticationTypeInfo::get(v).raw_name, static_cast<Int8>(v))
#define AUTH_TYPE_NAME_AND_VALUE(v) std::make_pair(toString(v), static_cast<Int8>(v))
auto identified_with_column = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values
{

View File

@ -1,8 +1,18 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/SystemLog.h>
#include <base/scope_guard.h>
#include <Common/logger_useful.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/quoteString.h>
#include <Common/setThreadName.h>
#include <Interpreters/AsynchronousInsertLog.h>
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/BackupLog.h>
#include <Interpreters/BlobStorageLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/CrashLog.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/FilesystemReadPrefetchesLog.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
#include <Interpreters/InterpreterRenameQuery.h>
@ -10,38 +20,31 @@
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/ProcessorsProfileLog.h>
#include <Interpreters/BlobStorageLog.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/QueryViewsLog.h>
#include <Interpreters/S3QueueLog.h>
#include <Interpreters/SessionLog.h>
#include <Interpreters/TextLog.h>
#include <Interpreters/TraceLog.h>
#include <Interpreters/TransactionsInfoLog.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/FilesystemReadPrefetchesLog.h>
#include <Interpreters/S3QueueLog.h>
#include <Interpreters/ZooKeeperLog.h>
#include <Interpreters/BackupLog.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTRenameQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Processors/Executors/PushingPipelineExecutor.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <base/scope_guard.h>
#include <fmt/core.h>
#include <Poco/Util/AbstractConfiguration.h>
#include "Common/quoteString.h"
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/logger_useful.h>
#include <Common/setThreadName.h>
#include <fmt/core.h>
namespace DB
{
@ -91,7 +94,7 @@ namespace
if (!storage_p.parse(pos, storage, expected))
return false;
ParserKeyword s_comment("COMMENT");
ParserKeyword s_comment(Keyword::COMMENT);
ParserStringLiteral string_literal_parser;
ASTPtr comment;

View File

@ -19,7 +19,7 @@ std::string ASTQueryWithOnCluster::getRewrittenQueryWithoutOnCluster(const Witho
bool ASTQueryWithOnCluster::parse(Pos & pos, std::string & cluster_str, Expected & expected)
{
if (!ParserKeyword{"CLUSTER"}.ignore(pos, expected))
if (!ParserKeyword(Keyword::CLUSTER).ignore(pos, expected))
return false;
return parseIdentifierOrStringLiteral(pos, expected, cluster_str);

View File

@ -33,7 +33,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::RENAME_TO}.ignore(pos, expected))
return false;
return parseIdentifierOrStringLiteral(pos, expected, new_name);
@ -44,13 +44,13 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword{"NOT KEYED"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NOT_KEYED}.ignore(pos, expected))
{
key_type = QuotaKeyType::NONE;
return true;
}
if (!ParserKeyword{"KEY BY"}.ignore(pos, expected) && !ParserKeyword{"KEYED BY"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::KEY_BY}.ignore(pos, expected) && !ParserKeyword{Keyword::KEYED_BY}.ignore(pos, expected))
return false;
Strings names;
@ -84,7 +84,7 @@ namespace
{
for (auto qt : collections::range(QuotaType::MAX))
{
if (ParserKeyword{QuotaTypeInfo::get(qt).keyword}.ignore(pos, expected))
if (ParserKeyword::createDeprecated(QuotaTypeInfo::get(qt).keyword).ignore(pos, expected))
{
quota_type = qt;
return true;
@ -141,7 +141,7 @@ namespace
auto parse_limit = [&]
{
max_prefix_encountered |= ParserKeyword{"MAX"}.ignore(pos, expected);
max_prefix_encountered |= ParserKeyword{Keyword::MAX}.ignore(pos, expected);
QuotaType quota_type;
if (!parseQuotaType(pos, expected, quota_type))
@ -153,7 +153,7 @@ namespace
}
else
{
if (!ParserKeyword{"MAX"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::MAX}.ignore(pos, expected))
return false;
}
@ -178,13 +178,13 @@ namespace
auto parse_interval_with_limits = [&]
{
if (!ParserKeyword{"FOR"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::FOR}.ignore(pos, expected))
return false;
ASTCreateQuotaQuery::Limits limits;
limits.randomize_interval = ParserKeyword{"RANDOMIZED"}.ignore(pos, expected);
limits.randomize_interval = ParserKeyword{Keyword::RANDOMIZED}.ignore(pos, expected);
ParserKeyword{"INTERVAL"}.ignore(pos, expected);
ParserKeyword{Keyword::INTERVAL}.ignore(pos, expected);
ASTPtr num_intervals_ast;
if (!ParserNumber{}.parse(pos, num_intervals_ast, expected))
@ -199,11 +199,11 @@ namespace
limits.duration = std::chrono::seconds(static_cast<UInt64>(num_intervals * interval_kind.toAvgSeconds()));
std::vector<std::pair<QuotaType, QuotaValue>> new_limits;
if (ParserKeyword{"NO LIMITS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NO_LIMITS}.ignore(pos, expected))
{
limits.drop = true;
}
else if (ParserKeyword{"TRACKING ONLY"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::TRACKING_ONLY}.ignore(pos, expected))
{
}
else if (parseLimits(pos, expected, new_limits))
@ -232,7 +232,7 @@ namespace
ASTPtr node;
ParserRolesOrUsersSet roles_p;
roles_p.allowAll().allowRoles().allowUsers().allowCurrentUser().useIDMode(id_mode);
if (!ParserKeyword{"TO"}.ignore(pos, expected) || !roles_p.parse(pos, node, expected))
if (!ParserKeyword{Keyword::TO}.ignore(pos, expected) || !roles_p.parse(pos, node, expected))
return false;
roles = std::static_pointer_cast<ASTRolesOrUsersSet>(node);
@ -244,7 +244,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -255,14 +255,14 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH QUOTA"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ATTACH_QUOTA}.ignore(pos, expected))
return false;
}
else
{
if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALTER_QUOTA}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::CREATE_QUOTA}.ignore(pos, expected))
return false;
}
@ -271,14 +271,14 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_NOT_EXISTS}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::OR_REPLACE}.ignore(pos, expected))
or_replace = true;
}
@ -313,7 +313,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
continue;
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
break;

View File

@ -18,7 +18,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::RENAME_TO}.ignore(pos, expected))
return false;
return parseRoleName(pos, expected, new_name);
@ -29,7 +29,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SETTINGS}.ignore(pos, expected))
return false;
ASTPtr new_settings_ast;
@ -47,7 +47,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -58,14 +58,14 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH ROLE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ATTACH_ROLE}.ignore(pos, expected))
return false;
}
else
{
if (ParserKeyword{"ALTER ROLE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALTER_ROLE}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE ROLE"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::CREATE_ROLE}.ignore(pos, expected))
return false;
}
@ -74,14 +74,14 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_NOT_EXISTS}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::OR_REPLACE}.ignore(pos, expected))
or_replace = true;
}
@ -112,7 +112,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
continue;
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
break;

View File

@ -23,7 +23,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::RENAME_TO}.ignore(pos, expected))
return false;
return parseIdentifierOrStringLiteral(pos, expected, new_short_name);
@ -34,16 +34,16 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"AS"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::AS}.ignore(pos, expected))
return false;
if (ParserKeyword{"RESTRICTIVE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::RESTRICTIVE}.ignore(pos, expected))
{
is_restrictive = true;
return true;
}
if (!ParserKeyword{"PERMISSIVE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::PERMISSIVE}.ignore(pos, expected))
return false;
is_restrictive = false;
@ -55,7 +55,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword("NONE").ignore(pos, expected))
if (ParserKeyword(Keyword::NONE).ignore(pos, expected))
{
expr = nullptr;
return true;
@ -89,7 +89,7 @@ namespace
auto parse_command = [&]
{
if (ParserKeyword{"ALL"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALL}.ignore(pos, expected))
{
addAllCommands(res_commands);
return true;
@ -98,7 +98,7 @@ namespace
for (auto filter_type : collections::range(RowPolicyFilterType::MAX))
{
std::string_view command = RowPolicyFilterTypeInfo::get(filter_type).command;
if (ParserKeyword{command.data()}.ignore(pos, expected))
if (ParserKeyword::createDeprecated(command.data()).ignore(pos, expected))
{
res_commands.emplace(command);
return true;
@ -125,7 +125,7 @@ namespace
{
boost::container::flat_set<std::string_view> commands;
if (ParserKeyword{"FOR"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FOR}.ignore(pos, expected))
{
if (!parseCommands(pos, expected, commands))
return false;
@ -135,12 +135,12 @@ namespace
std::optional<ASTPtr> filter;
std::optional<ASTPtr> check;
if (ParserKeyword{"USING"}.ignore(pos, expected))
if (ParserKeyword{Keyword::USING}.ignore(pos, expected))
{
if (!parseFilterExpression(pos, expected, filter.emplace()))
return false;
}
if (ParserKeyword{"WITH CHECK"}.ignore(pos, expected))
if (ParserKeyword{Keyword::WITH_CHECK}.ignore(pos, expected))
{
if (!parseFilterExpression(pos, expected, check.emplace()))
return false;
@ -179,7 +179,7 @@ namespace
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserKeyword{"TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::TO}.ignore(pos, expected))
return false;
ParserRolesOrUsersSet roles_p;
@ -196,7 +196,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -207,14 +207,14 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH POLICY"}.ignore(pos, expected) && !ParserKeyword{"ATTACH ROW POLICY"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ATTACH_POLICY}.ignore(pos, expected) && !ParserKeyword{Keyword::ATTACH_ROW_POLICY}.ignore(pos, expected))
return false;
}
else
{
if (ParserKeyword{"ALTER POLICY"}.ignore(pos, expected) || ParserKeyword{"ALTER ROW POLICY"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALTER_POLICY}.ignore(pos, expected) || ParserKeyword{Keyword::ALTER_ROW_POLICY}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE POLICY"}.ignore(pos, expected) && !ParserKeyword{"CREATE ROW POLICY"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::CREATE_POLICY}.ignore(pos, expected) && !ParserKeyword{Keyword::CREATE_ROW_POLICY}.ignore(pos, expected))
return false;
}
@ -223,14 +223,14 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_NOT_EXISTS}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::OR_REPLACE}.ignore(pos, expected))
or_replace = true;
}
@ -273,7 +273,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
continue;
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
break;

View File

@ -20,7 +20,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::RENAME_TO}.ignore(pos, expected))
return false;
return parseIdentifierOrStringLiteral(pos, expected, new_name);
@ -31,7 +31,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SETTINGS}.ignore(pos, expected))
return false;
ASTPtr new_settings_ast;
@ -50,7 +50,7 @@ namespace
return IParserBase::wrapParseImpl(pos, [&]
{
ASTPtr ast;
if (!ParserKeyword{"TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::TO}.ignore(pos, expected))
return false;
ParserRolesOrUsersSet roles_p;
@ -67,7 +67,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -78,14 +78,14 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH SETTINGS PROFILE"}.ignore(pos, expected) && !ParserKeyword{"ATTACH PROFILE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ATTACH_SETTINGS_PROFILE}.ignore(pos, expected) && !ParserKeyword{Keyword::ATTACH_PROFILE}.ignore(pos, expected))
return false;
}
else
{
if (ParserKeyword{"ALTER SETTINGS PROFILE"}.ignore(pos, expected) || ParserKeyword{"ALTER PROFILE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALTER_SETTINGS_PROFILE}.ignore(pos, expected) || ParserKeyword{Keyword::ALTER_PROFILE}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE SETTINGS PROFILE"}.ignore(pos, expected) && !ParserKeyword{"CREATE PROFILE"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::CREATE_SETTINGS_PROFILE}.ignore(pos, expected) && !ParserKeyword{Keyword::CREATE_PROFILE}.ignore(pos, expected))
return false;
}
@ -94,14 +94,14 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_NOT_EXISTS}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::OR_REPLACE}.ignore(pos, expected))
or_replace = true;
}
@ -132,7 +132,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
if (cluster.empty() && parseOnCluster(pos, expected, cluster))
continue;
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
break;

View File

@ -32,7 +32,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::RENAME_TO}.ignore(pos, expected))
return false;
String maybe_new_name;
@ -48,7 +48,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword{"NOT IDENTIFIED"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NOT_IDENTIFIED}.ignore(pos, expected))
{
auth_data = std::make_shared<ASTAuthenticationData>();
auth_data->type = AuthenticationType::NO_PASSWORD;
@ -56,7 +56,7 @@ namespace
return true;
}
if (!ParserKeyword{"IDENTIFIED"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::IDENTIFIED}.ignore(pos, expected))
return false;
std::optional<AuthenticationType> type;
@ -69,11 +69,11 @@ namespace
bool expect_public_ssh_key = false;
bool expect_http_auth_server = false;
if (ParserKeyword{"WITH"}.ignore(pos, expected))
if (ParserKeyword{Keyword::WITH}.ignore(pos, expected))
{
for (auto check_type : collections::range(AuthenticationType::MAX))
{
if (ParserKeyword{AuthenticationTypeInfo::get(check_type).raw_name}.ignore(pos, expected))
if (ParserKeyword{AuthenticationTypeInfo::get(check_type).keyword}.ignore(pos, expected))
{
type = check_type;
@ -96,17 +96,17 @@ namespace
if (!type)
{
if (ParserKeyword{"SHA256_HASH"}.ignore(pos, expected))
if (ParserKeyword{Keyword::SHA256_HASH}.ignore(pos, expected))
{
type = AuthenticationType::SHA256_PASSWORD;
expect_hash = true;
}
else if (ParserKeyword{"DOUBLE_SHA1_HASH"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::DOUBLE_SHA1_HASH}.ignore(pos, expected))
{
type = AuthenticationType::DOUBLE_SHA1_PASSWORD;
expect_hash = true;
}
else if (ParserKeyword{"BCRYPT_HASH"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::BCRYPT_HASH}.ignore(pos, expected))
{
type = AuthenticationType::BCRYPT_PASSWORD;
expect_hash = true;
@ -128,12 +128,12 @@ namespace
if (expect_password || expect_hash)
{
if (!ParserKeyword{"BY"}.ignore(pos, expected) || !ParserStringAndSubstitution{}.parse(pos, value, expected))
if (!ParserKeyword{Keyword::BY}.ignore(pos, expected) || !ParserStringAndSubstitution{}.parse(pos, value, expected))
return false;
if (expect_hash && type == AuthenticationType::SHA256_PASSWORD)
{
if (ParserKeyword{"SALT"}.ignore(pos, expected))
if (ParserKeyword{Keyword::SALT}.ignore(pos, expected))
{
if (!ParserStringAndSubstitution{}.parse(pos, parsed_salt, expected))
return false;
@ -142,12 +142,12 @@ namespace
}
else if (expect_ldap_server_name)
{
if (!ParserKeyword{"SERVER"}.ignore(pos, expected) || !ParserStringAndSubstitution{}.parse(pos, value, expected))
if (!ParserKeyword{Keyword::SERVER}.ignore(pos, expected) || !ParserStringAndSubstitution{}.parse(pos, value, expected))
return false;
}
else if (expect_kerberos_realm)
{
if (ParserKeyword{"REALM"}.ignore(pos, expected))
if (ParserKeyword{Keyword::REALM}.ignore(pos, expected))
{
if (!ParserStringAndSubstitution{}.parse(pos, value, expected))
return false;
@ -155,7 +155,7 @@ namespace
}
else if (expect_common_names)
{
if (!ParserKeyword{"CN"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::CN}.ignore(pos, expected))
return false;
if (!ParserList{std::make_unique<ParserStringAndSubstitution>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, common_names, expected))
@ -163,7 +163,7 @@ namespace
}
else if (expect_public_ssh_key)
{
if (!ParserKeyword{"BY"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::BY}.ignore(pos, expected))
return false;
if (!ParserList{std::make_unique<ParserPublicSSHKey>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, common_names, expected))
@ -171,12 +171,12 @@ namespace
}
else if (expect_http_auth_server)
{
if (!ParserKeyword{"SERVER"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SERVER}.ignore(pos, expected))
return false;
if (!ParserStringAndSubstitution{}.parse(pos, value, expected))
return false;
if (ParserKeyword{"SCHEME"}.ignore(pos, expected))
if (ParserKeyword{Keyword::SCHEME}.ignore(pos, expected))
{
if (!ParserStringAndSubstitution{}.parse(pos, http_auth_scheme, expected))
return false;
@ -215,22 +215,22 @@ namespace
auto parse_host = [&]
{
if (ParserKeyword{"NONE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NONE}.ignore(pos, expected))
return true;
if (ParserKeyword{"ANY"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ANY}.ignore(pos, expected))
{
res_hosts.addAnyHost();
return true;
}
if (ParserKeyword{"LOCAL"}.ignore(pos, expected))
if (ParserKeyword{Keyword::LOCAL}.ignore(pos, expected))
{
res_hosts.addLocalHost();
return true;
}
if (ParserKeyword{"REGEXP"}.ignore(pos, expected))
if (ParserKeyword{Keyword::REGEXP}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
@ -241,7 +241,7 @@ namespace
return true;
}
if (ParserKeyword{"NAME"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NAME}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
@ -253,7 +253,7 @@ namespace
return true;
}
if (ParserKeyword{"IP"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IP}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
@ -265,7 +265,7 @@ namespace
return true;
}
if (ParserKeyword{"LIKE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::LIKE}.ignore(pos, expected))
{
ASTPtr ast;
if (!ParserList{std::make_unique<ParserStringLiteral>(), std::make_unique<ParserToken>(TokenType::Comma), false}.parse(pos, ast, expected))
@ -292,10 +292,10 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!prefix.empty() && !ParserKeyword{prefix}.ignore(pos, expected))
if (!prefix.empty() && !ParserKeyword::createDeprecated(prefix).ignore(pos, expected))
return false;
if (!ParserKeyword{"HOST"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::HOST}.ignore(pos, expected))
return false;
AllowedClientHosts res_hosts;
@ -312,7 +312,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"DEFAULT ROLE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::DEFAULT_ROLE}.ignore(pos, expected))
return false;
ASTPtr ast;
@ -332,7 +332,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SETTINGS}.ignore(pos, expected))
return false;
ASTPtr new_settings_ast;
@ -350,7 +350,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"GRANTEES"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::GRANTEES}.ignore(pos, expected))
return false;
ASTPtr ast;
@ -368,7 +368,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
@ -376,7 +376,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"DEFAULT DATABASE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::DEFAULT_DATABASE}.ignore(pos, expected))
return false;
ASTPtr ast;
@ -393,7 +393,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"VALID UNTIL"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::VALID_UNTIL}.ignore(pos, expected))
return false;
ParserStringAndSubstitution until_p;
@ -409,14 +409,14 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool alter = false;
if (attach_mode)
{
if (!ParserKeyword{"ATTACH USER"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ATTACH_USER}.ignore(pos, expected))
return false;
}
else
{
if (ParserKeyword{"ALTER USER"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ALTER_USER}.ignore(pos, expected))
alter = true;
else if (!ParserKeyword{"CREATE USER"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::CREATE_USER}.ignore(pos, expected))
return false;
}
@ -425,14 +425,14 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool or_replace = false;
if (alter)
{
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
}
else
{
if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_NOT_EXISTS}.ignore(pos, expected))
if_not_exists = true;
else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::OR_REPLACE}.ignore(pos, expected))
or_replace = true;
}
@ -508,7 +508,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!new_name && (names->size() == 1) && parseRenameTo(pos, expected, new_name))
continue;
if (parseHosts(pos, expected, "ADD", new_hosts))
if (parseHosts(pos, expected, toStringView(Keyword::ADD), new_hosts))
{
if (!add_hosts)
add_hosts.emplace();
@ -516,7 +516,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
continue;
}
if (parseHosts(pos, expected, "DROP", new_hosts))
if (parseHosts(pos, expected, toStringView(Keyword::DROP), new_hosts))
{
if (!remove_hosts)
remove_hosts.emplace();
@ -525,7 +525,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
}
if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
if (storage_name.empty() && ParserKeyword{Keyword::IN}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name))
continue;
break;

View File

@ -18,8 +18,8 @@ namespace
for (auto i : collections::range(AccessEntityType::MAX))
{
const auto & type_info = AccessEntityTypeInfo::get(i);
if (ParserKeyword{type_info.name}.ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected)))
if (ParserKeyword::createDeprecated(type_info.name).ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword::createDeprecated(type_info.alias).ignore(pos, expected)))
{
type = i;
return true;
@ -33,7 +33,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -41,7 +41,7 @@ namespace
bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"DROP"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::DROP}.ignore(pos, expected))
return false;
AccessEntityType type;
@ -49,7 +49,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
bool if_exists = false;
if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IF_EXISTS}.ignore(pos, expected))
if_exists = true;
Strings names;
@ -78,7 +78,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
}
if (ParserKeyword{"FROM"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FROM}.ignore(pos, expected))
parseAccessStorageName(pos, expected, storage_name);
if (cluster.empty())

View File

@ -28,7 +28,7 @@ namespace
if (pos_->type != TokenType::BareWord)
return false;
std::string_view word{pos_->begin, pos_->size()};
return !(boost::iequals(word, "ON") || boost::iequals(word, "TO") || boost::iequals(word, "FROM"));
return !(boost::iequals(word, toStringView(Keyword::ON)) || boost::iequals(word, toStringView(Keyword::TO)) || boost::iequals(word, toStringView(Keyword::FROM)));
};
expected.add(pos, "access type");
@ -132,7 +132,7 @@ namespace
++is_global_with_parameter;
}
if (!ParserKeyword{"ON"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
return false;
if (is_global_with_parameter && is_global_with_parameter == access_and_columns.size())
@ -197,7 +197,7 @@ namespace
{
AccessRightsElement default_element(AccessType::ALL);
if (!ParserKeyword{"ON"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
return false;
String database_name;
@ -265,7 +265,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{is_revoke ? "FROM" : "TO"}.ignore(pos, expected))
if (!ParserKeyword{is_revoke ? Keyword::FROM : Keyword::TO}.ignore(pos, expected))
return false;
ASTPtr ast;
@ -283,7 +283,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -291,14 +291,14 @@ namespace
bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (attach_mode && !ParserKeyword{"ATTACH"}.ignore(pos, expected))
if (attach_mode && !ParserKeyword{Keyword::ATTACH}.ignore(pos, expected))
return false;
bool is_replace = false;
bool is_revoke = false;
if (ParserKeyword{"REVOKE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::REVOKE}.ignore(pos, expected))
is_revoke = true;
else if (!ParserKeyword{"GRANT"}.ignore(pos, expected))
else if (!ParserKeyword{Keyword::GRANT}.ignore(pos, expected))
return false;
String cluster;
@ -308,9 +308,9 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool admin_option = false;
if (is_revoke)
{
if (ParserKeyword{"GRANT OPTION FOR"}.ignore(pos, expected))
if (ParserKeyword{Keyword::GRANT_OPTION_FOR}.ignore(pos, expected))
grant_option = true;
else if (ParserKeyword{"ADMIN OPTION FOR"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::ADMIN_OPTION_FOR}.ignore(pos, expected))
admin_option = true;
}
@ -318,7 +318,7 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
std::shared_ptr<ASTRolesOrUsersSet> roles;
bool current_grants = false;
if (!is_revoke && ParserKeyword{"CURRENT GRANTS"}.ignore(pos, expected))
if (!is_revoke && ParserKeyword{Keyword::CURRENT_GRANTS}.ignore(pos, expected))
{
current_grants = true;
if (!parseCurrentGrants(pos, expected, elements))
@ -342,12 +342,12 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!is_revoke)
{
if (ParserKeyword{"WITH GRANT OPTION"}.ignore(pos, expected))
if (ParserKeyword{Keyword::WITH_GRANT_OPTION}.ignore(pos, expected))
grant_option = true;
else if (ParserKeyword{"WITH ADMIN OPTION"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::WITH_ADMIN_OPTION}.ignore(pos, expected))
admin_option = true;
if (ParserKeyword{"WITH REPLACE OPTION"}.ignore(pos, expected))
if (ParserKeyword{Keyword::WITH_REPLACE_OPTION}.ignore(pos, expected))
is_replace = true;
}

View File

@ -18,8 +18,8 @@ namespace
for (auto i : collections::range(AccessEntityType::MAX))
{
const auto & type_info = AccessEntityTypeInfo::get(i);
if (ParserKeyword{type_info.name}.ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected)))
if (ParserKeyword::createDeprecated(type_info.name).ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword::createDeprecated(type_info.alias).ignore(pos, expected)))
{
type = i;
return true;
@ -33,7 +33,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -41,7 +41,7 @@ namespace
bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"MOVE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::MOVE}.ignore(pos, expected))
return false;
AccessEntityType type;
@ -74,7 +74,7 @@ bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
}
if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name))
if (!ParserKeyword{Keyword::TO}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name))
return false;
if (cluster.empty())

View File

@ -15,11 +15,11 @@ namespace
return IParserBase::wrapParseImpl(pos, [&]
{
String key_base64;
if (!ParserKeyword{"KEY"}.ignore(pos, expected) || !parseIdentifierOrStringLiteral(pos, expected, key_base64))
if (!ParserKeyword{Keyword::KEY}.ignore(pos, expected) || !parseIdentifierOrStringLiteral(pos, expected, key_base64))
return false;
String type;
if (!ParserKeyword{"TYPE"}.ignore(pos, expected) || !parseIdentifierOrStringLiteral(pos, expected, type))
if (!ParserKeyword{Keyword::TYPE}.ignore(pos, expected) || !parseIdentifierOrStringLiteral(pos, expected, type))
return false;
ast = std::make_shared<ASTPublicSSHKey>();

View File

@ -19,7 +19,7 @@ namespace
if (!id_mode)
return parseRoleName(pos, expected, res);
if (!ParserKeyword{"ID"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ID}.ignore(pos, expected))
return false;
if (!ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
return false;
@ -53,16 +53,16 @@ namespace
auto parse_element = [&]
{
if (ParserKeyword{"NONE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NONE}.ignore(pos, expected))
return true;
if (allow_all && ParserKeyword{"ALL"}.ignore(pos, expected))
if (allow_all && ParserKeyword{Keyword::ALL}.ignore(pos, expected))
{
res_all = true;
return true;
}
if (allow_any && ParserKeyword{"ANY"}.ignore(pos, expected))
if (allow_any && ParserKeyword{Keyword::ANY}.ignore(pos, expected))
{
res_all = true;
return true;
@ -102,7 +102,7 @@ namespace
bool & except_current_user)
{
return IParserBase::wrapParseImpl(pos, [&] {
if (!ParserKeyword{"EXCEPT"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::EXCEPT}.ignore(pos, expected))
return false;
bool unused;

View File

@ -16,7 +16,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
@ -39,13 +39,13 @@ namespace
res_table_name = RowPolicyName::ANY_TABLE_MARK;
}
/// If table is specified without DB it cannot be followed by "ON"
/// (but can be followed by "ON CLUSTER").
/// If table is specified without DB it cannot be followed by Keyword::ON
/// (but can be followed by Keyword::ON CLUSTER).
/// The following code is necessary to figure out while parsing something like
/// policy1 ON table1, policy2 ON table2
/// that policy2 is another policy, not another table.
auto end_pos = pos;
if (res_database.empty() && ParserKeyword{"ON"}.ignore(pos, expected))
if (res_database.empty() && ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
String unused;
if (ASTQueryWithOnCluster::parse(pos, unused, expected))
@ -65,7 +65,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && parseDBAndTableName(pos, expected, database, table_name);
return ParserKeyword{Keyword::ON}.ignore(pos, expected) && parseDBAndTableName(pos, expected, database, table_name);
});
}
@ -74,7 +74,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"ON"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ON}.ignore(pos, expected))
return false;
std::vector<std::pair<String, String>> res;

View File

@ -29,7 +29,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"TO"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::TO}.ignore(pos, expected))
return false;
ASTPtr ast;
@ -50,11 +50,11 @@ bool ParserSetRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
{
using Kind = ASTSetRoleQuery::Kind;
Kind kind;
if (ParserKeyword{"SET ROLE DEFAULT"}.ignore(pos, expected))
if (ParserKeyword{Keyword::SET_ROLE_DEFAULT}.ignore(pos, expected))
kind = Kind::SET_ROLE_DEFAULT;
else if (ParserKeyword{"SET ROLE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::SET_ROLE}.ignore(pos, expected))
kind = Kind::SET_ROLE;
else if (ParserKeyword{"SET DEFAULT ROLE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::SET_DEFAULT_ROLE}.ignore(pos, expected))
kind = Kind::SET_DEFAULT_ROLE;
else
return false;

View File

@ -15,12 +15,12 @@ namespace
{
bool parseProfileKeyword(IParserBase::Pos & pos, Expected & expected, bool use_inherit_keyword)
{
if (ParserKeyword{"PROFILE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::PROFILE}.ignore(pos, expected))
return true;
if (use_inherit_keyword && ParserKeyword{"INHERIT"}.ignore(pos, expected))
if (use_inherit_keyword && ParserKeyword{Keyword::INHERIT}.ignore(pos, expected))
{
ParserKeyword{"PROFILE"}.ignore(pos, expected);
ParserKeyword{Keyword::PROFILE}.ignore(pos, expected);
return true;
}
@ -36,7 +36,7 @@ namespace
if (!id_mode)
return parseIdentifierOrStringLiteral(pos, expected, res);
if (!ParserKeyword{"ID"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ID}.ignore(pos, expected))
return false;
if (!ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
return false;
@ -73,8 +73,8 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
bool is_min_value = ParserKeyword{"MIN"}.ignore(pos, expected);
bool is_max_value = !is_min_value && ParserKeyword{"MAX"}.ignore(pos, expected);
bool is_min_value = ParserKeyword{Keyword::MIN}.ignore(pos, expected);
bool is_max_value = !is_min_value && ParserKeyword{Keyword::MAX}.ignore(pos, expected);
if (!is_min_value && !is_max_value)
return false;
@ -99,17 +99,17 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword{"READONLY"}.ignore(pos, expected) || ParserKeyword{"CONST"}.ignore(pos, expected))
if (ParserKeyword{Keyword::READONLY}.ignore(pos, expected) || ParserKeyword{Keyword::CONST}.ignore(pos, expected))
{
writability = SettingConstraintWritability::CONST;
return true;
}
else if (ParserKeyword{"WRITABLE"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::WRITABLE}.ignore(pos, expected))
{
writability = SettingConstraintWritability::WRITABLE;
return true;
}
else if (ParserKeyword{"CHANGEABLE_IN_READONLY"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::CHANGEABLE_IN_READONLY}.ignore(pos, expected))
{
writability = SettingConstraintWritability::CHANGEABLE_IN_READONLY;
return true;
@ -151,7 +151,7 @@ namespace
if (!has_value_or_constraint)
return false;
if (boost::iequals(res_setting_name, "PROFILE") && !res_value && !res_min_value && !res_max_value
if (boost::iequals(res_setting_name, toStringView(Keyword::PROFILE)) && !res_value && !res_min_value && !res_max_value
&& res_writability == SettingConstraintWritability::CONST)
{
/// Ambiguity: "profile readonly" can be treated either as a profile named "readonly" or
@ -224,7 +224,7 @@ bool ParserSettingsProfileElements::parseImpl(Pos & pos, ASTPtr & node, Expected
{
std::vector<std::shared_ptr<ASTSettingsProfileElement>> elements;
if (ParserKeyword{"NONE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::NONE}.ignore(pos, expected))
{
}
else

View File

@ -15,8 +15,8 @@ namespace
for (auto i : collections::range(AccessEntityType::MAX))
{
const auto & type_info = AccessEntityTypeInfo::get(i);
if (ParserKeyword{type_info.plural_name}.ignore(pos, expected)
|| (!type_info.plural_alias.empty() && ParserKeyword{type_info.plural_alias}.ignore(pos, expected)))
if (ParserKeyword::createDeprecated(type_info.plural_name).ignore(pos, expected)
|| (!type_info.plural_alias.empty() && ParserKeyword::createDeprecated(type_info.plural_alias).ignore(pos, expected)))
{
type = i;
return true;
@ -29,7 +29,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected)
return ParserKeyword{Keyword::ON}.ignore(pos, expected)
&& parseDatabaseAndTableNameOrAsterisks(pos, expected, database, any_database, table, any_table);
});
}
@ -38,7 +38,7 @@ namespace
bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SHOW"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SHOW}.ignore(pos, expected))
return false;
AccessEntityType type;
@ -51,17 +51,17 @@ bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected
{
all = true;
}
else if (ParserKeyword{"CURRENT ROLES"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::CURRENT_ROLES}.ignore(pos, expected))
{
type = AccessEntityType::ROLE;
current_roles = true;
}
else if (ParserKeyword{"ENABLED ROLES"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::ENABLED_ROLES}.ignore(pos, expected))
{
type = AccessEntityType::ROLE;
enabled_roles = true;
}
else if (ParserKeyword{"CURRENT QUOTA"}.ignore(pos, expected) || ParserKeyword{"QUOTA"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::CURRENT_QUOTA}.ignore(pos, expected) || ParserKeyword{Keyword::QUOTA}.ignore(pos, expected))
{
type = AccessEntityType::QUOTA;
current_quota = true;

View File

@ -20,7 +20,7 @@ protected:
{
auto query = std::make_shared<ASTShowAccessQuery>();
if (!ParserKeyword("SHOW ACCESS").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_ACCESS).ignore(pos, expected))
return false;
node = query;

View File

@ -25,8 +25,8 @@ namespace
for (auto i : collections::range(AccessEntityType::MAX))
{
const auto & type_info = AccessEntityTypeInfo::get(i);
if (ParserKeyword{type_info.name}.ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected)))
if (ParserKeyword::createDeprecated(type_info.name).ignore(pos, expected)
|| (!type_info.alias.empty() && ParserKeyword::createDeprecated(type_info.alias).ignore(pos, expected)))
{
type = i;
plural = false;
@ -37,8 +37,8 @@ namespace
for (auto i : collections::range(AccessEntityType::MAX))
{
const auto & type_info = AccessEntityTypeInfo::get(i);
if (ParserKeyword{type_info.plural_name}.ignore(pos, expected)
|| (!type_info.plural_alias.empty() && ParserKeyword{type_info.plural_alias}.ignore(pos, expected)))
if (ParserKeyword::createDeprecated(type_info.plural_name).ignore(pos, expected)
|| (!type_info.plural_alias.empty() && ParserKeyword::createDeprecated(type_info.plural_alias).ignore(pos, expected)))
{
type = i;
plural = true;
@ -53,7 +53,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected)
return ParserKeyword{Keyword::ON}.ignore(pos, expected)
&& parseDatabaseAndTableNameOrAsterisks(pos, expected, database, any_database, table, any_table);
});
}
@ -62,7 +62,7 @@ namespace
bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SHOW CREATE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SHOW_CREATE}.ignore(pos, expected))
return false;
AccessEntityType type;

View File

@ -10,12 +10,12 @@ namespace DB
{
bool ParserShowGrantsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SHOW GRANTS"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SHOW_GRANTS}.ignore(pos, expected))
return false;
std::shared_ptr<ASTRolesOrUsersSet> for_roles;
if (ParserKeyword{"FOR"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FOR}.ignore(pos, expected))
{
ASTPtr for_roles_ast;
ParserRolesOrUsersSet for_roles_p;

View File

@ -10,7 +10,7 @@ bool ParserShowPrivilegesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
{
auto query = std::make_shared<ASTShowPrivilegesQuery>();
if (!ParserKeyword("SHOW PRIVILEGES").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_PRIVILEGES).ignore(pos, expected))
return false;
node = query;

View File

@ -32,7 +32,7 @@ bool parseCurrentUserTag(IParser::Pos & pos, Expected & expected)
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) && !ParserKeyword{"currentUser"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::CURRENTUSER}.ignore(pos, expected) && !ParserKeyword{Keyword::CURRENT_USER}.ignore(pos, expected))
return false;
if (ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected))

View File

@ -1,9 +1,117 @@
#include <Parsers/CommonParsers.h>
#include <Common/ErrorCodes.h>
#include <base/find_symbols.h>
#include <algorithm>
#include <cctype>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
using Strings = std::vector<String>;
class KeyWordToStringConverter
{
public:
static const KeyWordToStringConverter & instance()
{
static const KeyWordToStringConverter res;
return res;
}
std::string_view convert(Keyword type) const
{
return mapping[static_cast<size_t>(type)];
}
const std::vector<String> & getMapping() const
{
return mapping;
}
private:
KeyWordToStringConverter()
{
#define KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(identifier, value) \
checkUnderscore(value); \
addToMapping(Keyword::identifier, value);
APPLY_FOR_PARSER_KEYWORDS(KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
#undef KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING
#define KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(identifier, value) \
addToMapping(Keyword::identifier, value);
APPLY_FOR_PARSER_KEYWORDS_WITH_UNDERSCORES(KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
#undef KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING
#ifndef NDEBUG
#define KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(identifier, value) \
check(#identifier, value);
APPLY_FOR_PARSER_KEYWORDS(KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
APPLY_FOR_PARSER_KEYWORDS_WITH_UNDERSCORES(KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING)
#undef KEYWORD_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING
#endif
}
void addToMapping(Keyword identifier, std::string_view value)
{
size_t index = static_cast<size_t>(identifier);
mapping.resize(std::max(index + 1, mapping.size()));
mapping[index] = value;
}
void checkUnderscore(std::string_view value)
{
if (value.contains('_'))
throw Exception(ErrorCodes::LOGICAL_ERROR,
"The keyword {} has underscore. If this is intentional, please declare it in another list.", value);
}
[[ maybe_unused ]] void check(std::string_view identifier, std::string_view value)
{
if (value == "TRUE" || value == "FALSE" || value == "NULL")
return;
if (identifier.size() != value.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "The length of the keyword identifier and the length of its value are different.");
for (size_t i = 0; i < identifier.size(); ++i)
{
if (std::tolower(identifier[i]) == '_' && std::tolower(value[i]) == ' ')
continue;
if (std::tolower(identifier[i]) == std::tolower(value[i]))
continue;
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Keyword identifier {} differs from its value {} in {} position: {} and {}",
identifier, value, i, identifier[i], value[i]);
}
}
Strings mapping;
};
}
std::string_view toStringView(Keyword type)
{
return KeyWordToStringConverter::instance().convert(type);
}
const std::vector<String> & getAllKeyWords()
{
return KeyWordToStringConverter::instance().getMapping();
}
ParserKeyword::ParserKeyword(Keyword keyword)
: s(toStringView(keyword))
{}
bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::BareWord)
@ -38,4 +146,5 @@ bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expecte
return true;
}
}

View File

@ -3,10 +3,575 @@
#include <Parsers/IParserBase.h>
#include <cassert>
#include <string_view>
namespace DB
{
#define APPLY_FOR_PARSER_KEYWORDS(MR_MACROS) \
MR_MACROS(ADD_COLUMN, "ADD COLUMN") \
MR_MACROS(ADD_CONSTRAINT, "ADD CONSTRAINT") \
MR_MACROS(ADD_INDEX, "ADD INDEX") \
MR_MACROS(ADD_PROJECTION, "ADD PROJECTION") \
MR_MACROS(ADD_STATISTIC, "ADD STATISTIC") \
MR_MACROS(ADD, "ADD") \
MR_MACROS(ADMIN_OPTION_FOR, "ADMIN OPTION FOR") \
MR_MACROS(AFTER, "AFTER") \
MR_MACROS(ALGORITHM, "ALGORITHM") \
MR_MACROS(ALIAS, "ALIAS") \
MR_MACROS(ALL, "ALL") \
MR_MACROS(ALTER_COLUMN, "ALTER COLUMN") \
MR_MACROS(ALTER_DATABASE, "ALTER DATABASE") \
MR_MACROS(ALTER_LIVE_VIEW, "ALTER LIVE VIEW") \
MR_MACROS(ALTER_POLICY, "ALTER POLICY") \
MR_MACROS(ALTER_PROFILE, "ALTER PROFILE") \
MR_MACROS(ALTER_QUOTA, "ALTER QUOTA") \
MR_MACROS(ALTER_ROLE, "ALTER ROLE") \
MR_MACROS(ALTER_ROW_POLICY, "ALTER ROW POLICY") \
MR_MACROS(ALTER_SETTINGS_PROFILE, "ALTER SETTINGS PROFILE") \
MR_MACROS(ALTER_TABLE, "ALTER TABLE") \
MR_MACROS(ALTER_TEMPORARY_TABLE, "ALTER TEMPORARY TABLE") \
MR_MACROS(ALTER_USER, "ALTER USER") \
MR_MACROS(ALTER, "ALTER") \
MR_MACROS(AND_STDOUT, "AND STDOUT") \
MR_MACROS(AND, "AND") \
MR_MACROS(ANTI, "ANTI") \
MR_MACROS(ANY, "ANY") \
MR_MACROS(APPEND, "APPEND") \
MR_MACROS(APPLY_DELETED_MASK, "APPLY DELETED MASK") \
MR_MACROS(APPLY, "APPLY") \
MR_MACROS(ARRAY_JOIN, "ARRAY JOIN") \
MR_MACROS(AS, "AS") \
MR_MACROS(ASC, "ASC") \
MR_MACROS(ASCENDING, "ASCENDING") \
MR_MACROS(ASOF, "ASOF") \
MR_MACROS(ASSUME, "ASSUME") \
MR_MACROS(AST, "AST") \
MR_MACROS(ASYNC, "ASYNC") \
MR_MACROS(ATTACH_PART, "ATTACH PART") \
MR_MACROS(ATTACH_PARTITION, "ATTACH PARTITION") \
MR_MACROS(ATTACH_POLICY, "ATTACH POLICY") \
MR_MACROS(ATTACH_PROFILE, "ATTACH PROFILE") \
MR_MACROS(ATTACH_QUOTA, "ATTACH QUOTA") \
MR_MACROS(ATTACH_ROLE, "ATTACH ROLE") \
MR_MACROS(ATTACH_ROW_POLICY, "ATTACH ROW POLICY") \
MR_MACROS(ATTACH_SETTINGS_PROFILE, "ATTACH SETTINGS PROFILE") \
MR_MACROS(ATTACH_USER, "ATTACH USER") \
MR_MACROS(ATTACH, "ATTACH") \
MR_MACROS(AZURE, "AZURE") \
MR_MACROS(BACKUP, "BACKUP") \
MR_MACROS(BAGEXPANSION, "bagexpansion") \
MR_MACROS(BEGIN_TRANSACTION, "BEGIN TRANSACTION") \
MR_MACROS(BETWEEN, "BETWEEN") \
MR_MACROS(BIDIRECTIONAL, "BIDIRECTIONAL") \
MR_MACROS(BOTH, "BOTH") \
MR_MACROS(BY, "BY") \
MR_MACROS(CASCADE, "CASCADE") \
MR_MACROS(CASE, "CASE") \
MR_MACROS(CAST, "CAST") \
MR_MACROS(CHANGE, "CHANGE") \
MR_MACROS(CHANGED, "CHANGED") \
MR_MACROS(CHAR_VARYING, "CHAR VARYING") \
MR_MACROS(CHAR, "CHAR") \
MR_MACROS(CHARACTER_LARGE_OBJECT, "CHARACTER LARGE OBJECT") \
MR_MACROS(CHARACTER_VARYING, "CHARACTER VARYING") \
MR_MACROS(CHARACTER, "CHARACTER") \
MR_MACROS(CHECK_ALL_TABLES, "CHECK ALL TABLES") \
MR_MACROS(CHECK_TABLE, "CHECK TABLE") \
MR_MACROS(CHECK, "CHECK") \
MR_MACROS(CLEANUP, "CLEANUP") \
MR_MACROS(CLEAR_COLUMN, "CLEAR COLUMN") \
MR_MACROS(CLEAR_INDEX, "CLEAR INDEX") \
MR_MACROS(CLEAR_PROJECTION, "CLEAR PROJECTION") \
MR_MACROS(CLEAR_STATISTIC, "CLEAR STATISTIC") \
MR_MACROS(CLUSTER, "CLUSTER") \
MR_MACROS(CLUSTERS, "CLUSTERS") \
MR_MACROS(CN, "CN") \
MR_MACROS(CODEC, "CODEC") \
MR_MACROS(COLLATE, "COLLATE") \
MR_MACROS(COLUMN, "COLUMN") \
MR_MACROS(COLUMNS, "COLUMNS") \
MR_MACROS(COMMENT_COLUMN, "COMMENT COLUMN") \
MR_MACROS(COMMENT, "COMMENT") \
MR_MACROS(COMMIT, "COMMIT") \
MR_MACROS(COMPRESSION, "COMPRESSION") \
MR_MACROS(CONST, "CONST") \
MR_MACROS(CONSTRAINT, "CONSTRAINT") \
MR_MACROS(CREATE_POLICY, "CREATE POLICY") \
MR_MACROS(CREATE_PROFILE, "CREATE PROFILE") \
MR_MACROS(CREATE_QUOTA, "CREATE QUOTA") \
MR_MACROS(CREATE_ROLE, "CREATE ROLE") \
MR_MACROS(CREATE_ROW_POLICY, "CREATE ROW POLICY") \
MR_MACROS(CREATE_SETTINGS_PROFILE, "CREATE SETTINGS PROFILE") \
MR_MACROS(CREATE_TABLE, "CREATE TABLE") \
MR_MACROS(CREATE_TEMPORARY_TABLE, "CREATE TEMPORARY TABLE") \
MR_MACROS(CREATE_USER, "CREATE USER") \
MR_MACROS(CREATE, "CREATE") \
MR_MACROS(CROSS, "CROSS") \
MR_MACROS(CUBE, "CUBE") \
MR_MACROS(CURRENT_GRANTS, "CURRENT GRANTS") \
MR_MACROS(CURRENT_QUOTA, "CURRENT QUOTA") \
MR_MACROS(CURRENT_ROLES, "CURRENT ROLES") \
MR_MACROS(CURRENT_ROW, "CURRENT ROW") \
MR_MACROS(CURRENT_TRANSACTION, "CURRENT TRANSACTION") \
MR_MACROS(CURRENTUSER, "CURRENTUSER") \
MR_MACROS(D, "D") \
MR_MACROS(DATABASE, "DATABASE") \
MR_MACROS(DATABASES, "DATABASES") \
MR_MACROS(DATE, "DATE") \
MR_MACROS(DAY, "DAY") \
MR_MACROS(DAYS, "DAYS") \
MR_MACROS(DD, "DD") \
MR_MACROS(DEDUPLICATE, "DEDUPLICATE") \
MR_MACROS(DEFAULT_DATABASE, "DEFAULT DATABASE") \
MR_MACROS(DEFAULT_ROLE, "DEFAULT ROLE") \
MR_MACROS(DEFAULT, "DEFAULT") \
MR_MACROS(DEFINER, "DEFINER") \
MR_MACROS(DELETE, "DELETE") \
MR_MACROS(DEPENDS_ON, "DEPENDS ON") \
MR_MACROS(DESC, "DESC") \
MR_MACROS(DESCENDING, "DESCENDING") \
MR_MACROS(DESCRIBE, "DESCRIBE") \
MR_MACROS(DETACH_PART, "DETACH PART") \
MR_MACROS(DETACH_PARTITION, "DETACH PARTITION") \
MR_MACROS(DETACH, "DETACH") \
MR_MACROS(DICTIONARIES, "DICTIONARIES") \
MR_MACROS(DICTIONARY, "DICTIONARY") \
MR_MACROS(DISK, "DISK") \
MR_MACROS(DISTINCT_ON, "DISTINCT ON") \
MR_MACROS(DISTINCT, "DISTINCT") \
MR_MACROS(DIV, "DIV") \
MR_MACROS(DROP_COLUMN, "DROP COLUMN") \
MR_MACROS(DROP_CONSTRAINT, "DROP CONSTRAINT") \
MR_MACROS(DROP_DEFAULT, "DROP DEFAULT") \
MR_MACROS(DROP_DETACHED_PART, "DROP DETACHED PART") \
MR_MACROS(DROP_DETACHED_PARTITION, "DROP DETACHED PARTITION") \
MR_MACROS(DROP_INDEX, "DROP INDEX") \
MR_MACROS(DROP_PART, "DROP PART") \
MR_MACROS(DROP_PARTITION, "DROP PARTITION") \
MR_MACROS(DROP_PROJECTION, "DROP PROJECTION") \
MR_MACROS(DROP_STATISTIC, "DROP STATISTIC") \
MR_MACROS(DROP_TABLE, "DROP TABLE") \
MR_MACROS(DROP_TEMPORARY_TABLE, "DROP TEMPORARY TABLE") \
MR_MACROS(DROP, "DROP") \
MR_MACROS(ELSE, "ELSE") \
MR_MACROS(EMPTY_AS, "EMPTY AS") \
MR_MACROS(EMPTY, "EMPTY") \
MR_MACROS(ENABLED_ROLES, "ENABLED ROLES") \
MR_MACROS(END, "END") \
MR_MACROS(ENFORCED, "ENFORCED") \
MR_MACROS(ENGINE, "ENGINE") \
MR_MACROS(EPHEMERAL_SEQUENTIAL, "EPHEMERAL SEQUENTIAL") \
MR_MACROS(EPHEMERAL, "EPHEMERAL") \
MR_MACROS(ESTIMATE, "ESTIMATE") \
MR_MACROS(EVENT, "EVENT") \
MR_MACROS(EVENTS, "EVENTS") \
MR_MACROS(EVERY, "EVERY") \
MR_MACROS(EXCEPT_DATABASE, "EXCEPT DATABASE") \
MR_MACROS(EXCEPT_DATABASES, "EXCEPT DATABASES") \
MR_MACROS(EXCEPT_TABLE, "EXCEPT TABLE") \
MR_MACROS(EXCEPT_TABLES, "EXCEPT TABLES") \
MR_MACROS(EXCEPT, "EXCEPT") \
MR_MACROS(EXCHANGE_DICTIONARIES, "EXCHANGE DICTIONARIES") \
MR_MACROS(EXCHANGE_TABLES, "EXCHANGE TABLES") \
MR_MACROS(EXISTS, "EXISTS") \
MR_MACROS(EXPLAIN, "EXPLAIN") \
MR_MACROS(EXPRESSION, "EXPRESSION") \
MR_MACROS(EXTENDED, "EXTENDED") \
MR_MACROS(EXTERNAL_DDL_FROM, "EXTERNAL DDL FROM") \
MR_MACROS(FALSE_KEYWORD, "FALSE") /*The name differs from the value*/ \
MR_MACROS(FETCH_PART, "FETCH PART") \
MR_MACROS(FETCH_PARTITION, "FETCH PARTITION") \
MR_MACROS(FETCH, "FETCH") \
MR_MACROS(FIELDS, "FIELDS") \
MR_MACROS(FILE, "FILE") \
MR_MACROS(FILESYSTEM_CACHE, "FILESYSTEM CACHE") \
MR_MACROS(FILESYSTEM_CACHES, "FILESYSTEM CACHES") \
MR_MACROS(FILTER, "FILTER") \
MR_MACROS(FINAL, "FINAL") \
MR_MACROS(FIRST, "FIRST") \
MR_MACROS(FOLLOWING, "FOLLOWING") \
MR_MACROS(FOR, "FOR") \
MR_MACROS(FOREIGN_KEY, "FOREIGN KEY") \
MR_MACROS(FOREIGN, "FOREIGN") \
MR_MACROS(FORGET_PARTITION, "FORGET PARTITION") \
MR_MACROS(FORMAT, "FORMAT") \
MR_MACROS(FREEZE, "FREEZE") \
MR_MACROS(FROM_INFILE, "FROM INFILE") \
MR_MACROS(FROM_SHARD, "FROM SHARD") \
MR_MACROS(FROM, "FROM") \
MR_MACROS(FULL, "FULL") \
MR_MACROS(FULLTEXT, "FULLTEXT") \
MR_MACROS(FUNCTION, "FUNCTION") \
MR_MACROS(GLOBAL_IN, "GLOBAL IN") \
MR_MACROS(GLOBAL_NOT_IN, "GLOBAL NOT IN") \
MR_MACROS(GLOBAL, "GLOBAL") \
MR_MACROS(GRANT_OPTION_FOR, "GRANT OPTION FOR") \
MR_MACROS(GRANT, "GRANT") \
MR_MACROS(GRANTEES, "GRANTEES") \
MR_MACROS(GRANULARITY, "GRANULARITY") \
MR_MACROS(GROUP_BY, "GROUP BY") \
MR_MACROS(GROUPING_SETS, "GROUPING SETS") \
MR_MACROS(GROUPS, "GROUPS") \
MR_MACROS(H, "H") \
MR_MACROS(HASH, "HASH") \
MR_MACROS(HAVING, "HAVING") \
MR_MACROS(HDFS, "HDFS") \
MR_MACROS(HH, "HH") \
MR_MACROS(HIERARCHICAL, "HIERARCHICAL") \
MR_MACROS(HOST, "HOST") \
MR_MACROS(HOUR, "HOUR") \
MR_MACROS(HOURS, "HOURS") \
MR_MACROS(HTTP, "HTTP") \
MR_MACROS(ID, "ID") \
MR_MACROS(IDENTIFIED, "IDENTIFIED") \
MR_MACROS(IF_EMPTY, "IF EMPTY") \
MR_MACROS(IF_EXISTS, "IF EXISTS") \
MR_MACROS(IF_NOT_EXISTS, "IF NOT EXISTS") \
MR_MACROS(IGNORE_NULLS, "IGNORE NULLS") \
MR_MACROS(ILIKE, "ILIKE") \
MR_MACROS(IN_PARTITION, "IN PARTITION") \
MR_MACROS(IN, "IN") \
MR_MACROS(INDEX, "INDEX") \
MR_MACROS(INDEXES, "INDEXES") \
MR_MACROS(INDICES, "INDICES") \
MR_MACROS(INHERIT, "INHERIT") \
MR_MACROS(INJECTIVE, "INJECTIVE") \
MR_MACROS(INNER, "INNER") \
MR_MACROS(INSERT_INTO, "INSERT INTO") \
MR_MACROS(INTERPOLATE, "INTERPOLATE") \
MR_MACROS(INTERSECT, "INTERSECT") \
MR_MACROS(INTERVAL, "INTERVAL") \
MR_MACROS(INTO_OUTFILE, "INTO OUTFILE") \
MR_MACROS(INVISIBLE, "INVISIBLE") \
MR_MACROS(INVOKER, "INVOKER") \
MR_MACROS(IP, "IP") \
MR_MACROS(IS_NOT_DISTINCT_FROM, "IS NOT DISTINCT FROM") \
MR_MACROS(IS_NOT_NULL, "IS NOT NULL") \
MR_MACROS(IS_NULL, "IS NULL") \
MR_MACROS(JOIN, "JOIN") \
MR_MACROS(KERBEROS, "KERBEROS") \
MR_MACROS(KEY_BY, "KEY BY") \
MR_MACROS(KEY, "KEY") \
MR_MACROS(KEYED_BY, "KEYED BY") \
MR_MACROS(KEYS, "KEYS") \
MR_MACROS(KILL, "KILL") \
MR_MACROS(KIND, "KIND") \
MR_MACROS(LARGE_OBJECT, "LARGE OBJECT") \
MR_MACROS(LAST, "LAST") \
MR_MACROS(LAYOUT, "LAYOUT") \
MR_MACROS(LDAP, "LDAP") \
MR_MACROS(LEADING, "LEADING") \
MR_MACROS(LEFT_ARRAY_JOIN, "LEFT ARRAY JOIN") \
MR_MACROS(LEFT, "LEFT") \
MR_MACROS(LESS_THAN, "LESS THAN") \
MR_MACROS(LEVEL, "LEVEL") \
MR_MACROS(LIFETIME, "LIFETIME") \
MR_MACROS(LIGHTWEIGHT, "LIGHTWEIGHT") \
MR_MACROS(LIKE, "LIKE") \
MR_MACROS(LIMIT, "LIMIT") \
MR_MACROS(LINEAR, "LINEAR") \
MR_MACROS(LIST, "LIST") \
MR_MACROS(LIVE, "LIVE") \
MR_MACROS(LOCAL, "LOCAL") \
MR_MACROS(M, "M") \
MR_MACROS(MATCH, "MATCH") \
MR_MACROS(MATERIALIZE_COLUMN, "MATERIALIZE COLUMN") \
MR_MACROS(MATERIALIZE_INDEX, "MATERIALIZE INDEX") \
MR_MACROS(MATERIALIZE_PROJECTION, "MATERIALIZE PROJECTION") \
MR_MACROS(MATERIALIZE_STATISTIC, "MATERIALIZE STATISTIC") \
MR_MACROS(MATERIALIZE_TTL, "MATERIALIZE TTL") \
MR_MACROS(MATERIALIZE, "MATERIALIZE") \
MR_MACROS(MATERIALIZED, "MATERIALIZED") \
MR_MACROS(MAX, "MAX") \
MR_MACROS(MCS, "MCS") \
MR_MACROS(MEMORY, "MEMORY") \
MR_MACROS(MERGES, "MERGES") \
MR_MACROS(MI, "MI") \
MR_MACROS(MICROSECOND, "MICROSECOND") \
MR_MACROS(MICROSECONDS, "MICROSECONDS") \
MR_MACROS(MILLISECOND, "MILLISECOND") \
MR_MACROS(MILLISECONDS, "MILLISECONDS") \
MR_MACROS(MIN, "MIN") \
MR_MACROS(MINUTE, "MINUTE") \
MR_MACROS(MINUTES, "MINUTES") \
MR_MACROS(MM, "MM") \
MR_MACROS(MOD, "MOD") \
MR_MACROS(MODIFY_COLUMN, "MODIFY COLUMN") \
MR_MACROS(MODIFY_COMMENT, "MODIFY COMMENT") \
MR_MACROS(MODIFY_ORDER_BY, "MODIFY ORDER BY") \
MR_MACROS(MODIFY_QUERY, "MODIFY QUERY") \
MR_MACROS(MODIFY_REFRESH, "MODIFY REFRESH") \
MR_MACROS(MODIFY_SAMPLE_BY, "MODIFY SAMPLE BY") \
MR_MACROS(MODIFY_SETTING, "MODIFY SETTING") \
MR_MACROS(MODIFY_SQL_SECURITY, "MODIFY SQL SECURITY") \
MR_MACROS(MODIFY_TTL, "MODIFY TTL") \
MR_MACROS(MODIFY, "MODIFY") \
MR_MACROS(MONTH, "MONTH") \
MR_MACROS(MONTHS, "MONTHS") \
MR_MACROS(MOVE_PART, "MOVE PART") \
MR_MACROS(MOVE_PARTITION, "MOVE PARTITION") \
MR_MACROS(MOVE, "MOVE") \
MR_MACROS(MS, "MS") \
MR_MACROS(MUTATION, "MUTATION") \
MR_MACROS(N, "N") \
MR_MACROS(NAME, "NAME") \
MR_MACROS(NAMED_COLLECTION, "NAMED COLLECTION") \
MR_MACROS(NANOSECOND, "NANOSECOND") \
MR_MACROS(NANOSECONDS, "NANOSECONDS") \
MR_MACROS(NEXT, "NEXT") \
MR_MACROS(NO_ACTION, "NO ACTION") \
MR_MACROS(NO_DELAY, "NO DELAY") \
MR_MACROS(NO_LIMITS, "NO LIMITS") \
MR_MACROS(NONE, "NONE") \
MR_MACROS(NOT_BETWEEN, "NOT BETWEEN") \
MR_MACROS(NOT_IDENTIFIED, "NOT IDENTIFIED") \
MR_MACROS(NOT_ILIKE, "NOT ILIKE") \
MR_MACROS(NOT_IN, "NOT IN") \
MR_MACROS(NOT_KEYED, "NOT KEYED") \
MR_MACROS(NOT_LIKE, "NOT LIKE") \
MR_MACROS(NOT_OVERRIDABLE, "NOT OVERRIDABLE") \
MR_MACROS(NOT, "NOT") \
MR_MACROS(NS, "NS") \
MR_MACROS(NULL_KEYWORD, "NULL") \
MR_MACROS(NULLS, "NULLS") \
MR_MACROS(OFFSET, "OFFSET") \
MR_MACROS(ON_DELETE, "ON DELETE") \
MR_MACROS(ON_UPDATE, "ON UPDATE") \
MR_MACROS(ON_VOLUME, "ON VOLUME") \
MR_MACROS(ON, "ON") \
MR_MACROS(ONLY, "ONLY") \
MR_MACROS(OPTIMIZE_TABLE, "OPTIMIZE TABLE") \
MR_MACROS(OR_REPLACE, "OR REPLACE") \
MR_MACROS(OR, "OR") \
MR_MACROS(ORDER_BY, "ORDER BY") \
MR_MACROS(OUTER, "OUTER") \
MR_MACROS(OVER, "OVER") \
MR_MACROS(OVERRIDABLE, "OVERRIDABLE") \
MR_MACROS(PART, "PART") \
MR_MACROS(PARTIAL, "PARTIAL") \
MR_MACROS(PARTITION_BY, "PARTITION BY") \
MR_MACROS(PARTITION, "PARTITION") \
MR_MACROS(PARTITIONS, "PARTITIONS") \
MR_MACROS(PASTE, "PASTE") \
MR_MACROS(PERIODIC_REFRESH, "PERIODIC REFRESH") \
MR_MACROS(PERMANENTLY, "PERMANENTLY") \
MR_MACROS(PERMISSIVE, "PERMISSIVE") \
MR_MACROS(PERSISTENT_SEQUENTIAL, "PERSISTENT SEQUENTIAL") \
MR_MACROS(PERSISTENT, "PERSISTENT") \
MR_MACROS(PIPELINE, "PIPELINE") \
MR_MACROS(PLAN, "PLAN") \
MR_MACROS(POPULATE, "POPULATE") \
MR_MACROS(PRECEDING, "PRECEDING") \
MR_MACROS(PRECISION, "PRECISION") \
MR_MACROS(PREWHERE, "PREWHERE") \
MR_MACROS(PRIMARY_KEY, "PRIMARY KEY") \
MR_MACROS(PRIMARY, "PRIMARY") \
MR_MACROS(PROFILE, "PROFILE") \
MR_MACROS(PROJECTION, "PROJECTION") \
MR_MACROS(PROTOBUF, "Protobuf") \
MR_MACROS(PULL, "PULL") \
MR_MACROS(Q, "Q") \
MR_MACROS(QQ, "QQ") \
MR_MACROS(QUARTER, "QUARTER") \
MR_MACROS(QUARTERS, "QUARTERS") \
MR_MACROS(QUERY_TREE, "QUERY TREE") \
MR_MACROS(QUERY, "QUERY") \
MR_MACROS(QUOTA, "QUOTA") \
MR_MACROS(RANDOMIZE_FOR, "RANDOMIZE FOR") \
MR_MACROS(RANDOMIZED, "RANDOMIZED") \
MR_MACROS(RANGE, "RANGE") \
MR_MACROS(READONLY, "READONLY") \
MR_MACROS(REALM, "REALM") \
MR_MACROS(RECOMPRESS, "RECOMPRESS") \
MR_MACROS(REFERENCES, "REFERENCES") \
MR_MACROS(REFRESH, "REFRESH") \
MR_MACROS(REGEXP, "REGEXP") \
MR_MACROS(REMOVE_SAMPLE_BY, "REMOVE SAMPLE BY") \
MR_MACROS(REMOVE_TTL, "REMOVE TTL") \
MR_MACROS(REMOVE, "REMOVE") \
MR_MACROS(RENAME_COLUMN, "RENAME COLUMN") \
MR_MACROS(RENAME_DATABASE, "RENAME DATABASE") \
MR_MACROS(RENAME_DICTIONARY, "RENAME DICTIONARY") \
MR_MACROS(RENAME_TABLE, "RENAME TABLE") \
MR_MACROS(RENAME_TO, "RENAME TO") \
MR_MACROS(RENAME, "RENAME") \
MR_MACROS(REPLACE_PARTITION, "REPLACE PARTITION") \
MR_MACROS(REPLACE, "REPLACE") \
MR_MACROS(RESET_SETTING, "RESET SETTING") \
MR_MACROS(RESPECT_NULLS, "RESPECT NULLS") \
MR_MACROS(RESTORE, "RESTORE") \
MR_MACROS(RESTRICT, "RESTRICT") \
MR_MACROS(RESTRICTIVE, "RESTRICTIVE") \
MR_MACROS(RESUME, "RESUME") \
MR_MACROS(REVOKE, "REVOKE") \
MR_MACROS(RIGHT, "RIGHT") \
MR_MACROS(ROLLBACK, "ROLLBACK") \
MR_MACROS(ROLLUP, "ROLLUP") \
MR_MACROS(ROW, "ROW") \
MR_MACROS(ROWS, "ROWS") \
MR_MACROS(S, "S") \
MR_MACROS(S3, "S3") \
MR_MACROS(SALT, "SALT") \
MR_MACROS(SAMPLE_BY, "SAMPLE BY") \
MR_MACROS(SAMPLE, "SAMPLE") \
MR_MACROS(SCHEME, "SCHEME") \
MR_MACROS(SECOND, "SECOND") \
MR_MACROS(SECONDS, "SECONDS") \
MR_MACROS(SELECT, "SELECT") \
MR_MACROS(SEMI, "SEMI") \
MR_MACROS(SERVER, "SERVER") \
MR_MACROS(SET_DEFAULT_ROLE, "SET DEFAULT ROLE") \
MR_MACROS(SET_DEFAULT, "SET DEFAULT") \
MR_MACROS(SET_FAKE_TIME, "SET FAKE TIME") \
MR_MACROS(SET_NULL, "SET NULL") \
MR_MACROS(SET_ROLE_DEFAULT, "SET ROLE DEFAULT") \
MR_MACROS(SET_ROLE, "SET ROLE") \
MR_MACROS(SET_TRANSACTION_SNAPSHOT, "SET TRANSACTION SNAPSHOT") \
MR_MACROS(SET, "SET") \
MR_MACROS(SETTINGS, "SETTINGS") \
MR_MACROS(SHOW_ACCESS, "SHOW ACCESS") \
MR_MACROS(SHOW_CREATE, "SHOW CREATE") \
MR_MACROS(SHOW_ENGINES, "SHOW ENGINES") \
MR_MACROS(SHOW_FUNCTIONS, "SHOW FUNCTIONS") \
MR_MACROS(SHOW_GRANTS, "SHOW GRANTS") \
MR_MACROS(SHOW_PRIVILEGES, "SHOW PRIVILEGES") \
MR_MACROS(SHOW_PROCESSLIST, "SHOW PROCESSLIST") \
MR_MACROS(SHOW_SETTING, "SHOW SETTING") \
MR_MACROS(SHOW, "SHOW") \
MR_MACROS(SIGNED, "SIGNED") \
MR_MACROS(SIMPLE, "SIMPLE") \
MR_MACROS(SOURCE, "SOURCE") \
MR_MACROS(SPATIAL, "SPATIAL") \
MR_MACROS(SQL_SECURITY, "SQL SECURITY") \
MR_MACROS(SS, "SS") \
MR_MACROS(STATISTIC, "STATISTIC") \
MR_MACROS(STEP, "STEP") \
MR_MACROS(STORAGE, "STORAGE") \
MR_MACROS(STRICT, "STRICT") \
MR_MACROS(SUBPARTITION_BY, "SUBPARTITION BY") \
MR_MACROS(SUBPARTITION, "SUBPARTITION") \
MR_MACROS(SUBPARTITIONS, "SUBPARTITIONS") \
MR_MACROS(SUSPEND, "SUSPEND") \
MR_MACROS(SYNC, "SYNC") \
MR_MACROS(SYNTAX, "SYNTAX") \
MR_MACROS(SYSTEM, "SYSTEM") \
MR_MACROS(TABLE_OVERRIDE, "TABLE OVERRIDE") \
MR_MACROS(TABLE, "TABLE") \
MR_MACROS(TABLES, "TABLES") \
MR_MACROS(TEMPORARY_TABLE, "TEMPORARY TABLE") \
MR_MACROS(TEMPORARY, "TEMPORARY") \
MR_MACROS(TEST, "TEST") \
MR_MACROS(THEN, "THEN") \
MR_MACROS(TIMESTAMP, "TIMESTAMP") \
MR_MACROS(TO_DISK, "TO DISK") \
MR_MACROS(TO_INNER_UUID, "TO INNER UUID") \
MR_MACROS(TO_SHARD, "TO SHARD") \
MR_MACROS(TO_TABLE, "TO TABLE") \
MR_MACROS(TO_VOLUME, "TO VOLUME") \
MR_MACROS(TO, "TO") \
MR_MACROS(TOP, "TOP") \
MR_MACROS(TOTALS, "TOTALS") \
MR_MACROS(TRACKING_ONLY, "TRACKING ONLY") \
MR_MACROS(TRAILING, "TRAILING") \
MR_MACROS(TRANSACTION, "TRANSACTION") \
MR_MACROS(TRIGGER, "TRIGGER") \
MR_MACROS(TRUE_KEYWORD, "TRUE") /*The name differs from the value*/ \
MR_MACROS(TRUNCATE, "TRUNCATE") \
MR_MACROS(TTL, "TTL") \
MR_MACROS(TYPE, "TYPE") \
MR_MACROS(TYPEOF, "TYPEOF") \
MR_MACROS(UNBOUNDED, "UNBOUNDED") \
MR_MACROS(UNDROP, "UNDROP") \
MR_MACROS(UNFREEZE, "UNFREEZE") \
MR_MACROS(UNION, "UNION") \
MR_MACROS(UNIQUE, "UNIQUE") \
MR_MACROS(UNSET_FAKE_TIME, "UNSET FAKE TIME") \
MR_MACROS(UNSIGNED, "UNSIGNED") \
MR_MACROS(UPDATE, "UPDATE") \
MR_MACROS(URL, "URL") \
MR_MACROS(USE, "USE") \
MR_MACROS(USING, "USING") \
MR_MACROS(UUID, "UUID") \
MR_MACROS(VALID_UNTIL, "VALID UNTIL") \
MR_MACROS(VALUES, "VALUES") \
MR_MACROS(VARYING, "VARYING") \
MR_MACROS(VIEW, "VIEW") \
MR_MACROS(VISIBLE, "VISIBLE") \
MR_MACROS(WATCH, "WATCH") \
MR_MACROS(WATERMARK, "WATERMARK") \
MR_MACROS(WEEK, "WEEK") \
MR_MACROS(WEEKS, "WEEKS") \
MR_MACROS(WHEN, "WHEN") \
MR_MACROS(WHERE, "WHERE") \
MR_MACROS(WINDOW, "WINDOW") \
MR_MACROS(WITH_ADMIN_OPTION, "WITH ADMIN OPTION") \
MR_MACROS(WITH_CHECK, "WITH CHECK") \
MR_MACROS(WITH_FILL, "WITH FILL") \
MR_MACROS(WITH_GRANT_OPTION, "WITH GRANT OPTION") \
MR_MACROS(WITH_NAME, "WITH NAME") \
MR_MACROS(WITH_REPLACE_OPTION, "WITH REPLACE OPTION") \
MR_MACROS(WITH_TIES, "WITH TIES") \
MR_MACROS(WITH, "WITH") \
MR_MACROS(WK, "WK") \
MR_MACROS(WRITABLE, "WRITABLE") \
MR_MACROS(WW, "WW") \
MR_MACROS(YEAR, "YEAR") \
MR_MACROS(YEARS, "YEARS") \
MR_MACROS(YY, "YY") \
MR_MACROS(YYYY, "YYYY") \
MR_MACROS(ZKPATH, "ZKPATH") \
/// The list of keywords where underscore is intentional
#define APPLY_FOR_PARSER_KEYWORDS_WITH_UNDERSCORES(MR_MACROS) \
MR_MACROS(ALLOWED_LATENESS, "ALLOWED_LATENESS") \
MR_MACROS(AUTO_INCREMENT, "AUTO_INCREMENT") \
MR_MACROS(BASE_BACKUP, "base_backup") \
MR_MACROS(BCRYPT_HASH, "BCRYPT_HASH") \
MR_MACROS(BCRYPT_PASSWORD, "BCRYPT_PASSWORD") \
MR_MACROS(CHANGEABLE_IN_READONLY, "CHANGEABLE_IN_READONLY") \
MR_MACROS(CLUSTER_HOST_IDS, "cluster_host_ids") \
MR_MACROS(CURRENT_USER, "CURRENT_USER") \
MR_MACROS(DOUBLE_SHA1_HASH, "DOUBLE_SHA1_HASH") \
MR_MACROS(DOUBLE_SHA1_PASSWORD, "DOUBLE_SHA1_PASSWORD") \
MR_MACROS(IS_OBJECT_ID, "IS_OBJECT_ID") \
MR_MACROS(NO_PASSWORD, "NO_PASSWORD") \
MR_MACROS(PART_MOVE_TO_SHARD, "PART_MOVE_TO_SHARD") \
MR_MACROS(PLAINTEXT_PASSWORD, "PLAINTEXT_PASSWORD") \
MR_MACROS(SHA256_HASH, "SHA256_HASH") \
MR_MACROS(SHA256_PASSWORD, "SHA256_PASSWORD") \
MR_MACROS(SQL_TSI_DAY, "SQL_TSI_DAY") \
MR_MACROS(SQL_TSI_HOUR, "SQL_TSI_HOUR") \
MR_MACROS(SQL_TSI_MICROSECOND, "SQL_TSI_MICROSECOND") \
MR_MACROS(SQL_TSI_MILLISECOND, "SQL_TSI_MILLISECOND") \
MR_MACROS(SQL_TSI_MINUTE, "SQL_TSI_MINUTE") \
MR_MACROS(SQL_TSI_MONTH, "SQL_TSI_MONTH") \
MR_MACROS(SQL_TSI_NANOSECOND, "SQL_TSI_NANOSECOND") \
MR_MACROS(SQL_TSI_QUARTER, "SQL_TSI_QUARTER") \
MR_MACROS(SQL_TSI_SECOND, "SQL_TSI_SECOND") \
MR_MACROS(SQL_TSI_WEEK, "SQL_TSI_WEEK") \
MR_MACROS(SQL_TSI_YEAR, "SQL_TSI_YEAR") \
MR_MACROS(SSH_KEY, "SSH_KEY") \
MR_MACROS(SSL_CERTIFICATE, "SSL_CERTIFICATE") \
MR_MACROS(STRICTLY_ASCENDING, "STRICTLY_ASCENDING") \
MR_MACROS(WITH_ITEMINDEX, "with_itemindex") \
enum class Keyword : size_t
{
#define DECLARE_PARSER_KEYWORD_ENUM(identifier, name) \
identifier,
APPLY_FOR_PARSER_KEYWORDS(DECLARE_PARSER_KEYWORD_ENUM)
APPLY_FOR_PARSER_KEYWORDS_WITH_UNDERSCORES(DECLARE_PARSER_KEYWORD_ENUM)
#undef DECLARE_PARSER_KEYWORD_ENUM
};
std::string_view toStringView(Keyword type);
const std::vector<String> & getAllKeyWords();
/** Parse specified keyword such as SELECT or compound keyword such as ORDER BY.
* All case insensitive. Requires word boundary.
* For compound keywords, any whitespace characters and comments could be in the middle.
@ -17,9 +582,20 @@ class ParserKeyword : public IParserBase
private:
std::string_view s;
explicit ParserKeyword(std::string_view s_): s(s_) { assert(!s.empty()); }
public:
//NOLINTNEXTLINE Want to be able to init ParserKeyword("literal")
constexpr ParserKeyword(std::string_view s_): s(s_) { assert(!s.empty()); }
static ParserKeyword createDeprecated(std::string_view s_)
{
return ParserKeyword(s_);
}
static std::shared_ptr<ParserKeyword> createDeprecatedPtr(std::string_view s_)
{
return std::shared_ptr<ParserKeyword>(new ParserKeyword(s_));
}
explicit ParserKeyword(Keyword keyword);
constexpr const char * getName() const override { return s.data(); }

View File

@ -8,6 +8,7 @@
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/BinStringDecodeHelper.h>
#include "Parsers/CommonParsers.h"
#include <Parsers/DumpASTNode.h>
#include <Parsers/ASTAsterisk.h>
@ -290,7 +291,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
params.push_back(child->as<ASTIdentifier>()->getParam());
}
ParserKeyword s_uuid("UUID");
ParserKeyword s_uuid(Keyword::UUID);
UUID uuid = UUIDHelpers::Nil;
if (table_name_with_optional_uuid)
@ -337,7 +338,7 @@ bool ParserFilterClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
return false;
}
ParserKeyword parser_where("WHERE");
ParserKeyword parser_where(Keyword::WHERE);
if (!parser_where.ignore(pos, expected))
{
return false;
@ -391,9 +392,9 @@ bool ParserWindowReference::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & pos,
Expected & expected)
{
ParserKeyword keyword_rows("ROWS");
ParserKeyword keyword_groups("GROUPS");
ParserKeyword keyword_range("RANGE");
ParserKeyword keyword_rows(Keyword::ROWS);
ParserKeyword keyword_groups(Keyword::GROUPS);
ParserKeyword keyword_range(Keyword::RANGE);
node->frame_is_default = false;
if (keyword_rows.ignore(pos, expected))
@ -415,12 +416,12 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
return true;
}
ParserKeyword keyword_between("BETWEEN");
ParserKeyword keyword_unbounded("UNBOUNDED");
ParserKeyword keyword_preceding("PRECEDING");
ParserKeyword keyword_following("FOLLOWING");
ParserKeyword keyword_and("AND");
ParserKeyword keyword_current_row("CURRENT ROW");
ParserKeyword keyword_between(Keyword::BETWEEN);
ParserKeyword keyword_unbounded(Keyword::UNBOUNDED);
ParserKeyword keyword_preceding(Keyword::PRECEDING);
ParserKeyword keyword_following(Keyword::FOLLOWING);
ParserKeyword keyword_and(Keyword::AND);
ParserKeyword keyword_current_row(Keyword::CURRENT_ROW);
// There are two variants of grammar for the frame:
// 1) ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
@ -524,10 +525,10 @@ static bool tryParseFrameDefinition(ASTWindowDefinition * node, IParser::Pos & p
static bool parseWindowDefinitionParts(IParser::Pos & pos,
ASTWindowDefinition & node, Expected & expected)
{
ParserKeyword keyword_partition_by("PARTITION BY");
ParserKeyword keyword_partition_by(Keyword::PARTITION_BY);
ParserNotEmptyExpressionList columns_partition_by(
false /* we don't allow declaring aliases here*/);
ParserKeyword keyword_order_by("ORDER BY");
ParserKeyword keyword_order_by(Keyword::ORDER_BY);
ParserOrderByExpressionList columns_order_by;
if (keyword_partition_by.ignore(pos, expected))
@ -626,7 +627,7 @@ bool ParserWindowList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
elem->name = getIdentifierName(window_name_identifier);
ParserKeyword keyword_as("AS");
ParserKeyword keyword_as(Keyword::AS);
if (!keyword_as.ignore(pos, expected))
{
return false;
@ -854,7 +855,7 @@ bool ParserCastOperator::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword nested_parser("NULL");
ParserKeyword nested_parser(Keyword::NULL_KEYWORD);
if (nested_parser.parse(pos, node, expected))
{
node = std::make_shared<ASTLiteral>(Null());
@ -867,12 +868,12 @@ bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserBool::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (ParserKeyword("true").parse(pos, node, expected))
if (ParserKeyword(Keyword::TRUE_KEYWORD).parse(pos, node, expected))
{
node = std::make_shared<ASTLiteral>(true);
return true;
}
else if (ParserKeyword("false").parse(pos, node, expected))
else if (ParserKeyword(Keyword::FALSE_KEYWORD).parse(pos, node, expected))
{
node = std::make_shared<ASTLiteral>(false);
return true;
@ -1488,7 +1489,7 @@ const char * ParserAlias::restricted_keywords[] =
bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_as("AS");
ParserKeyword s_as(Keyword::AS);
ParserIdentifier id_p;
bool has_as_word = s_as.ignore(pos, expected);
@ -1519,11 +1520,11 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword apply("APPLY");
ParserKeyword except("EXCEPT");
ParserKeyword replace("REPLACE");
ParserKeyword as("AS");
ParserKeyword strict("STRICT");
ParserKeyword apply(Keyword::APPLY);
ParserKeyword except(Keyword::EXCEPT);
ParserKeyword replace(Keyword::REPLACE);
ParserKeyword as(Keyword::AS);
ParserKeyword strict(Keyword::STRICT);
if (allowed_transformers.isSet(ColumnTransformer::APPLY) && apply.ignore(pos, expected))
{
@ -1847,7 +1848,7 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword columns("COLUMNS");
ParserKeyword columns(Keyword::COLUMNS);
if (!columns.ignore(pos, expected))
return false;
@ -2052,18 +2053,18 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserExpressionWithOptionalAlias elem_p(false);
ParserKeyword ascending("ASCENDING");
ParserKeyword descending("DESCENDING");
ParserKeyword asc("ASC");
ParserKeyword desc("DESC");
ParserKeyword nulls("NULLS");
ParserKeyword first("FIRST");
ParserKeyword last("LAST");
ParserKeyword collate("COLLATE");
ParserKeyword with_fill("WITH FILL");
ParserKeyword from("FROM");
ParserKeyword to("TO");
ParserKeyword step("STEP");
ParserKeyword ascending(Keyword::ASCENDING);
ParserKeyword descending(Keyword::DESCENDING);
ParserKeyword asc(Keyword::ASC);
ParserKeyword desc(Keyword::DESC);
ParserKeyword nulls(Keyword::NULLS);
ParserKeyword first(Keyword::FIRST);
ParserKeyword last(Keyword::LAST);
ParserKeyword collate(Keyword::COLLATE);
ParserKeyword with_fill(Keyword::WITH_FILL);
ParserKeyword from(Keyword::FROM);
ParserKeyword to(Keyword::TO);
ParserKeyword step(Keyword::STEP);
ParserStringLiteral collate_locale_parser;
ParserExpressionWithOptionalAlias exp_parser(false);
@ -2139,7 +2140,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
bool ParserInterpolateElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword as("AS");
ParserKeyword as(Keyword::AS);
ParserExpression element_p;
ParserIdentifier ident_p;
@ -2210,18 +2211,18 @@ bool ParserFunctionWithKeyValueArguments::parseImpl(Pos & pos, ASTPtr & node, Ex
bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_to_disk("TO DISK");
ParserKeyword s_to_volume("TO VOLUME");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_delete("DELETE");
ParserKeyword s_where("WHERE");
ParserKeyword s_group_by("GROUP BY");
ParserKeyword s_set("SET");
ParserKeyword s_recompress("RECOMPRESS");
ParserKeyword s_codec("CODEC");
ParserKeyword s_materialize("MATERIALIZE");
ParserKeyword s_remove("REMOVE");
ParserKeyword s_modify("MODIFY");
ParserKeyword s_to_disk(Keyword::TO_DISK);
ParserKeyword s_to_volume(Keyword::TO_VOLUME);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_delete(Keyword::DELETE);
ParserKeyword s_where(Keyword::WHERE);
ParserKeyword s_group_by(Keyword::GROUP_BY);
ParserKeyword s_set(Keyword::SET);
ParserKeyword s_recompress(Keyword::RECOMPRESS);
ParserKeyword s_codec(Keyword::CODEC);
ParserKeyword s_materialize(Keyword::MATERIALIZE);
ParserKeyword s_remove(Keyword::REMOVE);
ParserKeyword s_modify(Keyword::MODIFY);
ParserIdentifier parser_identifier;
ParserStringLiteral parser_string_literal;

View File

@ -69,11 +69,11 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserUnionQueryElement elem_parser;
ParserKeyword s_union_parser("UNION");
ParserKeyword s_all_parser("ALL");
ParserKeyword s_distinct_parser("DISTINCT");
ParserKeyword s_except_parser("EXCEPT");
ParserKeyword s_intersect_parser("INTERSECT");
ParserKeyword s_union_parser(Keyword::UNION);
ParserKeyword s_all_parser(Keyword::ALL);
ParserKeyword s_distinct_parser(Keyword::DISTINCT);
ParserKeyword s_except_parser(Keyword::EXCEPT);
ParserKeyword s_intersect_parser(Keyword::INTERSECT);
ASTs elements;
auto parse_element = [&]
@ -135,7 +135,7 @@ static bool parseOperator(IParser::Pos & pos, std::string_view op, Expected & ex
{
if (!op.empty() && isWordCharASCII(op.front()))
{
return ParserKeyword(op).ignore(pos, expected);
return ParserKeyword::createDeprecated(op).ignore(pos, expected);
}
else if (op.length() == pos->size() && 0 == memcmp(op.data(), pos->begin, pos->size()))
{
@ -806,7 +806,6 @@ struct ParserExpressionImpl
{
static const std::vector<std::pair<std::string_view, Operator>> operators_table;
static const std::vector<std::pair<std::string_view, Operator>> unary_operators_table;
static const std::array<std::string_view, 1> overlapping_operators_to_skip;
static const Operator finish_between_operator;
@ -819,8 +818,8 @@ struct ParserExpressionImpl
ParserSubstitution substitution_parser;
ParserMySQLGlobalVariable mysql_global_variable_parser;
ParserKeyword any_parser{"ANY"};
ParserKeyword all_parser{"ALL"};
ParserKeyword any_parser{Keyword::ANY};
ParserKeyword all_parser{Keyword::ALL};
// Recursion
ParserQualifiedAsterisk qualified_asterisk_parser;
@ -885,7 +884,7 @@ public:
if (test_pos.isValid() && test_pos->type != TokenType::Semicolon)
{
/// If we can't parse FROM then return
if (!ParserKeyword("FROM").ignore(test_pos, test_expected))
if (!ParserKeyword(Keyword::FROM).ignore(test_pos, test_expected))
return true;
// If there is a comma after 'from' then the first one was a name of a column
@ -893,7 +892,7 @@ public:
return true;
/// If we parse a second FROM then the first one was a name of a column
if (ParserKeyword("FROM").ignore(test_pos, test_expected))
if (ParserKeyword(Keyword::FROM).ignore(test_pos, test_expected))
return true;
/// If we parse an explicit alias to FROM, then it was a name of a column
@ -981,8 +980,8 @@ public:
auto pos_after_bracket = pos;
auto old_expected = expected;
ParserKeyword all("ALL");
ParserKeyword distinct("DISTINCT");
ParserKeyword all(Keyword::ALL);
ParserKeyword distinct(Keyword::DISTINCT);
if (all.ignore(pos, expected))
has_all = true;
@ -1065,8 +1064,8 @@ public:
auto pos_after_bracket = pos;
auto old_expected = expected;
ParserKeyword all("ALL");
ParserKeyword distinct("DISTINCT");
ParserKeyword all(Keyword::ALL);
ParserKeyword distinct(Keyword::DISTINCT);
if (all.ignore(pos, expected))
has_all = true;
@ -1112,10 +1111,10 @@ public:
function_node->children.push_back(function_node->parameters);
}
ParserKeyword filter("FILTER");
ParserKeyword over("OVER");
ParserKeyword respect_nulls("RESPECT NULLS");
ParserKeyword ignore_nulls("IGNORE NULLS");
ParserKeyword filter(Keyword::FILTER);
ParserKeyword over(Keyword::OVER);
ParserKeyword respect_nulls(Keyword::RESPECT_NULLS);
ParserKeyword ignore_nulls(Keyword::IGNORE_NULLS);
if (filter.ignore(pos, expected))
{
@ -1267,7 +1266,7 @@ public:
/// 0. Parse all the cases (-> 1)
/// 1. Parse closing token (finished)
ParserKeyword as_keyword_parser("AS");
ParserKeyword as_keyword_parser(Keyword::AS);
ASTPtr alias;
/// expr AS type
@ -1348,7 +1347,7 @@ public:
if (elements.size() != 2)
return false;
elements = {makeASTFunction("CAST", elements[0], elements[1])};
elements = {makeASTFunction(toString(toStringView(Keyword::CAST)), elements[0], elements[1])};
finished = true;
return true;
}
@ -1374,7 +1373,7 @@ public:
if (state == 0)
{
IParser::Pos begin = pos;
ParserKeyword s_from("FROM");
ParserKeyword s_from(Keyword::FROM);
if (parseIntervalKind(pos, expected, interval_kind) && s_from.ignore(pos, expected))
{
@ -1447,7 +1446,7 @@ public:
if (state == 0)
{
if (ParserToken(TokenType::Comma).ignore(pos, expected) ||
ParserKeyword("FROM").ignore(pos, expected))
ParserKeyword(Keyword::FROM).ignore(pos, expected))
{
action = Action::OPERAND;
@ -1461,7 +1460,7 @@ public:
if (state == 1)
{
if (ParserToken(TokenType::Comma).ignore(pos, expected) ||
ParserKeyword("FOR").ignore(pos, expected))
ParserKeyword(Keyword::FOR).ignore(pos, expected))
{
action = Action::OPERAND;
@ -1518,7 +1517,7 @@ public:
state = 1;
}
if (ParserKeyword("IN").ignore(pos, expected))
if (ParserKeyword(Keyword::IN).ignore(pos, expected))
{
action = Action::OPERAND;
@ -1610,18 +1609,18 @@ public:
{
if (!trim_left && !trim_right)
{
if (ParserKeyword("BOTH").ignore(pos, expected))
if (ParserKeyword(Keyword::BOTH).ignore(pos, expected))
{
trim_left = true;
trim_right = true;
char_override = true;
}
else if (ParserKeyword("LEADING").ignore(pos, expected))
else if (ParserKeyword(Keyword::LEADING).ignore(pos, expected))
{
trim_left = true;
char_override = true;
}
else if (ParserKeyword("TRAILING").ignore(pos, expected))
else if (ParserKeyword(Keyword::TRAILING).ignore(pos, expected))
{
trim_right = true;
char_override = true;
@ -1645,7 +1644,7 @@ public:
if (state == 1)
{
if (ParserKeyword("FROM").ignore(pos, expected))
if (ParserKeyword(Keyword::FROM).ignore(pos, expected))
{
action = Action::OPERAND;
@ -2002,7 +2001,7 @@ public:
if (state == 0)
{
auto old_pos = pos;
has_case_expr = !ParserKeyword("WHEN").ignore(pos, expected);
has_case_expr = !ParserKeyword(Keyword::WHEN).ignore(pos, expected);
pos = old_pos;
state = 1;
@ -2010,7 +2009,7 @@ public:
if (state == 1)
{
if (ParserKeyword("WHEN").ignore(pos, expected))
if (ParserKeyword(Keyword::WHEN).ignore(pos, expected))
{
if ((has_case_expr || !elements.empty()) && !mergeElement())
return false;
@ -2018,7 +2017,7 @@ public:
action = Action::OPERAND;
state = 2;
}
else if (ParserKeyword("ELSE").ignore(pos, expected))
else if (ParserKeyword(Keyword::ELSE).ignore(pos, expected))
{
if (!mergeElement())
return false;
@ -2026,7 +2025,7 @@ public:
action = Action::OPERAND;
state = 3;
}
else if (ParserKeyword("END").ignore(pos, expected))
else if (ParserKeyword(Keyword::END).ignore(pos, expected))
{
if (!mergeElement())
return false;
@ -2045,7 +2044,7 @@ public:
if (state == 2)
{
if (ParserKeyword("THEN").ignore(pos, expected))
if (ParserKeyword(Keyword::THEN).ignore(pos, expected))
{
if (!mergeElement())
return false;
@ -2057,7 +2056,7 @@ public:
if (state == 3)
{
if (ParserKeyword("END").ignore(pos, expected))
if (ParserKeyword(Keyword::END).ignore(pos, expected))
{
if (!mergeElement())
return false;
@ -2119,7 +2118,7 @@ public:
return true;
}
if (!ParserKeyword{"ELSE"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::ELSE}.ignore(pos, expected))
return false;
state = 1;
@ -2303,7 +2302,7 @@ bool ParseDateOperatorExpression(IParser::Pos & pos, ASTPtr & node, Expected & e
auto begin = pos;
/// If no DATE keyword, go to the nested parser.
if (!ParserKeyword("DATE").ignore(pos, expected))
if (!ParserKeyword(Keyword::DATE).ignore(pos, expected))
return false;
ASTPtr expr;
@ -2322,7 +2321,7 @@ bool ParseTimestampOperatorExpression(IParser::Pos & pos, ASTPtr & node, Expecte
auto begin = pos;
/// If no TIMESTAMP keyword, go to the nested parser.
if (!ParserKeyword("TIMESTAMP").ignore(pos, expected))
if (!ParserKeyword(Keyword::TIMESTAMP).ignore(pos, expected))
return false;
ASTPtr expr;
@ -2378,13 +2377,13 @@ const std::vector<std::pair<std::string_view, Operator>> ParserExpressionImpl::o
{"->", Operator("lambda", 1, 2, OperatorType::Lambda)},
{"?", Operator("", 2, 0, OperatorType::StartIf)},
{":", Operator("if", 3, 3, OperatorType::FinishIf)},
{"OR", Operator("or", 3, 2, OperatorType::Mergeable)},
{"AND", Operator("and", 4, 2, OperatorType::Mergeable)},
{"IS NOT DISTINCT FROM", Operator("isNotDistinctFrom", 6, 2)},
{"IS NULL", Operator("isNull", 6, 1, OperatorType::IsNull)},
{"IS NOT NULL", Operator("isNotNull", 6, 1, OperatorType::IsNull)},
{"BETWEEN", Operator("", 7, 0, OperatorType::StartBetween)},
{"NOT BETWEEN", Operator("", 7, 0, OperatorType::StartNotBetween)},
{toStringView(Keyword::OR), Operator("or", 3, 2, OperatorType::Mergeable)},
{toStringView(Keyword::AND), Operator("and", 4, 2, OperatorType::Mergeable)},
{toStringView(Keyword::IS_NOT_DISTINCT_FROM), Operator("isNotDistinctFrom", 6, 2)},
{toStringView(Keyword::IS_NULL), Operator("isNull", 6, 1, OperatorType::IsNull)},
{toStringView(Keyword::IS_NOT_NULL), Operator("isNotNull", 6, 1, OperatorType::IsNull)},
{toStringView(Keyword::BETWEEN), Operator("", 7, 0, OperatorType::StartBetween)},
{toStringView(Keyword::NOT_BETWEEN), Operator("", 7, 0, OperatorType::StartNotBetween)},
{"==", Operator("equals", 9, 2, OperatorType::Comparison)},
{"!=", Operator("notEquals", 9, 2, OperatorType::Comparison)},
{"<=>", Operator("isNotDistinctFrom", 9, 2, OperatorType::Comparison)},
@ -2394,15 +2393,15 @@ const std::vector<std::pair<std::string_view, Operator>> ParserExpressionImpl::o
{"<", Operator("less", 9, 2, OperatorType::Comparison)},
{">", Operator("greater", 9, 2, OperatorType::Comparison)},
{"=", Operator("equals", 9, 2, OperatorType::Comparison)},
{"LIKE", Operator("like", 9, 2)},
{"ILIKE", Operator("ilike", 9, 2)},
{"NOT LIKE", Operator("notLike", 9, 2)},
{"NOT ILIKE", Operator("notILike", 9, 2)},
{"REGEXP", Operator("match", 9, 2)},
{"IN", Operator("in", 9, 2)},
{"NOT IN", Operator("notIn", 9, 2)},
{"GLOBAL IN", Operator("globalIn", 9, 2)},
{"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)},
{toStringView(Keyword::LIKE), Operator("like", 9, 2)},
{toStringView(Keyword::ILIKE), Operator("ilike", 9, 2)},
{toStringView(Keyword::NOT_LIKE), Operator("notLike", 9, 2)},
{toStringView(Keyword::NOT_ILIKE), Operator("notILike", 9, 2)},
{toStringView(Keyword::REGEXP), Operator("match", 9, 2)},
{toStringView(Keyword::IN), Operator("in", 9, 2)},
{toStringView(Keyword::NOT_IN), Operator("notIn", 9, 2)},
{toStringView(Keyword::GLOBAL_IN), Operator("globalIn", 9, 2)},
{toStringView(Keyword::GLOBAL_NOT_IN), Operator("globalNotIn", 9, 2)},
{"||", Operator("concat", 10, 2, OperatorType::Mergeable)},
{"+", Operator("plus", 11, 2)},
{"-", Operator("minus", 11, 2)},
@ -2410,26 +2409,22 @@ const std::vector<std::pair<std::string_view, Operator>> ParserExpressionImpl::o
{"*", Operator("multiply", 12, 2)},
{"/", Operator("divide", 12, 2)},
{"%", Operator("modulo", 12, 2)},
{"MOD", Operator("modulo", 12, 2)},
{"DIV", Operator("intDiv", 12, 2)},
{toStringView(Keyword::MOD), Operator("modulo", 12, 2)},
{toStringView(Keyword::DIV), Operator("intDiv", 12, 2)},
{".", Operator("tupleElement", 14, 2, OperatorType::TupleElement)},
{"[", Operator("arrayElement", 14, 2, OperatorType::ArrayElement)},
{"::", Operator("CAST", 14, 2, OperatorType::Cast)},
{"::", Operator(toString(toStringView(Keyword::CAST)), 14, 2, OperatorType::Cast)},
};
const std::vector<std::pair<std::string_view, Operator>> ParserExpressionImpl::unary_operators_table
{
{"NOT", Operator("not", 5, 1, OperatorType::Not)},
{"-", Operator("negate", 13, 1)},
{"", Operator("negate", 13, 1)}
{toStringView(Keyword::NOT), Operator("not", 5, 1, OperatorType::Not)},
{"-", Operator("negate", 13, 1)},
{"", Operator("negate", 13, 1)},
};
const Operator ParserExpressionImpl::finish_between_operator("", 8, 0, OperatorType::FinishBetween);
const std::array<std::string_view, 1> ParserExpressionImpl::overlapping_operators_to_skip
{
"IN PARTITION"
};
bool ParserExpressionImpl::parse(std::unique_ptr<Layer> start, IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
@ -2516,7 +2511,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
if (layers.back()->isCurrentElementEmpty())
{
auto old_pos = pos;
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_settings(Keyword::SETTINGS);
if (s_settings.ignore(pos, expected))
{
ParserSetQuery parser_settings(true);
@ -2608,13 +2603,13 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
auto current_checkpoint = layers.back()->current_checkpoint;
layers.back()->current_checkpoint = Checkpoint::None;
if (current_checkpoint != Checkpoint::Interval && parseOperator(pos, "INTERVAL", expected))
if (current_checkpoint != Checkpoint::Interval && parseOperator(pos, toStringView(Keyword::INTERVAL), expected))
{
layers.back()->saved_checkpoint = {old_pos, Checkpoint::Interval};
layers.push_back(std::make_unique<IntervalLayer>());
return Action::OPERAND;
}
else if (current_checkpoint != Checkpoint::Case && parseOperator(pos, "CASE", expected))
else if (current_checkpoint != Checkpoint::Case && parseOperator(pos, toStringView(Keyword::CASE), expected))
{
layers.back()->saved_checkpoint = {old_pos, Checkpoint::Case};
layers.push_back(std::make_unique<CaseLayer>());
@ -2689,9 +2684,8 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
///
/// 'IN PARTITION' here is not an 'IN' operator, so we should stop parsing immediately
Expected stub;
for (const auto & it : overlapping_operators_to_skip)
if (ParserKeyword{it}.checkWithoutMoving(pos, stub))
return Action::NONE;
if (ParserKeyword(Keyword::IN_PARTITION).checkWithoutMoving(pos, stub))
return Action::NONE;
/// Try to find operators from 'operators_table'
auto cur_op = operators_table.begin();

View File

@ -102,7 +102,7 @@ bool CountOf::convertImpl(String & out, IParser::Pos & pos)
bool Extract::convertImpl(String & out, IParser::Pos & pos)
{
ParserKeyword s_kql("typeof");
ParserKeyword s_kql(Keyword::TYPEOF);
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
Expected expected;
@ -243,7 +243,7 @@ bool ExtractAll::convertImpl(String & out, IParser::Pos & pos)
bool ExtractJSON::convertImpl(String & out, IParser::Pos & pos)
{
String datatype = "String";
ParserKeyword s_kql("typeof");
ParserKeyword s_kql(Keyword::TYPEOF);
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
Expected expected;

View File

@ -39,8 +39,8 @@ bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_ex
ParserToken close_bracket(TokenType::ClosingRoundBracket);
ParserToken comma(TokenType::Comma);
ParserKeyword s_to("to");
ParserKeyword s_type("typeof");
ParserKeyword s_to(Keyword::TO);
ParserKeyword s_type(Keyword::TYPEOF);
uint16_t bracket_count = 0;
Pos expr_begin_pos = pos;
Pos expr_end_pos = pos;
@ -144,10 +144,10 @@ bool ParserKQLMVExpand::parseColumnArrayExprs(ColumnArrayExprs & column_array_ex
bool ParserKQLMVExpand::parserMVExpand(KQLMVExpand & kql_mv_expand, Pos & pos, Expected & expected)
{
ParserKeyword s_bagexpansion("bagexpansion");
ParserKeyword s_kind("kind");
ParserKeyword s_with_itemindex("with_itemindex");
ParserKeyword s_limit("limit");
ParserKeyword s_bagexpansion(Keyword::BAGEXPANSION);
ParserKeyword s_kind(Keyword::KIND);
ParserKeyword s_with_itemindex(Keyword::WITH_ITEMINDEX);
ParserKeyword s_limit(Keyword::LIMIT);
ParserToken equals(TokenType::Equals);
ParserToken comma(TokenType::Comma);

View File

@ -34,7 +34,7 @@ bool ParserKQLMakeSeries ::parseAggregationColumns(AggregationColumns & aggregat
"variance"});
Expected expected;
ParserKeyword s_default("default");
ParserKeyword s_default(Keyword::DEFAULT);
ParserToken equals(TokenType::Equals);
ParserToken open_bracket(TokenType::OpeningRoundBracket);
ParserToken close_bracket(TokenType::ClosingRoundBracket);
@ -364,8 +364,8 @@ bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr &
bool ParserKQLMakeSeries ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
ParserKeyword s_on("on");
ParserKeyword s_by("by");
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_by(Keyword::BY);
ParserToken equals(TokenType::Equals);
ParserToken comma(TokenType::Comma);

View File

@ -399,7 +399,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!isValidKQLPos(pos))
return false;
ParserKeyword s_by("by");
ParserKeyword s_by(Keyword::BY);
if (s_by.ignore(pos, expected))
{
kql_operator = "order by";

View File

@ -55,7 +55,7 @@ static inline bool parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected &
else
{
alter_command->type = ASTAlterCommand::ADD_COLUMN;
ParserKeyword("COLUMN").ignore(pos, expected);
ParserKeyword(Keyword::COLUMN).ignore(pos, expected);
if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
{
@ -76,9 +76,9 @@ static inline bool parseAddCommand(IParser::Pos & pos, ASTPtr & node, Expected &
additional_columns = std::make_shared<ASTExpressionList>();
additional_columns->children.emplace_back(declare_column);
if (ParserKeyword("FIRST").ignore(pos, expected))
if (ParserKeyword(Keyword::FIRST).ignore(pos, expected))
alter_command->first = true;
else if (ParserKeyword("AFTER").ignore(pos, expected))
else if (ParserKeyword(Keyword::AFTER).ignore(pos, expected))
{
ASTPtr after_column;
ParserIdentifier identifier_p;
@ -103,12 +103,12 @@ static inline bool parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected
auto alter_command = std::make_shared<ASTAlterCommand>();
if (ParserKeyword("PRIMARY KEY").ignore(pos, expected))
if (ParserKeyword(Keyword::PRIMARY_KEY).ignore(pos, expected))
{
alter_command->index_type = "PRIMARY_KEY";
alter_command->type = ASTAlterCommand::DROP_INDEX;
}
else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected))
else if (ParserKeyword(Keyword::FOREIGN_KEY).ignore(pos, expected))
{
if (!identifier_p.parse(pos, name, expected))
return false;
@ -117,7 +117,7 @@ static inline bool parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected
alter_command->type = ASTAlterCommand::DROP_INDEX;
alter_command->index_name = getIdentifierName(name);
}
else if (ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("KEY").ignore(pos, expected))
else if (ParserKeyword(Keyword::INDEX).ignore(pos, expected) || ParserKeyword(Keyword::KEY).ignore(pos, expected))
{
if (!identifier_p.parse(pos, name, expected))
return false;
@ -126,7 +126,7 @@ static inline bool parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected
alter_command->type = ASTAlterCommand::DROP_INDEX;
alter_command->index_name = getIdentifierName(name);
}
else if (ParserKeyword("CONSTRAINT").ignore(pos, expected) || ParserKeyword("CHECK").ignore(pos, expected))
else if (ParserKeyword(Keyword::CONSTRAINT).ignore(pos, expected) || ParserKeyword(Keyword::CHECK).ignore(pos, expected))
{
if (!identifier_p.parse(pos, name, expected))
return false;
@ -136,7 +136,7 @@ static inline bool parseDropCommand(IParser::Pos & pos, ASTPtr & node, Expected
}
else
{
ParserKeyword("COLUMN").ignore(pos, expected);
ParserKeyword(Keyword::COLUMN).ignore(pos, expected);
if (!identifier_p.parse(pos, name, expected))
return false;
@ -156,30 +156,30 @@ static inline bool parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Expected
ParserIdentifier identifier_p;
auto alter_command = std::make_shared<ASTAlterCommand>();
if (ParserKeyword("INDEX").ignore(pos, expected))
if (ParserKeyword(Keyword::INDEX).ignore(pos, expected))
{
/// ALTER INDEX index_name {VISIBLE | INVISIBLE}
if (!identifier_p.parse(pos, name, expected))
return false;
alter_command->index_visible = ParserKeyword("VISIBLE").ignore(pos, expected);
alter_command->index_visible = ParserKeyword(Keyword::VISIBLE).ignore(pos, expected);
if (!alter_command->index_visible && !ParserKeyword("INVISIBLE").ignore(pos, expected))
if (!alter_command->index_visible && !ParserKeyword(Keyword::INVISIBLE).ignore(pos, expected))
return false;
alter_command->type = ASTAlterCommand::MODIFY_INDEX_VISIBLE;
alter_command->index_name = getIdentifierName(name);
}
else if (ParserKeyword("CHECK").ignore(pos, expected) || ParserKeyword("CONSTRAINT").ignore(pos, expected))
else if (ParserKeyword(Keyword::CHECK).ignore(pos, expected) || ParserKeyword(Keyword::CONSTRAINT).ignore(pos, expected))
{
/// ALTER {CHECK | CONSTRAINT} symbol [NOT] ENFORCED
if (!identifier_p.parse(pos, name, expected))
return false;
alter_command->not_check_enforced = ParserKeyword("NOT").ignore(pos, expected);
alter_command->not_check_enforced = ParserKeyword(Keyword::NOT).ignore(pos, expected);
if (!ParserKeyword("ENFORCED").ignore(pos, expected))
if (!ParserKeyword(Keyword::ENFORCED).ignore(pos, expected))
return false;
alter_command->type = ASTAlterCommand::MODIFY_CHECK;
@ -189,14 +189,14 @@ static inline bool parseAlterCommand(IParser::Pos & pos, ASTPtr & node, Expected
{
/// ALTER [COLUMN] col_name {SET DEFAULT {literal | (expr)} | DROP DEFAULT}
ParserKeyword("COLUMN").ignore(pos, expected);
ParserKeyword(Keyword::COLUMN).ignore(pos, expected);
if (!identifier_p.parse(pos, name, expected))
return false;
if (ParserKeyword("DROP DEFAULT").ignore(pos, expected))
if (ParserKeyword(Keyword::DROP_DEFAULT).ignore(pos, expected))
alter_command->type = ASTAlterCommand::DROP_COLUMN_DEFAULT;
else if (ParserKeyword("SET DEFAULT").ignore(pos, expected))
else if (ParserKeyword(Keyword::SET_DEFAULT).ignore(pos, expected))
{
ASTPtr default_expression;
ParserExpression expression_p;
@ -225,12 +225,12 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte
ParserIdentifier identifier_p;
auto alter_command = std::make_shared<ASTAlterCommand>();
if (ParserKeyword("COLUMN").ignore(pos, expected))
if (ParserKeyword(Keyword::COLUMN).ignore(pos, expected))
{
if (!identifier_p.parse(pos, old_name, expected))
return false;
if (!ParserKeyword("TO").ignore(pos, expected))
if (!ParserKeyword(Keyword::TO).ignore(pos, expected))
return false;
if (!identifier_p.parse(pos, new_name, expected))
@ -240,7 +240,7 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte
alter_command->old_name = getIdentifierName(old_name);
alter_command->column_name = getIdentifierName(new_name);
}
else if (ParserKeyword("TO").ignore(pos, expected) || ParserKeyword("AS").ignore(pos, expected))
else if (ParserKeyword(Keyword::TO).ignore(pos, expected) || ParserKeyword(Keyword::AS).ignore(pos, expected))
{
if (!ParserCompoundIdentifier(true).parse(pos, new_name, expected))
return false;
@ -250,12 +250,12 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte
alter_command->new_table_name = new_table_id.table_name;
alter_command->new_database_name = new_table_id.database_name;
}
else if (ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("KEY").ignore(pos, expected))
else if (ParserKeyword(Keyword::INDEX).ignore(pos, expected) || ParserKeyword(Keyword::KEY).ignore(pos, expected))
{
if (!identifier_p.parse(pos, old_name, expected))
return false;
if (!ParserKeyword("TO").ignore(pos, expected))
if (!ParserKeyword(Keyword::TO).ignore(pos, expected))
return false;
if (!identifier_p.parse(pos, new_name, expected))
@ -283,7 +283,7 @@ static inline bool parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected
{
auto alter_command = std::make_shared<ASTAlterCommand>();
if (ParserKeyword("ORDER BY").ignore(pos, expected))
if (ParserKeyword(Keyword::ORDER_BY).ignore(pos, expected))
{
/// ORDER BY col_name [, col_name] ...
ASTPtr columns;
@ -335,7 +335,7 @@ static inline bool parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expecte
ASTPtr old_column_name;
auto alter_command = std::make_shared<ASTAlterCommand>();
ParserKeyword("COLUMN").ignore(pos, expected);
ParserKeyword(Keyword::COLUMN).ignore(pos, expected);
if (exists_old_column_name && !ParserIdentifier().parse(pos, old_column_name, expected))
return false;
@ -343,9 +343,9 @@ static inline bool parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expecte
if (!ParserDeclareColumn().parse(pos, additional_column, expected))
return false;
if (ParserKeyword("FIRST").ignore(pos, expected))
if (ParserKeyword(Keyword::FIRST).ignore(pos, expected))
alter_command->first = true;
else if (ParserKeyword("AFTER").ignore(pos, expected))
else if (ParserKeyword(Keyword::AFTER).ignore(pos, expected))
{
ASTPtr after_column;
ParserIdentifier identifier_p;
@ -368,12 +368,12 @@ static inline bool parseModifyCommand(IParser::Pos & pos, ASTPtr & node, Expecte
bool ParserAlterCommand::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword k_add("ADD");
ParserKeyword k_drop("DROP");
ParserKeyword k_alter("ALTER");
ParserKeyword k_rename("RENAME");
ParserKeyword k_modify("MODIFY");
ParserKeyword k_change("CHANGE");
ParserKeyword k_add(Keyword::ADD);
ParserKeyword k_drop(Keyword::DROP);
ParserKeyword k_alter(Keyword::ALTER);
ParserKeyword k_rename(Keyword::RENAME);
ParserKeyword k_modify(Keyword::MODIFY);
ParserKeyword k_change(Keyword::CHANGE);
if (k_add.ignore(pos, expected))
return parseAddCommand(pos, node, expected);

View File

@ -33,7 +33,7 @@ bool ParserAlterQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & e
ASTPtr table;
ASTPtr command_list;
if (!ParserKeyword("ALTER TABLE").ignore(pos, expected))
if (!ParserKeyword(Keyword::ALTER_TABLE).ignore(pos, expected))
return false;
if (!ParserCompoundIdentifier(true).parse(pos, table, expected))

View File

@ -51,29 +51,29 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool is_temporary = false;
bool if_not_exists = false;
if (!ParserKeyword("CREATE").ignore(pos, expected))
if (!ParserKeyword(Keyword::CREATE).ignore(pos, expected))
return false;
if (ParserKeyword("TEMPORARY").ignore(pos, expected))
if (ParserKeyword(Keyword::TEMPORARY).ignore(pos, expected))
is_temporary = true;
if (!ParserKeyword("TABLE").ignore(pos, expected))
if (!ParserKeyword(Keyword::TABLE).ignore(pos, expected))
return false;
if (ParserKeyword("IF NOT EXISTS").ignore(pos, expected))
if (ParserKeyword(Keyword::IF_NOT_EXISTS).ignore(pos, expected))
if_not_exists = true;
if (!ParserCompoundIdentifier(true).parse(pos, table, expected))
return false;
if (ParserKeyword("LIKE").ignore(pos, expected))
if (ParserKeyword(Keyword::LIKE).ignore(pos, expected))
{
if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected))
return false;
}
else if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected))
{
if (ParserKeyword("LIKE").ignore(pos, expected))
if (ParserKeyword(Keyword::LIKE).ignore(pos, expected))
{
if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected))
return false;

View File

@ -32,22 +32,22 @@ bool ParserDeclareConstraint::parseImpl(IParser::Pos & pos, ASTPtr & node, Expec
ASTPtr index_check_expression;
ParserExpression p_expression;
if (ParserKeyword("CONSTRAINT").ignore(pos, expected))
if (ParserKeyword(Keyword::CONSTRAINT).ignore(pos, expected))
{
if (!ParserKeyword("CHECK").checkWithoutMoving(pos, expected))
if (!ParserKeyword(Keyword::CHECK).checkWithoutMoving(pos, expected))
ParserIdentifier().parse(pos, constraint_symbol, expected);
}
if (!ParserKeyword("CHECK").ignore(pos, expected))
if (!ParserKeyword(Keyword::CHECK).ignore(pos, expected))
return false;
if (!p_expression.parse(pos, index_check_expression, expected))
return false;
if (ParserKeyword("NOT").ignore(pos, expected))
if (ParserKeyword(Keyword::NOT).ignore(pos, expected))
{
if (!ParserKeyword("ENFORCED").ignore(pos, expected))
if (!ParserKeyword(Keyword::ENFORCED).ignore(pos, expected))
return false;
enforced = false;
@ -55,7 +55,7 @@ bool ParserDeclareConstraint::parseImpl(IParser::Pos & pos, ASTPtr & node, Expec
else
{
enforced = true;
ParserKeyword("ENFORCED").ignore(pos, expected);
ParserKeyword(Keyword::ENFORCED).ignore(pos, expected);
}
auto declare_constraint = std::make_shared<ASTDeclareConstraint>();

View File

@ -24,8 +24,8 @@ protected:
if (!p_expression.parse(pos, node, expected))
return false;
ParserKeyword("ASC").ignore(pos, expected);
ParserKeyword("DESC").ignore(pos, expected);
ParserKeyword(Keyword::ASC).ignore(pos, expected);
ParserKeyword(Keyword::DESC).ignore(pos, expected);
return true;
}
};
@ -61,12 +61,12 @@ ASTPtr ASTDeclareIndex::clone() const
static inline bool parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected)
{
ASTPtr temp_node;
ParserKeyword k_key("KEY");
ParserKeyword k_index("INDEX");
ParserKeyword k_key(Keyword::KEY);
ParserKeyword k_index(Keyword::INDEX);
ParserIdentifier p_identifier;
if (ParserKeyword("SPATIAL").ignore(pos, expected))
if (ParserKeyword(Keyword::SPATIAL).ignore(pos, expected))
{
if (!k_key.ignore(pos, expected))
k_index.ignore(pos, expected);
@ -75,7 +75,7 @@ static inline bool parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_
if (p_identifier.parse(pos, temp_node, expected))
index_name = temp_node->as<ASTIdentifier>()->name();
}
else if (ParserKeyword("FULLTEXT").ignore(pos, expected))
else if (ParserKeyword(Keyword::FULLTEXT).ignore(pos, expected))
{
if (!k_key.ignore(pos, expected))
k_index.ignore(pos, expected);
@ -96,7 +96,7 @@ static inline bool parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_
if (p_identifier.parse(pos, temp_node, expected))
index_name = temp_node->as<ASTIdentifier>()->name();
if (ParserKeyword("USING").ignore(pos, expected))
if (ParserKeyword(Keyword::USING).ignore(pos, expected))
{
if (!p_identifier.parse(pos, temp_node, expected))
return false;
@ -113,11 +113,11 @@ static inline bool parseDeclareConstraintIndex(IParser::Pos & pos, String & inde
ASTPtr temp_node;
ParserIdentifier p_identifier;
if (ParserKeyword("CONSTRAINT").ignore(pos, expected))
if (ParserKeyword(Keyword::CONSTRAINT).ignore(pos, expected))
{
if (!ParserKeyword("PRIMARY").checkWithoutMoving(pos, expected) && !ParserKeyword("UNIQUE").checkWithoutMoving(pos, expected)
&& !ParserKeyword("FOREIGN").checkWithoutMoving(pos, expected))
if (!ParserKeyword(Keyword::PRIMARY).checkWithoutMoving(pos, expected) && !ParserKeyword(Keyword::UNIQUE).checkWithoutMoving(pos, expected)
&& !ParserKeyword(Keyword::FOREIGN).checkWithoutMoving(pos, expected))
{
if (!p_identifier.parse(pos, temp_node, expected))
return false;
@ -126,16 +126,16 @@ static inline bool parseDeclareConstraintIndex(IParser::Pos & pos, String & inde
}
}
if (ParserKeyword("UNIQUE").ignore(pos, expected))
if (ParserKeyword(Keyword::UNIQUE).ignore(pos, expected))
{
if (!ParserKeyword("KEY").ignore(pos, expected))
ParserKeyword("INDEX").ignore(pos, expected);
if (!ParserKeyword(Keyword::KEY).ignore(pos, expected))
ParserKeyword(Keyword::INDEX).ignore(pos, expected);
if (p_identifier.parse(pos, temp_node, expected))
index_name = temp_node->as<ASTIdentifier>()->name(); /// reset index_name
index_type = "UNIQUE_BTREE"; /// default btree index_type
if (ParserKeyword("USING").ignore(pos, expected))
if (ParserKeyword(Keyword::USING).ignore(pos, expected))
{
if (!p_identifier.parse(pos, temp_node, expected))
return false;
@ -143,10 +143,10 @@ static inline bool parseDeclareConstraintIndex(IParser::Pos & pos, String & inde
index_type = "UNIQUE_" + temp_node->as<ASTIdentifier>()->name();
}
}
else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected))
else if (ParserKeyword(Keyword::PRIMARY_KEY).ignore(pos, expected))
{
index_type = "PRIMARY_KEY_BTREE"; /// default btree index_type
if (ParserKeyword("USING").ignore(pos, expected))
if (ParserKeyword(Keyword::USING).ignore(pos, expected))
{
if (!p_identifier.parse(pos, temp_node, expected))
return false;
@ -154,7 +154,7 @@ static inline bool parseDeclareConstraintIndex(IParser::Pos & pos, String & inde
index_type = "PRIMARY_KEY_" + temp_node->as<ASTIdentifier>()->name();
}
}
else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected))
else if (ParserKeyword(Keyword::FOREIGN_KEY).ignore(pos, expected))
{
index_type = "FOREIGN";
if (p_identifier.parse(pos, temp_node, expected))

View File

@ -23,7 +23,7 @@ bool ParserDeclareOptionImpl<recursive>::parseImpl(Pos & pos, ASTPtr & node, Exp
{
auto iterator = usage_parsers_cached.find(usage_name);
if (iterator == usage_parsers_cached.end())
iterator = usage_parsers_cached.insert(std::make_pair(usage_name, std::make_shared<ParserKeyword>(usage_name))).first;
iterator = usage_parsers_cached.insert(std::make_pair(usage_name, ParserKeyword::createDeprecatedPtr(usage_name))).first;
return iterator->second;
};

View File

@ -46,7 +46,7 @@ ASTPtr ASTDeclarePartition::clone() const
bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"PARTITION"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::PARTITION}.ignore(pos, expected))
return false;
ASTPtr options;
@ -60,15 +60,15 @@ bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
if (!p_identifier.parse(pos, partition_name, expected))
return false;
ParserKeyword p_values("VALUES");
ParserKeyword p_values(Keyword::VALUES);
if (p_values.ignore(pos, expected))
{
if (ParserKeyword{"IN"}.ignore(pos, expected))
if (ParserKeyword{Keyword::IN}.ignore(pos, expected))
{
if (!p_expression.parse(pos, in_expression, expected))
return false;
}
else if (ParserKeyword{"LESS THAN"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::LESS_THAN}.ignore(pos, expected))
{
if (!p_expression.parse(pos, less_than, expected))
return false;

View File

@ -49,30 +49,30 @@ static inline bool parsePartitionExpression(IParser::Pos & pos, std::string & ty
{
ASTPtr expression;
ParserExpression p_expression;
if (!subpartition && ParserKeyword("LIST").ignore(pos, expected))
if (!subpartition && ParserKeyword(Keyword::LIST).ignore(pos, expected))
{
type = "list";
ParserKeyword("COLUMNS").ignore(pos, expected);
ParserKeyword(Keyword::COLUMNS).ignore(pos, expected);
if (!p_expression.parse(pos, expression, expected))
return false;
}
else if (!subpartition && ParserKeyword("RANGE").ignore(pos, expected))
else if (!subpartition && ParserKeyword(Keyword::RANGE).ignore(pos, expected))
{
type = "range";
ParserKeyword("COLUMNS").ignore(pos, expected);
ParserKeyword(Keyword::COLUMNS).ignore(pos, expected);
if (!p_expression.parse(pos, expression, expected))
return false;
}
else
{
if (ParserKeyword("LINEAR").ignore(pos, expected))
if (ParserKeyword(Keyword::LINEAR).ignore(pos, expected))
type = "linear_";
if (ParserKeyword("KEY").ignore(pos, expected))
if (ParserKeyword(Keyword::KEY).ignore(pos, expected))
{
type += "key";
if (ParserKeyword("ALGORITHM").ignore(pos, expected))
if (ParserKeyword(Keyword::ALGORITHM).ignore(pos, expected))
{
if (!ParserToken(TokenType::Equals).ignore(pos, expected))
return false;
@ -93,7 +93,7 @@ static inline bool parsePartitionExpression(IParser::Pos & pos, std::string & ty
if (!p_expression.parse(pos, expression, expected))
return false;
}
else if (ParserKeyword("HASH").ignore(pos, expected))
else if (ParserKeyword(Keyword::HASH).ignore(pos, expected))
{
type += "hash";
if (!p_expression.parse(pos, expression, expected))
@ -117,25 +117,25 @@ bool ParserDeclarePartitionOptions::parseImpl(Pos & pos, ASTPtr & node, Expected
ASTPtr subpartition_expression;
ASTPtr declare_partitions;
if (!ParserKeyword("PARTITION BY").ignore(pos, expected))
if (!ParserKeyword(Keyword::PARTITION_BY).ignore(pos, expected))
return false;
if (!parsePartitionExpression(pos, partition_type, partition_expression, expected))
return false;
if (ParserKeyword("PARTITIONS").ignore(pos, expected))
if (ParserKeyword(Keyword::PARTITIONS).ignore(pos, expected))
{
ParserLiteral p_literal;
if (!p_literal.parse(pos, partition_numbers, expected))
return false;
}
if (ParserKeyword("SUBPARTITION BY").ignore(pos, expected))
if (ParserKeyword(Keyword::SUBPARTITION_BY).ignore(pos, expected))
{
if (!parsePartitionExpression(pos, subpartition_type, subpartition_expression, expected, true))
return false;
if (ParserKeyword("SUBPARTITIONS").ignore(pos, expected))
if (ParserKeyword(Keyword::SUBPARTITIONS).ignore(pos, expected))
{
ParserLiteral p_literal;
if (!p_literal.parse(pos, subpartition_numbers, expected))

View File

@ -13,15 +13,15 @@ namespace MySQLParser
bool parseReferenceOption(IParser::Pos & pos, ASTDeclareReference::ReferenceOption & option, Expected & expected)
{
if (ParserKeyword("RESTRICT").ignore(pos, expected))
if (ParserKeyword(Keyword::RESTRICT).ignore(pos, expected))
option = ASTDeclareReference::RESTRICT;
else if (ParserKeyword("CASCADE").ignore(pos, expected))
else if (ParserKeyword(Keyword::CASCADE).ignore(pos, expected))
option = ASTDeclareReference::CASCADE;
else if (ParserKeyword("SET NULL").ignore(pos, expected))
else if (ParserKeyword(Keyword::SET_NULL).ignore(pos, expected))
option = ASTDeclareReference::SET_NULL;
else if (ParserKeyword("NO ACTION").ignore(pos, expected))
else if (ParserKeyword(Keyword::NO_ACTION).ignore(pos, expected))
option = ASTDeclareReference::NO_ACTION;
else if (ParserKeyword("SET DEFAULT").ignore(pos, expected))
else if (ParserKeyword(Keyword::SET_DEFAULT).ignore(pos, expected))
option = ASTDeclareReference::SET_DEFAULT;
else
return false;
@ -53,7 +53,7 @@ bool ParserDeclareReference::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
ASTDeclareReference::ReferenceOption delete_option = ASTDeclareReference::RESTRICT;
ASTDeclareReference::ReferenceOption update_option = ASTDeclareReference::RESTRICT;
if (!ParserKeyword("REFERENCES").ignore(pos, expected))
if (!ParserKeyword(Keyword::REFERENCES).ignore(pos, expected))
return false;
if (!p_identifier.parse(pos, table_name, expected))
@ -62,13 +62,13 @@ bool ParserDeclareReference::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
if (!p_expression.parse(pos, expression, expected))
return false;
if (ParserKeyword("MATCH").ignore(pos, expected))
if (ParserKeyword(Keyword::MATCH).ignore(pos, expected))
{
if (ParserKeyword("FULL").ignore(pos, expected))
if (ParserKeyword(Keyword::FULL).ignore(pos, expected))
match_kind = ASTDeclareReference::MATCH_FULL;
else if (ParserKeyword("SIMPLE").ignore(pos, expected))
else if (ParserKeyword(Keyword::SIMPLE).ignore(pos, expected))
match_kind = ASTDeclareReference::MATCH_SIMPLE;
else if (ParserKeyword("PARTIAL").ignore(pos, expected))
else if (ParserKeyword(Keyword::PARTIAL).ignore(pos, expected))
match_kind = ASTDeclareReference::MATCH_PARTIAL;
else
return false;
@ -76,12 +76,12 @@ bool ParserDeclareReference::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
while (true)
{
if (ParserKeyword("ON DELETE").ignore(pos, expected))
if (ParserKeyword(Keyword::ON_DELETE).ignore(pos, expected))
{
if (!parseReferenceOption(pos, delete_option, expected))
return false;
}
else if (ParserKeyword("ON UPDATE").ignore(pos, expected))
else if (ParserKeyword(Keyword::ON_UPDATE).ignore(pos, expected))
{
if (!parseReferenceOption(pos, update_option, expected))
return false;

View File

@ -13,7 +13,7 @@ namespace MySQLParser
bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SUBPARTITION"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SUBPARTITION}.ignore(pos, expected))
return false;
ASTPtr options;

View File

@ -23,7 +23,7 @@ protected:
{
if constexpr (allow_default)
{
if (ParserKeyword("DEFAULT").ignore(pos, expected))
if (ParserKeyword(Keyword::DEFAULT).ignore(pos, expected))
{
node = std::make_shared<ASTIdentifier>("DEFAULT");
return true;
@ -49,11 +49,11 @@ protected:
if (!p_identifier.parse(pos, node, expected))
return false;
if (ParserKeyword("STORAGE").ignore(pos, expected))
if (ParserKeyword(Keyword::STORAGE).ignore(pos, expected))
{
if (!ParserKeyword("DISK").ignore(pos, expected))
if (!ParserKeyword(Keyword::DISK).ignore(pos, expected))
{
if (!ParserKeyword("MEMORY").ignore(pos, expected))
if (!ParserKeyword(Keyword::MEMORY).ignore(pos, expected))
return false;
}
}

View File

@ -23,20 +23,20 @@ ASTPtr ASTDropQuery::clone() const
bool ParserDropQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_truncate("TRUNCATE");
ParserKeyword s_table("TABLE");
ParserKeyword s_database("DATABASE");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_view("VIEW");
ParserKeyword on("ON");
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_truncate(Keyword::TRUNCATE);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_database(Keyword::DATABASE);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword on(Keyword::ON);
ParserIdentifier name_p(false);
ParserKeyword s_event("EVENT");
ParserKeyword s_function("FUNCTION");
ParserKeyword s_index("INDEX");
ParserKeyword s_server("SERVER");
ParserKeyword s_trigger("TRIGGER");
ParserKeyword s_event(Keyword::EVENT);
ParserKeyword s_function(Keyword::FUNCTION);
ParserKeyword s_index(Keyword::INDEX);
ParserKeyword s_server(Keyword::SERVER);
ParserKeyword s_trigger(Keyword::TRIGGER);
auto query = std::make_shared<ASTDropQuery>();
node = query;

View File

@ -11,15 +11,16 @@ namespace DB
bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_alter("ALTER");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_delete("DELETE");
ParserKeyword s_alter(Keyword::ALTER);
ParserKeyword s_collection(Keyword::NAMED_COLLECTION);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_delete(Keyword::DELETE);
ParserIdentifier name_p;
ParserKeyword s_set("SET");
ParserKeyword s_overridable("OVERRIDABLE");
ParserKeyword s_not_overridable("NOT OVERRIDABLE");
ParserKeyword s_set(Keyword::SET);
ParserKeyword s_overridable(Keyword::OVERRIDABLE);
ParserKeyword s_not_overridable(Keyword::NOT_OVERRIDABLE);
ParserToken s_comma(TokenType::Comma);
String cluster_str;

View File

@ -24,97 +24,97 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
auto command = std::make_shared<ASTAlterCommand>();
node = command;
ParserKeyword s_add_column("ADD COLUMN");
ParserKeyword s_drop_column("DROP COLUMN");
ParserKeyword s_clear_column("CLEAR COLUMN");
ParserKeyword s_modify_column("MODIFY COLUMN");
ParserKeyword s_alter_column("ALTER COLUMN");
ParserKeyword s_rename_column("RENAME COLUMN");
ParserKeyword s_comment_column("COMMENT COLUMN");
ParserKeyword s_materialize_column("MATERIALIZE COLUMN");
ParserKeyword s_add_column(Keyword::ADD_COLUMN);
ParserKeyword s_drop_column(Keyword::DROP_COLUMN);
ParserKeyword s_clear_column(Keyword::CLEAR_COLUMN);
ParserKeyword s_modify_column(Keyword::MODIFY_COLUMN);
ParserKeyword s_alter_column(Keyword::ALTER_COLUMN);
ParserKeyword s_rename_column(Keyword::RENAME_COLUMN);
ParserKeyword s_comment_column(Keyword::COMMENT_COLUMN);
ParserKeyword s_materialize_column(Keyword::MATERIALIZE_COLUMN);
ParserKeyword s_modify_order_by("MODIFY ORDER BY");
ParserKeyword s_modify_sample_by("MODIFY SAMPLE BY");
ParserKeyword s_modify_ttl("MODIFY TTL");
ParserKeyword s_materialize_ttl("MATERIALIZE TTL");
ParserKeyword s_modify_setting("MODIFY SETTING");
ParserKeyword s_reset_setting("RESET SETTING");
ParserKeyword s_modify_query("MODIFY QUERY");
ParserKeyword s_modify_sql_security("MODIFY SQL SECURITY");
ParserKeyword s_modify_refresh("MODIFY REFRESH");
ParserKeyword s_modify_order_by(Keyword::MODIFY_ORDER_BY);
ParserKeyword s_modify_sample_by(Keyword::MODIFY_SAMPLE_BY);
ParserKeyword s_modify_ttl(Keyword::MODIFY_TTL);
ParserKeyword s_materialize_ttl(Keyword::MATERIALIZE_TTL);
ParserKeyword s_modify_setting(Keyword::MODIFY_SETTING);
ParserKeyword s_reset_setting(Keyword::RESET_SETTING);
ParserKeyword s_modify_query(Keyword::MODIFY_QUERY);
ParserKeyword s_modify_sql_security(Keyword::MODIFY_SQL_SECURITY);
ParserKeyword s_modify_refresh(Keyword::MODIFY_REFRESH);
ParserKeyword s_add_index("ADD INDEX");
ParserKeyword s_drop_index("DROP INDEX");
ParserKeyword s_clear_index("CLEAR INDEX");
ParserKeyword s_materialize_index("MATERIALIZE INDEX");
ParserKeyword s_add_index(Keyword::ADD_INDEX);
ParserKeyword s_drop_index(Keyword::DROP_INDEX);
ParserKeyword s_clear_index(Keyword::CLEAR_INDEX);
ParserKeyword s_materialize_index(Keyword::MATERIALIZE_INDEX);
ParserKeyword s_add_statistic("ADD STATISTIC");
ParserKeyword s_drop_statistic("DROP STATISTIC");
ParserKeyword s_clear_statistic("CLEAR STATISTIC");
ParserKeyword s_materialize_statistic("MATERIALIZE STATISTIC");
ParserKeyword s_add_statistic(Keyword::ADD_STATISTIC);
ParserKeyword s_drop_statistic(Keyword::DROP_STATISTIC);
ParserKeyword s_clear_statistic(Keyword::CLEAR_STATISTIC);
ParserKeyword s_materialize_statistic(Keyword::MATERIALIZE_STATISTIC);
ParserKeyword s_add_constraint("ADD CONSTRAINT");
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
ParserKeyword s_add_constraint(Keyword::ADD_CONSTRAINT);
ParserKeyword s_drop_constraint(Keyword::DROP_CONSTRAINT);
ParserKeyword s_add_projection("ADD PROJECTION");
ParserKeyword s_drop_projection("DROP PROJECTION");
ParserKeyword s_clear_projection("CLEAR PROJECTION");
ParserKeyword s_materialize_projection("MATERIALIZE PROJECTION");
ParserKeyword s_modify_comment("MODIFY COMMENT");
ParserKeyword s_add_projection(Keyword::ADD_PROJECTION);
ParserKeyword s_drop_projection(Keyword::DROP_PROJECTION);
ParserKeyword s_clear_projection(Keyword::CLEAR_PROJECTION);
ParserKeyword s_materialize_projection(Keyword::MATERIALIZE_PROJECTION);
ParserKeyword s_modify_comment(Keyword::MODIFY_COMMENT);
ParserKeyword s_add("ADD");
ParserKeyword s_drop("DROP");
ParserKeyword s_modify("MODIFY");
ParserKeyword s_add(Keyword::ADD);
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_modify(Keyword::MODIFY);
ParserKeyword s_attach_partition("ATTACH PARTITION");
ParserKeyword s_attach_part("ATTACH PART");
ParserKeyword s_detach_partition("DETACH PARTITION");
ParserKeyword s_detach_part("DETACH PART");
ParserKeyword s_drop_partition("DROP PARTITION");
ParserKeyword s_drop_part("DROP PART");
ParserKeyword s_forget_partition("FORGET PARTITION");
ParserKeyword s_move_partition("MOVE PARTITION");
ParserKeyword s_move_part("MOVE PART");
ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION");
ParserKeyword s_drop_detached_part("DROP DETACHED PART");
ParserKeyword s_fetch_partition("FETCH PARTITION");
ParserKeyword s_fetch_part("FETCH PART");
ParserKeyword s_replace_partition("REPLACE PARTITION");
ParserKeyword s_freeze("FREEZE");
ParserKeyword s_unfreeze("UNFREEZE");
ParserKeyword s_partition("PARTITION");
ParserKeyword s_attach_partition(Keyword::ATTACH_PARTITION);
ParserKeyword s_attach_part(Keyword::ATTACH_PART);
ParserKeyword s_detach_partition(Keyword::DETACH_PARTITION);
ParserKeyword s_detach_part(Keyword::DETACH_PART);
ParserKeyword s_drop_partition(Keyword::DROP_PARTITION);
ParserKeyword s_drop_part(Keyword::DROP_PART);
ParserKeyword s_forget_partition(Keyword::FORGET_PARTITION);
ParserKeyword s_move_partition(Keyword::MOVE_PARTITION);
ParserKeyword s_move_part(Keyword::MOVE_PART);
ParserKeyword s_drop_detached_partition(Keyword::DROP_DETACHED_PARTITION);
ParserKeyword s_drop_detached_part(Keyword::DROP_DETACHED_PART);
ParserKeyword s_fetch_partition(Keyword::FETCH_PARTITION);
ParserKeyword s_fetch_part(Keyword::FETCH_PART);
ParserKeyword s_replace_partition(Keyword::REPLACE_PARTITION);
ParserKeyword s_freeze(Keyword::FREEZE);
ParserKeyword s_unfreeze(Keyword::UNFREEZE);
ParserKeyword s_partition(Keyword::PARTITION);
ParserKeyword s_first("FIRST");
ParserKeyword s_after("AFTER");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_from("FROM");
ParserKeyword s_in_partition("IN PARTITION");
ParserKeyword s_with("WITH");
ParserKeyword s_name("NAME");
ParserKeyword s_first(Keyword::FIRST);
ParserKeyword s_after(Keyword::AFTER);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_from(Keyword::FROM);
ParserKeyword s_in_partition(Keyword::IN_PARTITION);
ParserKeyword s_with(Keyword::WITH);
ParserKeyword s_name(Keyword::NAME);
ParserKeyword s_to_disk("TO DISK");
ParserKeyword s_to_volume("TO VOLUME");
ParserKeyword s_to_table("TO TABLE");
ParserKeyword s_to_shard("TO SHARD");
ParserKeyword s_to_disk(Keyword::TO_DISK);
ParserKeyword s_to_volume(Keyword::TO_VOLUME);
ParserKeyword s_to_table(Keyword::TO_TABLE);
ParserKeyword s_to_shard(Keyword::TO_SHARD);
ParserKeyword s_delete("DELETE");
ParserKeyword s_update("UPDATE");
ParserKeyword s_where("WHERE");
ParserKeyword s_to("TO");
ParserKeyword s_delete(Keyword::DELETE);
ParserKeyword s_update(Keyword::UPDATE);
ParserKeyword s_where(Keyword::WHERE);
ParserKeyword s_to(Keyword::TO);
ParserKeyword s_remove("REMOVE");
ParserKeyword s_default("DEFAULT");
ParserKeyword s_materialized("MATERIALIZED");
ParserKeyword s_alias("ALIAS");
ParserKeyword s_comment("COMMENT");
ParserKeyword s_codec("CODEC");
ParserKeyword s_ttl("TTL");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_remove(Keyword::REMOVE);
ParserKeyword s_default(Keyword::DEFAULT);
ParserKeyword s_materialized(Keyword::MATERIALIZED);
ParserKeyword s_alias(Keyword::ALIAS);
ParserKeyword s_comment(Keyword::COMMENT);
ParserKeyword s_codec(Keyword::CODEC);
ParserKeyword s_ttl(Keyword::TTL);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_remove_ttl("REMOVE TTL");
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
ParserKeyword s_apply_deleted_mask("APPLY DELETED MASK");
ParserKeyword s_remove_ttl(Keyword::REMOVE_TTL);
ParserKeyword s_remove_sample_by(Keyword::REMOVE_SAMPLE_BY);
ParserKeyword s_apply_deleted_mask(Keyword::APPLY_DELETED_MASK);
ParserToken parser_opening_round_bracket(TokenType::OpeningRoundBracket);
ParserToken parser_closing_round_bracket(TokenType::ClosingRoundBracket);
@ -720,19 +720,19 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (s_remove.ignore(pos, expected))
{
if (s_default.ignore(pos, expected))
command->remove_property = "DEFAULT";
command->remove_property = toStringView(Keyword::DEFAULT);
else if (s_materialized.ignore(pos, expected))
command->remove_property = "MATERIALIZED";
command->remove_property = toStringView(Keyword::MATERIALIZED);
else if (s_alias.ignore(pos, expected))
command->remove_property = "ALIAS";
command->remove_property = toStringView(Keyword::ALIAS);
else if (s_comment.ignore(pos, expected))
command->remove_property = "COMMENT";
command->remove_property = toStringView(Keyword::COMMENT);
else if (s_codec.ignore(pos, expected))
command->remove_property = "CODEC";
command->remove_property = toStringView(Keyword::CODEC);
else if (s_ttl.ignore(pos, expected))
command->remove_property = "TTL";
command->remove_property = toStringView(Keyword::TTL);
else if (s_settings.ignore(pos, expected))
command->remove_property = "SETTINGS";
command->remove_property = toStringView(Keyword::SETTINGS);
else
return false;
}
@ -981,9 +981,9 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto query = std::make_shared<ASTAlterQuery>();
node = query;
ParserKeyword s_alter_table("ALTER TABLE");
ParserKeyword s_alter_temporary_table("ALTER TEMPORARY TABLE");
ParserKeyword s_alter_database("ALTER DATABASE");
ParserKeyword s_alter_table(Keyword::ALTER_TABLE);
ParserKeyword s_alter_temporary_table(Keyword::ALTER_TEMPORARY_TABLE);
ParserKeyword s_alter_database(Keyword::ALTER_DATABASE);
ASTAlterQuery::AlterObjectType alter_object_type;
@ -1009,7 +1009,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword(Keyword::ON).ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;

View File

@ -24,7 +24,7 @@ namespace
bool parsePartitions(IParser::Pos & pos, Expected & expected, std::optional<ASTs> & partitions)
{
if (!ParserKeyword{"PARTITION"}.ignore(pos, expected) && !ParserKeyword{"PARTITIONS"}.ignore(pos, expected))
if (!ParserKeyword(Keyword::PARTITION).ignore(pos, expected) && !ParserKeyword(Keyword::PARTITIONS).ignore(pos, expected))
return false;
ASTs result;
@ -47,7 +47,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"EXCEPT DATABASE"}.ignore(pos, expected) && !ParserKeyword{"EXCEPT DATABASES"}.ignore(pos, expected))
if (!ParserKeyword(Keyword::EXCEPT_DATABASE).ignore(pos, expected) && !ParserKeyword(Keyword::EXCEPT_DATABASES).ignore(pos, expected))
return false;
std::set<String> result;
@ -71,7 +71,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"EXCEPT TABLE"}.ignore(pos, expected) && !ParserKeyword{"EXCEPT TABLES"}.ignore(pos, expected))
if (!ParserKeyword(Keyword::EXCEPT_TABLE).ignore(pos, expected) && !ParserKeyword(Keyword::EXCEPT_TABLES).ignore(pos, expected))
return false;
std::set<DatabaseAndTableName> result;
@ -107,8 +107,8 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (ParserKeyword{"TABLE"}.ignore(pos, expected) || ParserKeyword{"DICTIONARY"}.ignore(pos, expected) ||
ParserKeyword{"VIEW"}.ignore(pos, expected))
if (ParserKeyword(Keyword::TABLE).ignore(pos, expected) || ParserKeyword(Keyword::DICTIONARY).ignore(pos, expected) ||
ParserKeyword(Keyword::VIEW).ignore(pos, expected))
{
element.type = ElementType::TABLE;
if (!parseDatabaseAndTableName(pos, expected, element.database_name, element.table_name))
@ -116,7 +116,7 @@ namespace
element.new_database_name = element.database_name;
element.new_table_name = element.table_name;
if (ParserKeyword("AS").ignore(pos, expected))
if (ParserKeyword(Keyword::AS).ignore(pos, expected))
{
if (!parseDatabaseAndTableName(pos, expected, element.new_database_name, element.new_table_name))
return false;
@ -126,7 +126,7 @@ namespace
return true;
}
if (ParserKeyword{"TEMPORARY TABLE"}.ignore(pos, expected))
if (ParserKeyword(Keyword::TEMPORARY_TABLE).ignore(pos, expected))
{
element.type = ElementType::TEMPORARY_TABLE;
@ -136,7 +136,7 @@ namespace
element.table_name = getIdentifierName(ast);
element.new_table_name = element.table_name;
if (ParserKeyword("AS").ignore(pos, expected))
if (ParserKeyword(Keyword::AS).ignore(pos, expected))
{
ast = nullptr;
if (!ParserIdentifier{}.parse(pos, ast, expected))
@ -147,7 +147,7 @@ namespace
return true;
}
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
if (ParserKeyword(Keyword::DATABASE).ignore(pos, expected))
{
element.type = ElementType::DATABASE;
@ -157,7 +157,7 @@ namespace
element.database_name = getIdentifierName(ast);
element.new_database_name = element.database_name;
if (ParserKeyword("AS").ignore(pos, expected))
if (ParserKeyword(Keyword::AS).ignore(pos, expected))
{
ast = nullptr;
if (!ParserIdentifier{}.parse(pos, ast, expected))
@ -169,7 +169,7 @@ namespace
return true;
}
if (ParserKeyword{"ALL"}.ignore(pos, expected))
if (ParserKeyword(Keyword::ALL).ignore(pos, expected))
{
element.type = ElementType::ALL;
parseExceptDatabases(pos, expected, element.except_databases);
@ -219,7 +219,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"base_backup"}.ignore(pos, expected)
return ParserKeyword{Keyword::BASE_BACKUP}.ignore(pos, expected)
&& ParserToken(TokenType::Equals).ignore(pos, expected)
&& parseBackupName(pos, expected, base_backup_name);
});
@ -234,7 +234,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"cluster_host_ids"}.ignore(pos, expected)
return ParserKeyword{Keyword::CLUSTER_HOST_IDS}.ignore(pos, expected)
&& ParserToken(TokenType::Equals).ignore(pos, expected)
&& parseClusterHostIDs(pos, expected, cluster_host_ids);
});
@ -244,7 +244,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected))
if (!ParserKeyword(Keyword::SETTINGS).ignore(pos, expected))
return false;
SettingsChanges settings_changes;
@ -291,9 +291,9 @@ namespace
bool parseSyncOrAsync(IParser::Pos & pos, Expected & expected, ASTPtr & settings)
{
bool async;
if (ParserKeyword{"ASYNC"}.ignore(pos, expected))
if (ParserKeyword(Keyword::ASYNC).ignore(pos, expected))
async = true;
else if (ParserKeyword{"SYNC"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::SYNC).ignore(pos, expected))
async = false;
else
return false;
@ -318,7 +318,7 @@ namespace
{
return IParserBase::wrapParseImpl(pos, [&]
{
return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
return ParserKeyword(Keyword::ON).ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected);
});
}
}
@ -327,9 +327,9 @@ namespace
bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
Kind kind;
if (ParserKeyword{"BACKUP"}.ignore(pos, expected))
if (ParserKeyword(Keyword::BACKUP).ignore(pos, expected))
kind = Kind::BACKUP;
else if (ParserKeyword{"RESTORE"}.ignore(pos, expected))
else if (ParserKeyword(Keyword::RESTORE).ignore(pos, expected))
kind = Kind::RESTORE;
else
return false;
@ -341,7 +341,7 @@ bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
String cluster;
parseOnCluster(pos, expected, cluster);
if (!ParserKeyword{(kind == Kind::BACKUP) ? "TO" : "FROM"}.ignore(pos, expected))
if (!ParserKeyword((kind == Kind::BACKUP) ? Keyword::TO : Keyword::FROM).ignore(pos, expected))
return false;
ASTPtr backup_name;

View File

@ -10,11 +10,11 @@ namespace DB
bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_case{"CASE"};
ParserKeyword s_when{"WHEN"};
ParserKeyword s_then{"THEN"};
ParserKeyword s_else{"ELSE"};
ParserKeyword s_end{ "END"};
ParserKeyword s_case{Keyword::CASE};
ParserKeyword s_when{Keyword::WHEN};
ParserKeyword s_then{Keyword::THEN};
ParserKeyword s_else{Keyword::ELSE};
ParserKeyword s_end{ Keyword::END};
ParserExpressionWithOptionalAlias p_expr{false};
if (!s_case.ignore(pos, expected))

View File

@ -12,7 +12,7 @@ namespace DB
bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_check_table("CHECK ALL TABLES");
ParserKeyword s_check_table(Keyword::CHECK_ALL_TABLES);
if (s_check_table.ignore(pos, expected))
{
auto query = std::make_shared<ASTCheckAllTablesQuery>();
@ -25,9 +25,9 @@ bool ParserCheckQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserCheckQuery::parseCheckTable(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_check_table("CHECK TABLE");
ParserKeyword s_partition("PARTITION");
ParserKeyword s_part("PART");
ParserKeyword s_check_table(Keyword::CHECK_TABLE);
ParserKeyword s_partition(Keyword::PARTITION);
ParserKeyword s_part(Keyword::PART);
ParserToken s_dot(TokenType::Dot);
ParserPartition partition_parser;

View File

@ -13,13 +13,13 @@ namespace DB
bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_function("FUNCTION");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_function(Keyword::FUNCTION);
ParserKeyword s_or_replace(Keyword::OR_REPLACE);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserIdentifier function_name_p;
ParserKeyword s_as("AS");
ParserKeyword s_as(Keyword::AS);
ParserExpression lambda_p;
ASTPtr function_name;

View File

@ -15,11 +15,12 @@ namespace DB
bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_type("TYPE");
ParserKeyword s_granularity("GRANULARITY");
ParserKeyword s_type(Keyword::TYPE);
ParserKeyword s_granularity(Keyword::GRANULARITY);
ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket);
ParserOrderByExpressionList order_list;
ParserDataType data_type_p;
ParserExpression expression_p;
ParserUnsignedInteger granularity_p;
@ -79,11 +80,12 @@ bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
auto query = std::make_shared<ASTCreateIndexQuery>();
node = query;
ParserKeyword s_create("CREATE");
ParserKeyword s_unique("UNIQUE");
ParserKeyword s_index("INDEX");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_unique(Keyword::UNIQUE);
ParserKeyword s_index(Keyword::INDEX);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserIdentifier index_name_p;
ParserCreateIndexDeclaration parser_create_idx_decl;

View File

@ -40,7 +40,7 @@ namespace
ASTPtr parseComment(IParser::Pos & pos, Expected & expected)
{
ParserKeyword s_comment("COMMENT");
ParserKeyword s_comment(Keyword::COMMENT);
ParserStringLiteral string_literal_parser;
ASTPtr comment;
@ -88,7 +88,11 @@ bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserSQLSecurity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserToken s_eq(TokenType::Equals);
ParserKeyword s_definer("DEFINER");
ParserKeyword s_definer(Keyword::DEFINER);
ParserKeyword s_current_user{Keyword::CURRENT_USER};
ParserKeyword s_sql_security{Keyword::SQL_SECURITY};
ParserKeyword s_invoker{Keyword::INVOKER};
ParserKeyword s_none{Keyword::NONE};
bool is_definer_current_user = false;
ASTPtr definer;
@ -99,7 +103,7 @@ bool ParserSQLSecurity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!definer && s_definer.ignore(pos, expected))
{
s_eq.ignore(pos, expected);
if (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected))
if (s_current_user.ignore(pos, expected))
is_definer_current_user = true;
else if (!ParserUserNameWithHost{}.parse(pos, definer, expected))
return false;
@ -107,13 +111,13 @@ bool ParserSQLSecurity::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
continue;
}
if (!type && ParserKeyword{"SQL SECURITY"}.ignore(pos, expected))
if (!type && s_sql_security.ignore(pos, expected))
{
if (s_definer.ignore(pos, expected))
type = SQLSecurityType::DEFINER;
else if (ParserKeyword{"INVOKER"}.ignore(pos, expected))
else if (s_invoker.ignore(pos, expected))
type = SQLSecurityType::INVOKER;
else if (ParserKeyword{"NONE"}.ignore(pos, expected))
else if (s_none.ignore(pos, expected))
type = SQLSecurityType::NONE;
else
return false;
@ -170,8 +174,8 @@ bool ParserNameList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_type("TYPE");
ParserKeyword s_granularity("GRANULARITY");
ParserKeyword s_type(Keyword::TYPE);
ParserKeyword s_granularity(Keyword::GRANULARITY);
ParserIdentifier name_p;
ParserDataType data_type_p;
@ -225,7 +229,7 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_type("TYPE");
ParserKeyword s_type(Keyword::TYPE);
ParserList columns_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserIdentifier type_p;
@ -252,8 +256,8 @@ bool ParserStatisticDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_check("CHECK");
ParserKeyword s_assume("ASSUME");
ParserKeyword s_check(Keyword::CHECK);
ParserKeyword s_assume(Keyword::ASSUME);
ParserIdentifier name_p;
ParserExpression expression_p;
@ -317,7 +321,7 @@ bool ParserProjectionDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool ParserForeignKeyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_references("REFERENCES");
ParserKeyword s_references(Keyword::REFERENCES);
ParserCompoundIdentifier table_name_p(true, true);
ParserExpression expression_p;
@ -336,20 +340,20 @@ bool ParserForeignKeyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (!expression_p.parse(pos, expr, expected))
return false;
ParserKeyword s_on("ON");
ParserKeyword s_on(Keyword::ON);
while (s_on.ignore(pos, expected))
{
ParserKeyword s_delete("DELETE");
ParserKeyword s_update("UPDATE");
ParserKeyword s_delete(Keyword::DELETE);
ParserKeyword s_update(Keyword::UPDATE);
if (!s_delete.ignore(pos, expected) && !s_update.ignore(pos, expected))
return false;
ParserKeyword s_restrict("RESTRICT");
ParserKeyword s_cascade("CASCADE");
ParserKeyword s_set_null("SET NULL");
ParserKeyword s_no_action("NO ACTION");
ParserKeyword s_set_default("SET DEFAULT");
ParserKeyword s_restrict(Keyword::RESTRICT);
ParserKeyword s_cascade(Keyword::CASCADE);
ParserKeyword s_set_null(Keyword::SET_NULL);
ParserKeyword s_no_action(Keyword::NO_ACTION);
ParserKeyword s_set_default(Keyword::SET_DEFAULT);
if (!s_restrict.ignore(pos, expected) && !s_cascade.ignore(pos, expected) &&
!s_set_null.ignore(pos, expected) && !s_no_action.ignore(pos, expected) &&
@ -368,11 +372,11 @@ bool ParserForeignKeyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_index("INDEX");
ParserKeyword s_constraint("CONSTRAINT");
ParserKeyword s_projection("PROJECTION");
ParserKeyword s_foreign_key("FOREIGN KEY");
ParserKeyword s_primary_key("PRIMARY KEY");
ParserKeyword s_index(Keyword::INDEX);
ParserKeyword s_constraint(Keyword::CONSTRAINT);
ParserKeyword s_projection(Keyword::PROJECTION);
ParserKeyword s_foreign_key(Keyword::FOREIGN_KEY);
ParserKeyword s_primary_key(Keyword::PRIMARY_KEY);
ParserIndexDeclaration index_p;
ParserConstraintDeclaration constraint_p;
@ -512,14 +516,14 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E
bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_engine("ENGINE");
ParserKeyword s_engine(Keyword::ENGINE);
ParserToken s_eq(TokenType::Equals);
ParserKeyword s_partition_by("PARTITION BY");
ParserKeyword s_primary_key("PRIMARY KEY");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_ttl("TTL");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_partition_by(Keyword::PARTITION_BY);
ParserKeyword s_primary_key(Keyword::PRIMARY_KEY);
ParserKeyword s_order_by(Keyword::ORDER_BY);
ParserKeyword s_sample_by(Keyword::SAMPLE_BY);
ParserKeyword s_ttl(Keyword::TTL);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
ParserExpression expression_p;
@ -651,16 +655,16 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_replace("REPLACE");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_replace(Keyword::REPLACE);
ParserKeyword s_or_replace(Keyword::OR_REPLACE);
ParserKeyword s_temporary(Keyword::TEMPORARY);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true);
ParserKeyword s_from("FROM");
ParserKeyword s_on("ON");
ParserKeyword s_from(Keyword::FROM);
ParserKeyword s_on(Keyword::ON);
ParserToken s_dot(TokenType::Dot);
ParserToken s_comma(TokenType::Comma);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
@ -756,13 +760,13 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto need_parse_as_select = [&is_create_empty, &pos, &expected]()
{
if (ParserKeyword{"EMPTY AS"}.ignore(pos, expected))
if (ParserKeyword{Keyword::EMPTY_AS}.ignore(pos, expected))
{
is_create_empty = true;
return true;
}
return ParserKeyword{"AS"}.ignore(pos, expected);
return ParserKeyword{Keyword::AS}.ignore(pos, expected);
};
/// List of columns.
@ -895,13 +899,13 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true);
ParserKeyword s_as("AS");
ParserKeyword s_view("VIEW");
ParserKeyword s_live("LIVE");
ParserKeyword s_as(Keyword::AS);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword s_live(Keyword::LIVE);
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
@ -945,14 +949,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (!table_name_p.parse(pos, table, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
// TO [db.]table
if (ParserKeyword{"TO"}.ignore(pos, expected))
if (ParserKeyword{Keyword::TO}.ignore(pos, expected))
{
if (!table_name_p.parse(pos, to_table, expected))
return false;
@ -1018,15 +1022,21 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_temporary(Keyword::TEMPORARY);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true);
ParserKeyword s_as("AS");
ParserKeyword s_view("VIEW");
ParserKeyword s_window("WINDOW");
ParserKeyword s_populate("POPULATE");
ParserKeyword s_as(Keyword::AS);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword s_window(Keyword::WINDOW);
ParserKeyword s_populate(Keyword::POPULATE);
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_to(Keyword::TO);
ParserKeyword s_inner(Keyword::INNER);
ParserKeyword s_watermark(Keyword::WATERMARK);
ParserKeyword s_allowed_lateness(Keyword::ALLOWED_LATENESS);
ParserKeyword s_empty(Keyword::EMPTY);
ParserToken s_dot(TokenType::Dot);
ParserToken s_eq(TokenType::Equals);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
@ -1079,14 +1089,14 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (!table_name_p.parse(pos, table, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
// TO [db.]table
if (ParserKeyword{"TO"}.ignore(pos, expected))
if (s_to.ignore(pos, expected))
{
if (!table_name_p.parse(pos, to_table, expected))
return false;
@ -1102,7 +1112,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
return false;
}
if (ParserKeyword{"INNER"}.ignore(pos, expected))
if (s_inner.ignore(pos, expected))
{
/// Inner table ENGINE for WINDOW VIEW
storage_inner.parse(pos, inner_storage, expected);
@ -1115,13 +1125,13 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
// WATERMARK
if (ParserKeyword{"WATERMARK"}.ignore(pos, expected))
if (s_watermark.ignore(pos, expected))
{
s_eq.ignore(pos, expected);
if (ParserKeyword("STRICTLY_ASCENDING").ignore(pos,expected))
if (ParserKeyword(Keyword::STRICTLY_ASCENDING).ignore(pos,expected))
is_watermark_strictly_ascending = true;
else if (ParserKeyword("ASCENDING").ignore(pos,expected))
else if (ParserKeyword(Keyword::ASCENDING).ignore(pos,expected))
is_watermark_ascending = true;
else if (watermark_p.parse(pos, watermark, expected))
is_watermark_bounded = true;
@ -1130,7 +1140,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
// ALLOWED LATENESS
if (ParserKeyword{"ALLOWED_LATENESS"}.ignore(pos, expected))
if (s_allowed_lateness.ignore(pos, expected))
{
s_eq.ignore(pos, expected);
allowed_lateness = true;
@ -1141,7 +1151,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
if (s_populate.ignore(pos, expected))
is_populate = true;
else if (ParserKeyword{"EMPTY"}.ignore(pos, expected))
else if (s_empty.ignore(pos, expected))
is_create_empty = true;
/// AS SELECT ...
@ -1194,19 +1204,19 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool ParserTableOverrideDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_table_override("TABLE OVERRIDE");
ParserKeyword s_table_override(Keyword::TABLE_OVERRIDE);
ParserIdentifier table_name_p;
ParserToken lparen_p(TokenType::OpeningRoundBracket);
ParserToken rparen_p(TokenType::ClosingRoundBracket);
ParserTablePropertiesDeclarationList table_properties_p;
ParserExpression expression_p;
ParserTTLExpressionList parser_ttl_list;
ParserKeyword s_columns("COLUMNS");
ParserKeyword s_partition_by("PARTITION BY");
ParserKeyword s_primary_key("PRIMARY KEY");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_ttl("TTL");
ParserKeyword s_columns(Keyword::COLUMNS);
ParserKeyword s_partition_by(Keyword::PARTITION_BY);
ParserKeyword s_primary_key(Keyword::PRIMARY_KEY);
ParserKeyword s_order_by(Keyword::ORDER_BY);
ParserKeyword s_sample_by(Keyword::SAMPLE_BY);
ParserKeyword s_ttl(Keyword::TTL);
ASTPtr table_name;
ASTPtr columns;
ASTPtr partition_by;
@ -1332,10 +1342,12 @@ bool ParserTableOverridesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Ex
bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_database("DATABASE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_database(Keyword::DATABASE);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_uuid(Keyword::UUID);
ParserStorage storage_p{ParserStorage::DATABASE_ENGINE};
ParserIdentifier name_p(true);
ParserTableOverridesDeclarationList table_overrides_p;
@ -1366,7 +1378,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
if (!name_p.parse(pos, database, expected))
return false;
if (ParserKeyword("UUID").ignore(pos, expected))
if (s_uuid.ignore(pos, expected))
{
ParserStringLiteral uuid_p;
ASTPtr ast_uuid;
@ -1375,7 +1387,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -1411,17 +1423,18 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true);
ParserCompoundIdentifier to_table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ false);
ParserKeyword s_as("AS");
ParserKeyword s_view("VIEW");
ParserKeyword s_materialized("MATERIALIZED");
ParserKeyword s_populate("POPULATE");
ParserKeyword s_empty("EMPTY");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_as(Keyword::AS);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword s_materialized(Keyword::MATERIALIZED);
ParserKeyword s_populate(Keyword::POPULATE);
ParserKeyword s_empty(Keyword::EMPTY);
ParserKeyword s_or_replace(Keyword::OR_REPLACE);
ParserKeyword s_to{Keyword::TO};
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
@ -1484,13 +1497,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!table_name_p.parse(pos, table, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (ParserKeyword{"REFRESH"}.ignore(pos, expected))
if (ParserKeyword{Keyword::REFRESH}.ignore(pos, expected))
{
// REFRESH only with materialized views
if (!is_materialized_view)
@ -1499,13 +1512,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
if (is_materialized_view && ParserKeyword{"TO INNER UUID"}.ignore(pos, expected))
if (is_materialized_view && ParserKeyword{Keyword::TO_INNER_UUID}.ignore(pos, expected))
{
ParserStringLiteral literal_p;
if (!literal_p.parse(pos, to_inner_uuid, expected))
return false;
}
else if (is_materialized_view && ParserKeyword{"TO"}.ignore(pos, expected))
else if (is_materialized_view && ParserKeyword{Keyword::TO}.ignore(pos, expected))
{
// TO [db.]table
if (!table_name_p.parse(pos, to_table, expected))
@ -1535,7 +1548,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
else if (s_empty.ignore(pos, expected))
is_create_empty = true;
if (ParserKeyword{"TO"}.ignore(pos, expected))
if (s_to.ignore(pos, expected))
throw Exception(
ErrorCodes::SYNTAX_ERROR, "When creating a materialized view you can't declare both 'ENGINE' and 'TO [db].[table]'");
}
@ -1617,13 +1630,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_named_collection("NAMED COLLECTION");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_as("AS");
ParserKeyword s_not_overridable("NOT OVERRIDABLE");
ParserKeyword s_overridable("OVERRIDABLE");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_named_collection(Keyword::NAMED_COLLECTION);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_as(Keyword::AS);
ParserKeyword s_not_overridable(Keyword::NOT_OVERRIDABLE);
ParserKeyword s_overridable(Keyword::OVERRIDABLE);
ParserIdentifier name_p;
ParserToken s_comma(TokenType::Comma);
@ -1644,6 +1657,7 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
if (!name_p.parse(pos, collection_name, expected))
return false;
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
@ -1685,13 +1699,13 @@ bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expec
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_replace("REPLACE");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_attach(Keyword::ATTACH);
ParserKeyword s_replace(Keyword::REPLACE);
ParserKeyword s_or_replace(Keyword::OR_REPLACE);
ParserKeyword s_dictionary(Keyword::DICTIONARY);
ParserKeyword s_if_not_exists(Keyword::IF_NOT_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserCompoundIdentifier dict_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true);
ParserToken s_left_paren(TokenType::OpeningRoundBracket);
ParserToken s_right_paren(TokenType::ClosingRoundBracket);

View File

@ -129,24 +129,24 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration<ParserCompoundI
template <typename NameParser>
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_default{"DEFAULT"};
ParserKeyword s_null{"NULL"};
ParserKeyword s_not{"NOT"};
ParserKeyword s_materialized{"MATERIALIZED"};
ParserKeyword s_ephemeral{"EPHEMERAL"};
ParserKeyword s_alias{"ALIAS"};
ParserKeyword s_auto_increment{"AUTO_INCREMENT"};
ParserKeyword s_comment{"COMMENT"};
ParserKeyword s_codec{"CODEC"};
ParserKeyword s_stat{"STATISTIC"};
ParserKeyword s_ttl{"TTL"};
ParserKeyword s_remove{"REMOVE"};
ParserKeyword s_modify_setting("MODIFY SETTING");
ParserKeyword s_reset_setting("RESET SETTING");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_type{"TYPE"};
ParserKeyword s_collate{"COLLATE"};
ParserKeyword s_primary_key{"PRIMARY KEY"};
ParserKeyword s_default{Keyword::DEFAULT};
ParserKeyword s_null{Keyword::NULL_KEYWORD};
ParserKeyword s_not{Keyword::NOT};
ParserKeyword s_materialized{Keyword::MATERIALIZED};
ParserKeyword s_ephemeral{Keyword::EPHEMERAL};
ParserKeyword s_alias{Keyword::ALIAS};
ParserKeyword s_auto_increment{Keyword::AUTO_INCREMENT};
ParserKeyword s_comment{Keyword::COMMENT};
ParserKeyword s_codec{Keyword::CODEC};
ParserKeyword s_stat{Keyword::STATISTIC};
ParserKeyword s_ttl{Keyword::TTL};
ParserKeyword s_remove{Keyword::REMOVE};
ParserKeyword s_modify_setting(Keyword::MODIFY_SETTING);
ParserKeyword s_reset_setting(Keyword::RESET_SETTING);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_type{Keyword::TYPE};
ParserKeyword s_collate{Keyword::COLLATE};
ParserKeyword s_primary_key{Keyword::PRIMARY_KEY};
NameParser name_parser;
ParserDataType type_parser;

View File

@ -64,39 +64,39 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
/// only for certain first words, otherwise we don't know how many words to parse
if (type_name_upper == "NATIONAL")
{
if (ParserKeyword("CHARACTER LARGE OBJECT").ignore(pos))
type_name_suffix = "CHARACTER LARGE OBJECT";
else if (ParserKeyword("CHARACTER VARYING").ignore(pos))
type_name_suffix = "CHARACTER VARYING";
else if (ParserKeyword("CHAR VARYING").ignore(pos))
type_name_suffix = "CHAR VARYING";
else if (ParserKeyword("CHARACTER").ignore(pos))
type_name_suffix = "CHARACTER";
else if (ParserKeyword("CHAR").ignore(pos))
type_name_suffix = "CHAR";
if (ParserKeyword(Keyword::CHARACTER_LARGE_OBJECT).ignore(pos))
type_name_suffix = toStringView(Keyword::CHARACTER_LARGE_OBJECT);
else if (ParserKeyword(Keyword::CHARACTER_VARYING).ignore(pos))
type_name_suffix = toStringView(Keyword::CHARACTER_VARYING);
else if (ParserKeyword(Keyword::CHAR_VARYING).ignore(pos))
type_name_suffix = toStringView(Keyword::CHAR_VARYING);
else if (ParserKeyword(Keyword::CHARACTER).ignore(pos))
type_name_suffix = toStringView(Keyword::CHARACTER);
else if (ParserKeyword(Keyword::CHAR).ignore(pos))
type_name_suffix = toStringView(Keyword::CHAR);
}
else if (type_name_upper == "BINARY" ||
type_name_upper == "CHARACTER" ||
type_name_upper == "CHAR" ||
type_name_upper == "NCHAR")
{
if (ParserKeyword("LARGE OBJECT").ignore(pos))
type_name_suffix = "LARGE OBJECT";
else if (ParserKeyword("VARYING").ignore(pos))
type_name_suffix = "VARYING";
if (ParserKeyword(Keyword::LARGE_OBJECT).ignore(pos))
type_name_suffix = toStringView(Keyword::LARGE_OBJECT);
else if (ParserKeyword(Keyword::VARYING).ignore(pos))
type_name_suffix = toStringView(Keyword::VARYING);
}
else if (type_name_upper == "DOUBLE")
{
if (ParserKeyword("PRECISION").ignore(pos))
type_name_suffix = "PRECISION";
if (ParserKeyword(Keyword::PRECISION).ignore(pos))
type_name_suffix = toStringView(Keyword::PRECISION);
}
else if (type_name_upper.find("INT") != std::string::npos)
{
/// Support SIGNED and UNSIGNED integer type modifiers for compatibility with MySQL
if (ParserKeyword("SIGNED").ignore(pos))
type_name_suffix = "SIGNED";
else if (ParserKeyword("UNSIGNED").ignore(pos))
type_name_suffix = "UNSIGNED";
if (ParserKeyword(Keyword::SIGNED).ignore(pos))
type_name_suffix = toStringView(Keyword::SIGNED);
else if (ParserKeyword(Keyword::UNSIGNED).ignore(pos))
type_name_suffix = toStringView(Keyword::UNSIGNED);
else if (pos->type == TokenType::OpeningRoundBracket)
{
++pos;
@ -105,10 +105,10 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
if (ParserKeyword("SIGNED").ignore(pos))
type_name_suffix = "SIGNED";
else if (ParserKeyword("UNSIGNED").ignore(pos))
type_name_suffix = "UNSIGNED";
if (ParserKeyword(Keyword::SIGNED).ignore(pos))
type_name_suffix = toStringView(Keyword::SIGNED);
else if (ParserKeyword(Keyword::UNSIGNED).ignore(pos))
type_name_suffix = toStringView(Keyword::UNSIGNED);
}
}

View File

@ -10,7 +10,7 @@ bool ParserDatabaseOrNone::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
auto result = std::make_shared<ASTDatabaseOrNone>();
node = result;
if (ParserKeyword{"NONE"}.ignore(pos, expected))
if (ParserKeyword(Keyword::NONE).ignore(pos, expected))
{
result->none = true;
return true;

View File

@ -13,12 +13,12 @@ bool ParserDeleteQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto query = std::make_shared<ASTDeleteQuery>();
node = query;
ParserKeyword s_delete("DELETE");
ParserKeyword s_from("FROM");
ParserKeyword s_where("WHERE");
ParserKeyword s_delete(Keyword::DELETE);
ParserKeyword s_from(Keyword::FROM);
ParserKeyword s_where(Keyword::WHERE);
ParserExpression parser_exp_elem;
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_on{"ON"};
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_on{Keyword::ON};
if (s_delete.ignore(pos, expected))
{

View File

@ -9,9 +9,9 @@ namespace DB
bool ParserDescribeCacheQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword p_describe("DESCRIBE");
ParserKeyword p_desc("DESC");
ParserKeyword p_cache("FILESYSTEM CACHE");
ParserKeyword p_describe(Keyword::DESCRIBE);
ParserKeyword p_desc(Keyword::DESC);
ParserKeyword p_cache(Keyword::FILESYSTEM_CACHE);
ParserLiteral p_cache_name;
if ((!p_describe.ignore(pos, expected) && !p_desc.ignore(pos, expected))

View File

@ -14,10 +14,10 @@ namespace DB
bool ParserDescribeTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_describe("DESCRIBE");
ParserKeyword s_desc("DESC");
ParserKeyword s_table("TABLE");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_describe(Keyword::DESCRIBE);
ParserKeyword s_desc(Keyword::DESC);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p;
ParserSetQuery parser_settings(true);

View File

@ -165,12 +165,12 @@ bool ParserDictionarySettings::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword primary_key_keyword("PRIMARY KEY");
ParserKeyword source_keyword("SOURCE");
ParserKeyword lifetime_keyword("LIFETIME");
ParserKeyword range_keyword("RANGE");
ParserKeyword layout_keyword("LAYOUT");
ParserKeyword settings_keyword("SETTINGS");
ParserKeyword primary_key_keyword(Keyword::PRIMARY_KEY);
ParserKeyword source_keyword(Keyword::SOURCE);
ParserKeyword lifetime_keyword(Keyword::LIFETIME);
ParserKeyword range_keyword(Keyword::RANGE);
ParserKeyword layout_keyword(Keyword::LAYOUT);
ParserKeyword settings_keyword(Keyword::SETTINGS);
ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket);
ParserFunctionWithKeyValueArguments key_value_pairs_p;

View File

@ -12,12 +12,12 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
{
ParserIdentifier name_parser;
ParserDataType type_parser;
ParserKeyword s_default{"DEFAULT"};
ParserKeyword s_expression{"EXPRESSION"};
ParserKeyword s_hierarchical{"HIERARCHICAL"};
ParserKeyword s_bidirectional{"BIDIRECTIONAL"};
ParserKeyword s_injective{"INJECTIVE"};
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
ParserKeyword s_default{Keyword::DEFAULT};
ParserKeyword s_expression{Keyword::EXPRESSION};
ParserKeyword s_hierarchical{Keyword::HIERARCHICAL};
ParserKeyword s_bidirectional{Keyword::BIDIRECTIONAL};
ParserKeyword s_injective{Keyword::INJECTIVE};
ParserKeyword s_is_object_id{Keyword::IS_OBJECT_ID};
ParserLiteral default_parser;
ParserArrayOfLiterals array_literals_parser;
ParserExpression expression_parser;

View File

@ -9,10 +9,10 @@ namespace DB
bool ParserDropFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_function("FUNCTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_function(Keyword::FUNCTION);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserIdentifier function_name_p;
String cluster_str;

View File

@ -13,10 +13,10 @@ bool ParserDropIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected
auto query = std::make_shared<ASTDropIndexQuery>();
node = query;
ParserKeyword s_drop("DROP");
ParserKeyword s_index("INDEX");
ParserKeyword s_on("ON");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_index(Keyword::INDEX);
ParserKeyword s_on(Keyword::ON);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserIdentifier index_name_p;
String cluster_str;

View File

@ -9,10 +9,10 @@ namespace DB
bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_collection(Keyword::NAMED_COLLECTION);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_on(Keyword::ON);
ParserIdentifier name_p;
String cluster_str;

View File

@ -12,18 +12,18 @@ namespace
bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, const ASTDropQuery::Kind kind)
{
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_view("VIEW");
ParserKeyword s_database("DATABASE");
ParserKeyword s_temporary(Keyword::TEMPORARY);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_dictionary(Keyword::DICTIONARY);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword s_database(Keyword::DATABASE);
ParserToken s_dot(TokenType::Dot);
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_if_empty("IF EMPTY");
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_if_empty(Keyword::IF_EMPTY);
ParserIdentifier name_p(true);
ParserKeyword s_permanently("PERMANENTLY");
ParserKeyword s_no_delay("NO DELAY");
ParserKeyword s_sync("SYNC");
ParserKeyword s_permanently(Keyword::PERMANENTLY);
ParserKeyword s_no_delay(Keyword::NO_DELAY);
ParserKeyword s_sync(Keyword::SYNC);
ASTPtr database;
ASTPtr table;
@ -80,7 +80,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
}
/// common for tables / dictionaries / databases
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -122,9 +122,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, cons
bool ParserDropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_detach("DETACH");
ParserKeyword s_truncate("TRUNCATE");
ParserKeyword s_drop(Keyword::DROP);
ParserKeyword s_detach(Keyword::DETACH);
ParserKeyword s_truncate(Keyword::TRUNCATE);
if (s_drop.ignore(pos, expected))
return parseDropQuery(pos, node, expected, ASTDropQuery::Kind::Drop);

View File

@ -16,15 +16,15 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
{
ASTExplainQuery::ExplainKind kind;
ParserKeyword s_ast("AST");
ParserKeyword s_explain("EXPLAIN");
ParserKeyword s_syntax("SYNTAX");
ParserKeyword s_query_tree("QUERY TREE");
ParserKeyword s_pipeline("PIPELINE");
ParserKeyword s_plan("PLAN");
ParserKeyword s_estimates("ESTIMATE");
ParserKeyword s_table_override("TABLE OVERRIDE");
ParserKeyword s_current_transaction("CURRENT TRANSACTION");
ParserKeyword s_ast(Keyword::AST);
ParserKeyword s_explain(Keyword::EXPLAIN);
ParserKeyword s_syntax(Keyword::SYNTAX);
ParserKeyword s_query_tree(Keyword::QUERY_TREE);
ParserKeyword s_pipeline(Keyword::PIPELINE);
ParserKeyword s_plan(Keyword::PLAN);
ParserKeyword s_estimates(Keyword::ESTIMATE);
ParserKeyword s_table_override(Keyword::TABLE_OVERRIDE);
ParserKeyword s_current_transaction(Keyword::CURRENT_TRANSACTION);
if (s_explain.ignore(pos, expected))
{

View File

@ -27,7 +27,7 @@ namespace ErrorCodes
bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserFunction p_function;
ParserKeyword s_external("EXTERNAL DDL FROM");
ParserKeyword s_external(Keyword::EXTERNAL_DDL_FROM);
ASTPtr from;
auto external_ddl_query = std::make_shared<ASTExternalDDLQuery>();
@ -61,16 +61,16 @@ bool ParserExternalDDLQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expect
{
/// Syntax error is ignored, so we need to convert the error code for parsing failure
if (ParserKeyword("ALTER TABLE").ignore(pos))
if (ParserKeyword(Keyword::ALTER_TABLE).ignore(pos))
throw Exception(ErrorCodes::MYSQL_SYNTAX_ERROR, "Cannot parse MySQL alter query.");
if (ParserKeyword("RENAME TABLE").ignore(pos))
if (ParserKeyword(Keyword::RENAME_TABLE).ignore(pos))
throw Exception(ErrorCodes::MYSQL_SYNTAX_ERROR, "Cannot parse MySQL rename query.");
if (ParserKeyword("DROP TABLE").ignore(pos) || ParserKeyword("TRUNCATE").ignore(pos))
if (ParserKeyword(Keyword::DROP_TABLE).ignore(pos) || ParserKeyword(Keyword::TRUNCATE).ignore(pos))
throw Exception(ErrorCodes::MYSQL_SYNTAX_ERROR, "Cannot parse MySQL drop query.");
if (ParserKeyword("CREATE TABLE").ignore(pos) || ParserKeyword("CREATE TEMPORARY TABLE").ignore(pos))
if (ParserKeyword(Keyword::CREATE_TABLE).ignore(pos) || ParserKeyword(Keyword::CREATE_TEMPORARY_TABLE).ignore(pos))
throw Exception(ErrorCodes::MYSQL_SYNTAX_ERROR, "Cannot parse MySQL create query.");
}
#endif

View File

@ -26,19 +26,19 @@ namespace ErrorCodes
bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
/// Create parsers
ParserKeyword s_insert_into("INSERT INTO");
ParserKeyword s_from_infile("FROM INFILE");
ParserKeyword s_compression("COMPRESSION");
ParserKeyword s_table("TABLE");
ParserKeyword s_function("FUNCTION");
ParserKeyword s_insert_into(Keyword::INSERT_INTO);
ParserKeyword s_from_infile(Keyword::FROM_INFILE);
ParserKeyword s_compression(Keyword::COMPRESSION);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_function(Keyword::FUNCTION);
ParserToken s_dot(TokenType::Dot);
ParserKeyword s_values("VALUES");
ParserKeyword s_format("FORMAT");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_select("SELECT");
ParserKeyword s_watch("WATCH");
ParserKeyword s_partition_by("PARTITION BY");
ParserKeyword s_with("WITH");
ParserKeyword s_values(Keyword::VALUES);
ParserKeyword s_format(Keyword::FORMAT);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_select(Keyword::SELECT);
ParserKeyword s_watch(Keyword::WATCH);
ParserKeyword s_partition_by(Keyword::PARTITION_BY);
ParserKeyword s_with(Keyword::WITH);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserToken s_semicolon(TokenType::Semicolon);

View File

@ -14,16 +14,16 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
String cluster_str;
auto query = std::make_shared<ASTKillQueryQuery>();
ParserKeyword p_kill{"KILL"};
ParserKeyword p_query{"QUERY"};
ParserKeyword p_mutation{"MUTATION"};
ParserKeyword p_part_move_to_shard{"PART_MOVE_TO_SHARD"};
ParserKeyword p_transaction{"TRANSACTION"};
ParserKeyword p_on{"ON"};
ParserKeyword p_test{"TEST"};
ParserKeyword p_sync{"SYNC"};
ParserKeyword p_async{"ASYNC"};
ParserKeyword p_where{"WHERE"};
ParserKeyword p_kill{Keyword::KILL};
ParserKeyword p_query{Keyword::QUERY};
ParserKeyword p_mutation{Keyword::MUTATION};
ParserKeyword p_part_move_to_shard{Keyword::PART_MOVE_TO_SHARD};
ParserKeyword p_transaction{Keyword::TRANSACTION};
ParserKeyword p_on{Keyword::ON};
ParserKeyword p_test{Keyword::TEST};
ParserKeyword p_sync{Keyword::SYNC};
ParserKeyword p_async{Keyword::ASYNC};
ParserKeyword p_where{Keyword::WHERE};
ParserExpression p_where_expression;
if (!p_kill.ignore(pos, expected))

View File

@ -24,12 +24,12 @@ bool ParserOptimizeQueryColumnsSpecification::parseImpl(Pos & pos, ASTPtr & node
bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_optimize_table("OPTIMIZE TABLE");
ParserKeyword s_partition("PARTITION");
ParserKeyword s_final("FINAL");
ParserKeyword s_deduplicate("DEDUPLICATE");
ParserKeyword s_cleanup("CLEANUP");
ParserKeyword s_by("BY");
ParserKeyword s_optimize_table(Keyword::OPTIMIZE_TABLE);
ParserKeyword s_partition(Keyword::PARTITION);
ParserKeyword s_final(Keyword::FINAL);
ParserKeyword s_deduplicate(Keyword::DEDUPLICATE);
ParserKeyword s_cleanup(Keyword::CLEANUP);
ParserKeyword s_by(Keyword::BY);
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p(true);
ParserPartition partition_p;
@ -55,7 +55,7 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
return false;
}
if (ParserKeyword{"ON"}.ignore(pos, expected) && !ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected) && !ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
if (s_partition.ignore(pos, expected))

View File

@ -14,8 +14,8 @@ namespace DB
bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_id("ID");
ParserKeyword s_all("ALL");
ParserKeyword s_id(Keyword::ID);
ParserKeyword s_all(Keyword::ALL);
ParserStringLiteral parser_string_literal;
ParserSubstitution parser_substitution;
ParserExpression parser_expr;

View File

@ -15,10 +15,10 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
auto select_query = std::make_shared<ASTProjectionSelectQuery>();
node = select_query;
ParserKeyword s_with("WITH");
ParserKeyword s_select("SELECT");
ParserKeyword s_group_by("GROUP BY");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_with(Keyword::WITH);
ParserKeyword s_select(Keyword::SELECT);
ParserKeyword s_group_by(Keyword::GROUP_BY);
ParserKeyword s_order_by(Keyword::ORDER_BY);
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.

View File

@ -105,32 +105,32 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
/// FIXME: try to prettify this cast using `as<>()`
auto & query_with_output = dynamic_cast<ASTQueryWithOutput &>(*query);
ParserKeyword s_into_outfile("INTO OUTFILE");
ParserKeyword s_into_outfile(Keyword::INTO_OUTFILE);
if (s_into_outfile.ignore(pos, expected))
{
ParserStringLiteral out_file_p;
if (!out_file_p.parse(pos, query_with_output.out_file, expected))
return false;
ParserKeyword s_append("APPEND");
ParserKeyword s_append(Keyword::APPEND);
if (s_append.ignore(pos, expected))
{
query_with_output.is_outfile_append = true;
}
ParserKeyword s_truncate("TRUNCATE");
ParserKeyword s_truncate(Keyword::TRUNCATE);
if (s_truncate.ignore(pos, expected))
{
query_with_output.is_outfile_truncate = true;
}
ParserKeyword s_stdout("AND STDOUT");
ParserKeyword s_stdout(Keyword::AND_STDOUT);
if (s_stdout.ignore(pos, expected))
{
query_with_output.is_into_outfile_with_stdout = true;
}
ParserKeyword s_compression_method("COMPRESSION");
ParserKeyword s_compression_method(Keyword::COMPRESSION);
if (s_compression_method.ignore(pos, expected))
{
ParserStringLiteral compression;
@ -138,7 +138,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
query_with_output.children.push_back(query_with_output.compression);
ParserKeyword s_compression_level("LEVEL");
ParserKeyword s_compression_level(Keyword::LEVEL);
if (s_compression_level.ignore(pos, expected))
{
ParserNumber compression_level;
@ -152,7 +152,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
ParserKeyword s_format("FORMAT");
ParserKeyword s_format(Keyword::FORMAT);
if (s_format.ignore(pos, expected))
{
@ -166,7 +166,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
// SETTINGS key1 = value1, key2 = value2, ...
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_settings(Keyword::SETTINGS);
if (!query_with_output.settings_ast && s_settings.ignore(pos, expected))
{
ParserSetQuery parser_settings(true);

View File

@ -18,9 +18,16 @@ namespace ErrorCodes
bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_after{Keyword::AFTER};
ParserKeyword s_every{Keyword::EVERY};
ParserKeyword s_offset{Keyword::OFFSET};
ParserKeyword s_randomize_for{Keyword::RANDOMIZE_FOR};
ParserKeyword s_depends_on{Keyword::DEPENDS_ON};
ParserKeyword s_settings{Keyword::SETTINGS};
auto refresh = std::make_shared<ASTRefreshStrategy>();
if (ParserKeyword{"AFTER"}.ignore(pos, expected))
if (s_after.ignore(pos, expected))
{
refresh->schedule_kind = RefreshScheduleKind::AFTER;
ASTPtr period;
@ -29,14 +36,14 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
refresh->set(refresh->period, period);
}
else if (ParserKeyword{"EVERY"}.ignore(pos, expected))
else if (s_every.ignore(pos, expected))
{
refresh->schedule_kind = RefreshScheduleKind::EVERY;
ASTPtr period;
if (!ParserTimeInterval{{.allow_mixing_calendar_and_clock_units = false}}.parse(pos, period, expected))
return false;
refresh->set(refresh->period, period);
if (ParserKeyword{"OFFSET"}.ignore(pos, expected))
if (s_offset.ignore(pos, expected))
{
ASTPtr periodic_offset;
if (!ParserTimeInterval{{.allow_zero = true}}.parse(pos, periodic_offset, expected))
@ -53,7 +60,7 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (refresh->schedule_kind == RefreshScheduleKind::UNKNOWN)
return false;
if (ParserKeyword{"RANDOMIZE FOR"}.ignore(pos, expected))
if (s_randomize_for.ignore(pos, expected))
{
ASTPtr spread;
if (!ParserTimeInterval{{.allow_zero = true}}.parse(pos, spread, expected))
@ -62,7 +69,7 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
refresh->set(refresh->spread, spread);
}
if (ParserKeyword{"DEPENDS ON"}.ignore(pos, expected))
if (s_depends_on.ignore(pos, expected))
{
if (refresh->schedule_kind == RefreshScheduleKind::AFTER)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
@ -81,7 +88,7 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
// Refresh SETTINGS
if (ParserKeyword{"SETTINGS"}.ignore(pos, expected))
if (s_settings.ignore(pos, expected))
{
/// Settings are written like SET query, so parse them with ParserSetQuery
ASTPtr settings;

View File

@ -11,15 +11,15 @@ namespace DB
bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_rename("RENAME");
ParserKeyword s_rename_table("RENAME TABLE");
ParserKeyword s_exchange_tables("EXCHANGE TABLES");
ParserKeyword s_rename_dictionary("RENAME DICTIONARY");
ParserKeyword s_exchange_dictionaries("EXCHANGE DICTIONARIES");
ParserKeyword s_rename_database("RENAME DATABASE");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_to("TO");
ParserKeyword s_and("AND");
ParserKeyword s_rename(Keyword::RENAME);
ParserKeyword s_rename_table(Keyword::RENAME_TABLE);
ParserKeyword s_exchange_tables(Keyword::EXCHANGE_TABLES);
ParserKeyword s_rename_dictionary(Keyword::RENAME_DICTIONARY);
ParserKeyword s_exchange_dictionaries(Keyword::EXCHANGE_DICTIONARIES);
ParserKeyword s_rename_database(Keyword::RENAME_DATABASE);
ParserKeyword s_if_exists(Keyword::IF_EXISTS);
ParserKeyword s_to(Keyword::TO);
ParserKeyword s_and(Keyword::AND);
ParserToken s_comma(TokenType::Comma);
bool exchange = false;
@ -39,7 +39,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -94,7 +94,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;

View File

@ -37,35 +37,35 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
auto select_query = std::make_shared<ASTSelectQuery>();
node = select_query;
ParserKeyword s_select("SELECT");
ParserKeyword s_all("ALL");
ParserKeyword s_distinct("DISTINCT");
ParserKeyword s_distinct_on("DISTINCT ON");
ParserKeyword s_from("FROM");
ParserKeyword s_prewhere("PREWHERE");
ParserKeyword s_where("WHERE");
ParserKeyword s_group_by("GROUP BY");
ParserKeyword s_with("WITH");
ParserKeyword s_totals("TOTALS");
ParserKeyword s_having("HAVING");
ParserKeyword s_window("WINDOW");
ParserKeyword s_order_by("ORDER BY");
ParserKeyword s_limit("LIMIT");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_by("BY");
ParserKeyword s_rollup("ROLLUP");
ParserKeyword s_cube("CUBE");
ParserKeyword s_grouping_sets("GROUPING SETS");
ParserKeyword s_top("TOP");
ParserKeyword s_with_ties("WITH TIES");
ParserKeyword s_offset("OFFSET");
ParserKeyword s_fetch("FETCH");
ParserKeyword s_only("ONLY");
ParserKeyword s_row("ROW");
ParserKeyword s_rows("ROWS");
ParserKeyword s_first("FIRST");
ParserKeyword s_next("NEXT");
ParserKeyword s_interpolate("INTERPOLATE");
ParserKeyword s_select(Keyword::SELECT);
ParserKeyword s_all(Keyword::ALL);
ParserKeyword s_distinct(Keyword::DISTINCT);
ParserKeyword s_distinct_on(Keyword::DISTINCT_ON);
ParserKeyword s_from(Keyword::FROM);
ParserKeyword s_prewhere(Keyword::PREWHERE);
ParserKeyword s_where(Keyword::WHERE);
ParserKeyword s_group_by(Keyword::GROUP_BY);
ParserKeyword s_with(Keyword::WITH);
ParserKeyword s_totals(Keyword::TOTALS);
ParserKeyword s_having(Keyword::HAVING);
ParserKeyword s_window(Keyword::WINDOW);
ParserKeyword s_order_by(Keyword::ORDER_BY);
ParserKeyword s_limit(Keyword::LIMIT);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_by(Keyword::BY);
ParserKeyword s_rollup(Keyword::ROLLUP);
ParserKeyword s_cube(Keyword::CUBE);
ParserKeyword s_grouping_sets(Keyword::GROUPING_SETS);
ParserKeyword s_top(Keyword::TOP);
ParserKeyword s_with_ties(Keyword::WITH_TIES);
ParserKeyword s_offset(Keyword::OFFSET);
ParserKeyword s_fetch(Keyword::FETCH);
ParserKeyword s_only(Keyword::ONLY);
ParserKeyword s_row(Keyword::ROW);
ParserKeyword s_rows(Keyword::ROWS);
ParserKeyword s_first(Keyword::FIRST);
ParserKeyword s_next(Keyword::NEXT);
ParserKeyword s_interpolate(Keyword::INTERPOLATE);
ParserNotEmptyExpressionList exp_list(false);
ParserNotEmptyExpressionList exp_list_for_with_clause(false);

View File

@ -210,9 +210,9 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p
if (!s_eq.ignore(pos, expected))
return false;
if (ParserKeyword("TRUE").ignore(pos, expected))
if (ParserKeyword(Keyword::TRUE_KEYWORD).ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1)));
else if (ParserKeyword("FALSE").ignore(pos, expected))
else if (ParserKeyword(Keyword::FALSE_KEYWORD).ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0)));
/// for SETTINGS disk=disk(type='s3', path='', ...)
else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
@ -269,16 +269,16 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault(
}
/// Default
if (ParserKeyword("DEFAULT").ignore(pos, expected))
if (ParserKeyword(Keyword::DEFAULT).ignore(pos, expected))
{
default_settings = name;
return true;
}
/// Setting
if (ParserKeyword("TRUE").ignore(pos, expected))
if (ParserKeyword(Keyword::TRUE_KEYWORD).ignore(pos, expected))
node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1)));
else if (ParserKeyword("FALSE").ignore(pos, expected))
else if (ParserKeyword(Keyword::FALSE_KEYWORD).ignore(pos, expected))
node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0)));
else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
{
@ -303,13 +303,13 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!parse_only_internals)
{
ParserKeyword s_set("SET");
ParserKeyword s_set(Keyword::SET);
if (!s_set.ignore(pos, expected))
return false;
/// Parse SET TRANSACTION ... queries using ParserTransactionControl
if (ParserKeyword{"TRANSACTION"}.check(pos, expected))
if (ParserKeyword{Keyword::TRANSACTION}.check(pos, expected))
return false;
}

View File

@ -23,19 +23,19 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto query = std::make_shared<ASTShowColumnsQuery>();
if (!ParserKeyword("SHOW").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW).ignore(pos, expected))
return false;
if (ParserKeyword("EXTENDED").ignore(pos, expected))
if (ParserKeyword(Keyword::EXTENDED).ignore(pos, expected))
query->extended = true;
if (ParserKeyword("FULL").ignore(pos, expected))
if (ParserKeyword(Keyword::FULL).ignore(pos, expected))
query->full = true;
if (!(ParserKeyword("COLUMNS").ignore(pos, expected) || ParserKeyword("FIELDS").ignore(pos, expected)))
if (!(ParserKeyword(Keyword::COLUMNS).ignore(pos, expected) || ParserKeyword(Keyword::FIELDS).ignore(pos, expected)))
return false;
if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected))
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
{
if (!ParserCompoundIdentifier().parse(pos, from1, expected))
return false;
@ -55,7 +55,7 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
else
{
if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected))
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
if (!ParserIdentifier().parse(pos, from2, expected))
return false;
@ -65,10 +65,10 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
query->database = from2_str;
}
if (ParserKeyword("NOT").ignore(pos, expected))
if (ParserKeyword(Keyword::NOT).ignore(pos, expected))
query->not_like = true;
if (bool insensitive = ParserKeyword("ILIKE").ignore(pos, expected); insensitive || ParserKeyword("LIKE").ignore(pos, expected))
if (bool insensitive = ParserKeyword(Keyword::ILIKE).ignore(pos, expected); insensitive || ParserKeyword(Keyword::LIKE).ignore(pos, expected))
{
if (insensitive)
query->case_insensitive_like = true;
@ -78,11 +78,11 @@ bool ParserShowColumnsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
else if (query->not_like)
return false;
else if (ParserKeyword("WHERE").ignore(pos, expected))
else if (ParserKeyword(Keyword::WHERE).ignore(pos, expected))
if (!ParserExpressionWithOptionalAlias(false).parse(pos, query->where_expression, expected))
return false;
if (ParserKeyword("LIMIT").ignore(pos, expected))
if (ParserKeyword(Keyword::LIMIT).ignore(pos, expected))
if (!ParserExpressionWithOptionalAlias(false).parse(pos, query->limit_length, expected))
return false;

View File

@ -20,7 +20,7 @@ protected:
{
auto query = std::make_shared<ASTShowEnginesQuery>();
if (!ParserKeyword("SHOW ENGINES").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_ENGINES).ignore(pos, expected))
return false;
node = query;

View File

@ -13,10 +13,10 @@ bool ParserShowFunctionsQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
ASTPtr like;
auto query = std::make_shared<ASTShowFunctionsQuery>();
if (!ParserKeyword("SHOW FUNCTIONS").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_FUNCTIONS).ignore(pos, expected))
return false;
if (bool insensitive = ParserKeyword("ILIKE").ignore(pos, expected); insensitive || ParserKeyword("LIKE").ignore(pos, expected))
if (bool insensitive = ParserKeyword(Keyword::ILIKE).ignore(pos, expected); insensitive || ParserKeyword(Keyword::LIKE).ignore(pos, expected))
{
if (insensitive)
query->case_insensitive_like = true;

View File

@ -22,16 +22,16 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
auto query = std::make_shared<ASTShowIndexesQuery>();
if (!ParserKeyword("SHOW").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW).ignore(pos, expected))
return false;
if (ParserKeyword("EXTENDED").ignore(pos, expected))
if (ParserKeyword(Keyword::EXTENDED).ignore(pos, expected))
query->extended = true;
if (!(ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("INDEXES").ignore(pos, expected) || ParserKeyword("INDICES").ignore(pos, expected) || ParserKeyword("KEYS").ignore(pos, expected)))
if (!(ParserKeyword(Keyword::INDEX).ignore(pos, expected) || ParserKeyword(Keyword::INDEXES).ignore(pos, expected) || ParserKeyword(Keyword::INDICES).ignore(pos, expected) || ParserKeyword(Keyword::KEYS).ignore(pos, expected)))
return false;
if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected))
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
{
if (!ParserCompoundIdentifier().parse(pos, from1, expected))
return false;
@ -51,7 +51,7 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
else
{
if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected))
if (ParserKeyword(Keyword::FROM).ignore(pos, expected) || ParserKeyword(Keyword::IN).ignore(pos, expected))
if (!ParserIdentifier().parse(pos, from2, expected))
return false;
@ -61,7 +61,7 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
query->database = from2_str;
}
if (ParserKeyword("WHERE").ignore(pos, expected))
if (ParserKeyword(Keyword::WHERE).ignore(pos, expected))
if (!ParserExpressionWithOptionalAlias(false).parse(pos, query->where_expression, expected))
return false;

View File

@ -20,7 +20,7 @@ protected:
{
auto query = std::make_shared<ASTShowProcesslistQuery>();
if (!ParserKeyword("SHOW PROCESSLIST").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_PROCESSLIST).ignore(pos, expected))
return false;
node = query;

View File

@ -11,7 +11,7 @@ namespace DB
bool ParserShowSettingQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword("SHOW SETTING").ignore(pos, expected))
if (!ParserKeyword(Keyword::SHOW_SETTING).ignore(pos, expected))
return false;
ASTPtr setting_name_identifier;

View File

@ -17,25 +17,25 @@ namespace DB
bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_show("SHOW");
ParserKeyword s_full("FULL");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_tables("TABLES");
ParserKeyword s_databases("DATABASES");
ParserKeyword s_clusters("CLUSTERS");
ParserKeyword s_cluster("CLUSTER");
ParserKeyword s_dictionaries("DICTIONARIES");
ParserKeyword s_caches("FILESYSTEM CACHES");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_merges("MERGES");
ParserKeyword s_changed("CHANGED");
ParserKeyword s_from("FROM");
ParserKeyword s_in("IN");
ParserKeyword s_not("NOT");
ParserKeyword s_like("LIKE");
ParserKeyword s_ilike("ILIKE");
ParserKeyword s_where("WHERE");
ParserKeyword s_limit("LIMIT");
ParserKeyword s_show(Keyword::SHOW);
ParserKeyword s_full(Keyword::FULL);
ParserKeyword s_temporary(Keyword::TEMPORARY);
ParserKeyword s_tables(Keyword::TABLES);
ParserKeyword s_databases(Keyword::DATABASES);
ParserKeyword s_clusters(Keyword::CLUSTERS);
ParserKeyword s_cluster(Keyword::CLUSTER);
ParserKeyword s_dictionaries(Keyword::DICTIONARIES);
ParserKeyword s_caches(Keyword::FILESYSTEM_CACHES);
ParserKeyword s_settings(Keyword::SETTINGS);
ParserKeyword s_merges(Keyword::MERGES);
ParserKeyword s_changed(Keyword::CHANGED);
ParserKeyword s_from(Keyword::FROM);
ParserKeyword s_in(Keyword::IN);
ParserKeyword s_not(Keyword::NOT);
ParserKeyword s_like(Keyword::LIKE);
ParserKeyword s_ilike(Keyword::ILIKE);
ParserKeyword s_where(Keyword::WHERE);
ParserKeyword s_limit(Keyword::LIMIT);
ParserStringLiteral like_p;
ParserIdentifier name_p(true);
ParserExpressionWithOptionalAlias exp_elem(false);
@ -147,7 +147,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
/// Not expected due to "SHOW SETTINGS PROFILES"
/// Not expected due to Keyword::SHOW SETTINGS PROFILES
if (bool insensitive = s_ilike.ignore(pos, expected); insensitive || s_like.ignore(pos, expected))
{
if (insensitive)

View File

@ -28,7 +28,7 @@ namespace ErrorCodes
String cluster;
bool parsed_on_cluster = false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
@ -52,7 +52,7 @@ namespace ErrorCodes
if (!parsed_table && require_table)
return false;
if (!parsed_on_cluster && ParserKeyword{"ON"}.ignore(pos, expected))
if (!parsed_on_cluster && ParserKeyword{Keyword::ON}.ignore(pos, expected))
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
@ -82,7 +82,7 @@ enum class SystemQueryTargetType
String cluster;
bool parsed_on_cluster = false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
@ -108,7 +108,7 @@ enum class SystemQueryTargetType
return false;
}
if (!parsed_on_cluster && ParserKeyword{"ON"}.ignore(pos, expected))
if (!parsed_on_cluster && ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
@ -142,7 +142,7 @@ enum class SystemQueryTargetType
Expected & expected)
{
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -162,29 +162,29 @@ enum class SystemQueryTargetType
return false;
res->replica = ast->as<ASTLiteral &>().value.safeGet<String>();
if (ParserKeyword{"FROM SHARD"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FROM_SHARD}.ignore(pos, expected))
{
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
res->shard = ast->as<ASTLiteral &>().value.safeGet<String>();
}
if (ParserKeyword{"FROM"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FROM}.ignore(pos, expected))
{
// way 1. parse replica database
// way 2. parse replica table
// way 3. parse replica zkpath
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
if (ParserKeyword{Keyword::DATABASE}.ignore(pos, expected))
{
ParserIdentifier database_parser;
if (!database_parser.parse(pos, res->database, expected))
return false;
}
else if (!database && ParserKeyword{"TABLE"}.ignore(pos, expected))
else if (!database && ParserKeyword{Keyword::TABLE}.ignore(pos, expected))
{
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
}
else if (ParserKeyword{"ZKPATH"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::ZKPATH}.ignore(pos, expected))
{
ASTPtr path_ast;
if (!ParserStringLiteral{}.parse(pos, path_ast, expected))
@ -205,7 +205,7 @@ enum class SystemQueryTargetType
bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SYSTEM"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SYSTEM}.ignore(pos, expected))
return false;
using Type = ASTSystemQuery::Type;
@ -216,7 +216,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
for (const auto & type : magic_enum::enum_values<Type>())
{
if (ParserKeyword{ASTSystemQuery::typeToString(type)}.ignore(pos, expected))
if (ParserKeyword::createDeprecated(ASTSystemQuery::typeToString(type)).ignore(pos, expected))
{
res->type = type;
found = true;
@ -281,12 +281,12 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
return false;
if (res->type == Type::SYNC_REPLICA)
{
if (ParserKeyword{"STRICT"}.ignore(pos, expected))
if (ParserKeyword{Keyword::STRICT}.ignore(pos, expected))
res->sync_replica_mode = SyncReplicaMode::STRICT;
else if (ParserKeyword{"LIGHTWEIGHT"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::LIGHTWEIGHT}.ignore(pos, expected))
{
res->sync_replica_mode = SyncReplicaMode::LIGHTWEIGHT;
if (ParserKeyword{"FROM"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FROM}.ignore(pos, expected))
{
do
{
@ -297,7 +297,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
} while (ParserToken{TokenType::Comma}.ignore(pos, expected));
}
}
else if (ParserKeyword{"PULL"}.ignore(pos, expected))
else if (ParserKeyword{Keyword::PULL}.ignore(pos, expected))
res->sync_replica_mode = SyncReplicaMode::PULL;
}
break;
@ -360,7 +360,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
return true;
};
if (ParserKeyword{"ON VOLUME"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON_VOLUME}.ignore(pos, expected))
{
if (!parse_on_volume())
return false;
@ -369,7 +369,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
if (ParserKeyword{"ON VOLUME"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON_VOLUME}.ignore(pos, expected))
{
if (!parse_on_volume())
return false;
@ -419,10 +419,10 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table))
return false;
if (ParserKeyword{"UNSET FAKE TIME"}.ignore(pos, expected))
if (ParserKeyword{Keyword::UNSET_FAKE_TIME}.ignore(pos, expected))
break;
if (!ParserKeyword{"SET FAKE TIME"}.ignore(pos, expected))
if (!ParserKeyword{Keyword::SET_FAKE_TIME}.ignore(pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
@ -442,9 +442,9 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
return false;
ASTPtr seconds;
if (!(ParserKeyword{"FOR"}.ignore(pos, expected)
if (!(ParserKeyword{Keyword::FOR}.ignore(pos, expected)
&& ParserUnsignedInteger().parse(pos, seconds, expected)
&& ParserKeyword{"SECOND"}.ignore(pos, expected))) /// SECOND, not SECONDS to be consistent with INTERVAL parsing in SQL
&& ParserKeyword{Keyword::SECOND}.ignore(pos, expected))) /// SECOND, not SECONDS to be consistent with INTERVAL parsing in SQL
{
return false;
}
@ -459,10 +459,10 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
if (path_parser.parse(pos, ast, expected))
{
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
if (ParserKeyword{"KEY"}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected))
if (ParserKeyword{Keyword::KEY}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected))
{
res->key_to_drop = ast->as<ASTIdentifier>()->name();
if (ParserKeyword{"OFFSET"}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected))
if (ParserKeyword{Keyword::OFFSET}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected))
res->offset_to_drop = ast->as<ASTLiteral>()->value.safeGet<UInt64>();
}
}
@ -486,18 +486,18 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
case Type::DROP_SCHEMA_CACHE:
{
if (ParserKeyword{"FOR"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FOR}.ignore(pos, expected))
{
if (ParserKeyword{"FILE"}.ignore(pos, expected))
res->schema_cache_storage = "FILE";
else if (ParserKeyword{"S3"}.ignore(pos, expected))
res->schema_cache_storage = "S3";
else if (ParserKeyword{"HDFS"}.ignore(pos, expected))
res->schema_cache_storage = "HDFS";
else if (ParserKeyword{"URL"}.ignore(pos, expected))
res->schema_cache_storage = "URL";
else if (ParserKeyword{"AZURE"}.ignore(pos, expected))
res->schema_cache_storage = "AZURE";
if (ParserKeyword{Keyword::FILE}.ignore(pos, expected))
res->schema_cache_storage = toStringView(Keyword::FILE);
else if (ParserKeyword{Keyword::S3}.ignore(pos, expected))
res->schema_cache_storage = toStringView(Keyword::S3);
else if (ParserKeyword{Keyword::HDFS}.ignore(pos, expected))
res->schema_cache_storage = toStringView(Keyword::HDFS);
else if (ParserKeyword{Keyword::URL}.ignore(pos, expected))
res->schema_cache_storage = toStringView(Keyword::URL);
else if (ParserKeyword{Keyword::AZURE}.ignore(pos, expected))
res->schema_cache_storage = toStringView(Keyword::AZURE);
else
return false;
}
@ -505,10 +505,11 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
case Type::DROP_FORMAT_SCHEMA_CACHE:
{
if (ParserKeyword{"FOR"}.ignore(pos, expected))
if (ParserKeyword{Keyword::FOR}.ignore(pos, expected))
{
if (ParserKeyword{"Protobuf"}.ignore(pos, expected))
res->schema_cache_format = "Protobuf";
if (ParserKeyword{Keyword::PROTOBUF}.ignore(pos, expected))
res->schema_cache_format = toStringView(Keyword::PROTOBUF);
else
return false;
}
@ -517,7 +518,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::UNFREEZE:
{
ASTPtr ast;
if (ParserKeyword{"WITH NAME"}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected))
if (ParserKeyword{Keyword::WITH_NAME}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected))
{
res->backup_name = ast->as<ASTLiteral &>().value.get<const String &>();
}
@ -541,7 +542,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
for (const auto & cur_type : magic_enum::enum_values<ServerType::Type>())
{
if (ParserKeyword{ServerType::serverTypeToString(cur_type)}.ignore(pos, expected))
if (ParserKeyword::createDeprecated(ServerType::serverTypeToString(cur_type)).ignore(pos, expected))
{
type = cur_type;
break;
@ -573,7 +574,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
if (!parse_server_type(base_type, base_custom_name))
return false;
if (ParserKeyword{"EXCEPT"}.ignore(pos, expected))
if (ParserKeyword{Keyword::EXCEPT}.ignore(pos, expected))
{
if (base_type != ServerType::Type::QUERIES_ALL &&
base_type != ServerType::Type::QUERIES_DEFAULT &&

View File

@ -12,14 +12,14 @@ namespace DB
bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_exists("EXISTS");
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_show("SHOW");
ParserKeyword s_create("CREATE");
ParserKeyword s_database("DATABASE");
ParserKeyword s_table("TABLE");
ParserKeyword s_view("VIEW");
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_exists(Keyword::EXISTS);
ParserKeyword s_temporary(Keyword::TEMPORARY);
ParserKeyword s_show(Keyword::SHOW);
ParserKeyword s_create(Keyword::CREATE);
ParserKeyword s_database(Keyword::DATABASE);
ParserKeyword s_table(Keyword::TABLE);
ParserKeyword s_view(Keyword::VIEW);
ParserKeyword s_dictionary(Keyword::DICTIONARY);
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p(true);

View File

@ -31,11 +31,11 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
/// FINAL
if (ParserKeyword("FINAL").ignore(pos, expected))
if (ParserKeyword(Keyword::FINAL).ignore(pos, expected))
res->final = true;
/// SAMPLE number
if (ParserKeyword("SAMPLE").ignore(pos, expected))
if (ParserKeyword(Keyword::SAMPLE).ignore(pos, expected))
{
ParserSampleRatio ratio;
@ -43,7 +43,7 @@ bool ParserTableExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
/// OFFSET number
if (ParserKeyword("OFFSET").ignore(pos, expected))
if (ParserKeyword(Keyword::OFFSET).ignore(pos, expected))
{
if (!ratio.parse(pos, res->sample_offset, expected))
return false;
@ -76,7 +76,7 @@ bool ParserArrayJoin::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
Pos saved_pos = pos;
bool has_array_join = false;
if (ParserKeyword("LEFT ARRAY JOIN").ignore(pos, expected))
if (ParserKeyword(Keyword::LEFT_ARRAY_JOIN).ignore(pos, expected))
{
res->kind = ASTArrayJoin::Kind::Left;
has_array_join = true;
@ -86,9 +86,9 @@ bool ParserArrayJoin::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
pos = saved_pos;
/// INNER may be specified explicitly, otherwise it is assumed as default.
ParserKeyword("INNER").ignore(pos, expected);
ParserKeyword(Keyword::INNER).ignore(pos, expected);
if (ParserKeyword("ARRAY JOIN").ignore(pos, expected))
if (ParserKeyword(Keyword::ARRAY_JOIN).ignore(pos, expected))
{
res->kind = ASTArrayJoin::Kind::Inner;
has_array_join = true;
@ -111,15 +111,15 @@ bool ParserArrayJoin::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
void ParserTablesInSelectQueryElement::parseJoinStrictness(Pos & pos, ASTTableJoin & table_join)
{
if (ParserKeyword("ANY").ignore(pos))
if (ParserKeyword(Keyword::ANY).ignore(pos))
table_join.strictness = JoinStrictness::Any;
else if (ParserKeyword("ALL").ignore(pos))
else if (ParserKeyword(Keyword::ALL).ignore(pos))
table_join.strictness = JoinStrictness::All;
else if (ParserKeyword("ASOF").ignore(pos))
else if (ParserKeyword(Keyword::ASOF).ignore(pos))
table_join.strictness = JoinStrictness::Asof;
else if (ParserKeyword("SEMI").ignore(pos))
else if (ParserKeyword(Keyword::SEMI).ignore(pos))
table_join.strictness = JoinStrictness::Semi;
else if (ParserKeyword("ANTI").ignore(pos) || ParserKeyword("ONLY").ignore(pos))
else if (ParserKeyword(Keyword::ANTI).ignore(pos) || ParserKeyword(Keyword::ONLY).ignore(pos))
table_join.strictness = JoinStrictness::Anti;
}
@ -146,9 +146,9 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
}
else
{
if (ParserKeyword("GLOBAL").ignore(pos))
if (ParserKeyword(Keyword::GLOBAL).ignore(pos))
table_join->locality = JoinLocality::Global;
else if (ParserKeyword("LOCAL").ignore(pos))
else if (ParserKeyword(Keyword::LOCAL).ignore(pos))
table_join->locality = JoinLocality::Local;
table_join->strictness = JoinStrictness::Unspecified;
@ -157,17 +157,17 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
parseJoinStrictness(pos, *table_join);
bool no_kind = false;
if (ParserKeyword("INNER").ignore(pos))
if (ParserKeyword(Keyword::INNER).ignore(pos))
table_join->kind = JoinKind::Inner;
else if (ParserKeyword("LEFT").ignore(pos))
else if (ParserKeyword(Keyword::LEFT).ignore(pos))
table_join->kind = JoinKind::Left;
else if (ParserKeyword("RIGHT").ignore(pos))
else if (ParserKeyword(Keyword::RIGHT).ignore(pos))
table_join->kind = JoinKind::Right;
else if (ParserKeyword("FULL").ignore(pos))
else if (ParserKeyword(Keyword::FULL).ignore(pos))
table_join->kind = JoinKind::Full;
else if (ParserKeyword("CROSS").ignore(pos))
else if (ParserKeyword(Keyword::CROSS).ignore(pos))
table_join->kind = JoinKind::Cross;
else if (ParserKeyword("PASTE").ignore(pos))
else if (ParserKeyword(Keyword::PASTE).ignore(pos))
table_join->kind = JoinKind::Paste;
else
no_kind = true;
@ -180,7 +180,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
|| table_join->kind == JoinKind::Right
|| table_join->kind == JoinKind::Full)
{
ParserKeyword("OUTER").ignore(pos);
ParserKeyword(Keyword::OUTER).ignore(pos);
}
if (no_kind)
@ -201,7 +201,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
(table_join->kind != JoinKind::Left && table_join->kind != JoinKind::Right))
throw Exception(ErrorCodes::SYNTAX_ERROR, "SEMI|ANTI JOIN should be LEFT or RIGHT.");
if (!ParserKeyword("JOIN").ignore(pos, expected))
if (!ParserKeyword(Keyword::JOIN).ignore(pos, expected))
return false;
}
@ -211,7 +211,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
if (table_join->kind != JoinKind::Comma
&& table_join->kind != JoinKind::Cross && table_join->kind != JoinKind::Paste)
{
if (ParserKeyword("USING").ignore(pos, expected))
if (ParserKeyword(Keyword::USING).ignore(pos, expected))
{
/// Expression for USING could be in parentheses or not.
bool in_parens = pos->type == TokenType::OpeningRoundBracket;
@ -228,7 +228,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
++pos;
}
}
else if (ParserKeyword("ON").ignore(pos, expected))
else if (ParserKeyword(Keyword::ON).ignore(pos, expected))
{
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
return false;

View File

@ -12,13 +12,13 @@ bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
ASTTransactionControl::QueryType action;
UInt64 snapshot = 0;
if (ParserKeyword("BEGIN TRANSACTION").ignore(pos, expected))
if (ParserKeyword(Keyword::BEGIN_TRANSACTION).ignore(pos, expected))
action = ASTTransactionControl::BEGIN;
else if (ParserKeyword("COMMIT").ignore(pos, expected))
else if (ParserKeyword(Keyword::COMMIT).ignore(pos, expected))
action = ASTTransactionControl::COMMIT;
else if (ParserKeyword("ROLLBACK").ignore(pos, expected))
else if (ParserKeyword(Keyword::ROLLBACK).ignore(pos, expected))
action = ASTTransactionControl::ROLLBACK;
else if (ParserKeyword("SET TRANSACTION SNAPSHOT").ignore(pos, expected))
else if (ParserKeyword(Keyword::SET_TRANSACTION_SNAPSHOT).ignore(pos, expected))
{
action = ASTTransactionControl::SET_SNAPSHOT;
ASTPtr ast;

View File

@ -13,7 +13,7 @@ namespace
bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_table("TABLE");
ParserKeyword s_table(Keyword::TABLE);
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p(true);
@ -35,7 +35,7 @@ bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
if (!name_p.parse(pos, table, expected))
return false;
}
if (ParserKeyword("UUID").ignore(pos, expected))
if (ParserKeyword(Keyword::UUID).ignore(pos, expected))
{
ParserStringLiteral uuid_p;
ASTPtr ast_uuid;
@ -43,7 +43,7 @@ bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
return false;
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
}
if (ParserKeyword{"ON"}.ignore(pos, expected))
if (ParserKeyword{Keyword::ON}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
@ -70,7 +70,7 @@ bool parseUndropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
bool ParserUndropQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_undrop("UNDROP");
ParserKeyword s_undrop(Keyword::UNDROP);
if (s_undrop.ignore(pos, expected))
return parseUndropQuery(pos, node, expected);

View File

@ -10,7 +10,7 @@ namespace DB
bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_use("USE");
ParserKeyword s_use(Keyword::USE);
ParserIdentifier name_p{/*allow_query_parameter*/ true};
if (!s_use.ignore(pos, expected))

View File

@ -20,11 +20,11 @@ namespace DB
bool ParserWatchQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_watch("WATCH");
ParserKeyword s_watch(Keyword::WATCH);
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p(true);
ParserKeyword s_events("EVENTS");
ParserKeyword s_limit("LIMIT");
ParserKeyword s_events(Keyword::EVENTS);
ParserKeyword s_limit(Keyword::LIMIT);
ASTPtr database;
ASTPtr table;

View File

@ -13,7 +13,7 @@ namespace DB
bool ParserWithElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier s_ident;
ParserKeyword s_as("AS");
ParserKeyword s_as(Keyword::AS);
ParserSubquery s_subquery;
auto old_pos = pos;

View File

@ -7,89 +7,88 @@ namespace DB
{
bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result)
{
if (ParserKeyword("NANOSECOND").ignore(pos, expected) || ParserKeyword("NANOSECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_NANOSECOND").ignore(pos, expected)
|| ParserKeyword("NS").ignore(pos, expected))
if (ParserKeyword(Keyword::NANOSECOND).ignore(pos, expected) || ParserKeyword(Keyword::NANOSECONDS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_NANOSECOND).ignore(pos, expected)
|| ParserKeyword(Keyword::NS).ignore(pos, expected))
{
result = IntervalKind::Kind::Nanosecond;
return true;
}
if (ParserKeyword("MICROSECOND").ignore(pos, expected) || ParserKeyword("MICROSECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MICROSECOND").ignore(pos, expected)
|| ParserKeyword("MCS").ignore(pos, expected))
if (ParserKeyword(Keyword::MICROSECOND).ignore(pos, expected) || ParserKeyword(Keyword::MICROSECONDS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_MICROSECOND).ignore(pos, expected))
{
result = IntervalKind::Kind::Microsecond;
return true;
}
if (ParserKeyword("MILLISECOND").ignore(pos, expected) || ParserKeyword("MILLISECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MILLISECOND").ignore(pos, expected)
|| ParserKeyword("MS").ignore(pos, expected))
if (ParserKeyword(Keyword::MILLISECOND).ignore(pos, expected) || ParserKeyword(Keyword::MILLISECONDS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_MILLISECOND).ignore(pos, expected)
|| ParserKeyword(Keyword::MS).ignore(pos, expected))
{
result = IntervalKind::Kind::Millisecond;
return true;
}
if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SECONDS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected)
|| ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected))
if (ParserKeyword(Keyword::SECOND).ignore(pos, expected) || ParserKeyword(Keyword::SECONDS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_SECOND).ignore(pos, expected)
|| ParserKeyword(Keyword::SS).ignore(pos, expected) || ParserKeyword(Keyword::S).ignore(pos, expected))
{
result = IntervalKind::Kind::Second;
return true;
}
if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("MINUTES").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected)
|| ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected))
if (ParserKeyword(Keyword::MINUTE).ignore(pos, expected) || ParserKeyword(Keyword::MINUTES).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_MINUTE).ignore(pos, expected)
|| ParserKeyword(Keyword::MI).ignore(pos, expected) || ParserKeyword(Keyword::N).ignore(pos, expected))
{
result = IntervalKind::Kind::Minute;
return true;
}
if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("HOURS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected)
|| ParserKeyword("HH").ignore(pos, expected) || ParserKeyword("H").ignore(pos, expected))
if (ParserKeyword(Keyword::HOUR).ignore(pos, expected) || ParserKeyword(Keyword::HOURS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_HOUR).ignore(pos, expected)
|| ParserKeyword(Keyword::HH).ignore(pos, expected) || ParserKeyword(Keyword::H).ignore(pos, expected))
{
result = IntervalKind::Kind::Hour;
return true;
}
if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("DAYS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_DAY").ignore(pos, expected)
|| ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected))
if (ParserKeyword(Keyword::DAY).ignore(pos, expected) || ParserKeyword(Keyword::DAYS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_DAY).ignore(pos, expected)
|| ParserKeyword(Keyword::DD).ignore(pos, expected) || ParserKeyword(Keyword::D).ignore(pos, expected))
{
result = IntervalKind::Kind::Day;
return true;
}
if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("WEEKS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected)
|| ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected))
if (ParserKeyword(Keyword::WEEK).ignore(pos, expected) || ParserKeyword(Keyword::WEEKS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_WEEK).ignore(pos, expected)
|| ParserKeyword(Keyword::WK).ignore(pos, expected) || ParserKeyword(Keyword::WW).ignore(pos, expected))
{
result = IntervalKind::Kind::Week;
return true;
}
if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("MONTHS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected)
|| ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected))
if (ParserKeyword(Keyword::MONTH).ignore(pos, expected) || ParserKeyword(Keyword::MONTHS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_MONTH).ignore(pos, expected)
|| ParserKeyword(Keyword::MM).ignore(pos, expected) || ParserKeyword(Keyword::M).ignore(pos, expected))
{
result = IntervalKind::Kind::Month;
return true;
}
if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("QUARTERS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected)
|| ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected))
if (ParserKeyword(Keyword::QUARTER).ignore(pos, expected) || ParserKeyword(Keyword::QUARTERS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_QUARTER).ignore(pos, expected)
|| ParserKeyword(Keyword::QQ).ignore(pos, expected) || ParserKeyword(Keyword::Q).ignore(pos, expected))
{
result = IntervalKind::Kind::Quarter;
return true;
}
if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("YEARS").ignore(pos, expected)
|| ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected)
|| ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected))
if (ParserKeyword(Keyword::YEAR).ignore(pos, expected) || ParserKeyword(Keyword::YEARS).ignore(pos, expected)
|| ParserKeyword(Keyword::SQL_TSI_YEAR).ignore(pos, expected)
|| ParserKeyword(Keyword::YYYY).ignore(pos, expected) || ParserKeyword(Keyword::YY).ignore(pos, expected))
{
result = IntervalKind::Kind::Year;
return true;

View File

@ -0,0 +1,29 @@
#include <Common/Macros.h>
#include <Interpreters/Context.h>
#include <Storages/System/StorageSystemKeywords.h>
#include <Parsers/CommonParsers.h>
namespace DB
{
ColumnsDescription StorageSystemKeywords::getColumnsDescription()
{
return ColumnsDescription
{
{"keyword", std::make_shared<DataTypeString>(), "The keyword used in ClickHouse parser."},
};
}
void StorageSystemKeywords::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector<UInt8>) const
{
auto macros = context->getMacros();
for (const auto & keyword : getAllKeyWords())
{
res_columns[0]->insert(keyword);
}
}
}

Some files were not shown because too many files have changed in this diff Show More