ClickHouse/src/Parsers/ParserCreateQuery.cpp

1003 lines
28 KiB
C++
Raw Normal View History

2019-01-17 10:09:51 +00:00
#include <Common/typeid_cast.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
2018-12-25 18:45:08 +00:00
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTCreateQuery.h>
2019-03-23 22:45:28 +00:00
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
2018-02-25 00:50:53 +00:00
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
2019-05-12 11:36:02 +00:00
#include <Parsers/ASTConstraintDeclaration.h>
2019-10-09 20:09:10 +00:00
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
2020-07-02 20:39:31 +00:00
#include <IO/ReadHelpers.h>
2011-08-18 18:48:00 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket);
ParserIdentifier name_p;
ParserNameTypePairList columns_p;
ASTPtr name;
ASTPtr columns;
2017-04-02 17:37:49 +00:00
/// For now `name == 'Nested'`, probably alternative nested data structures will appear
if (!name_p.parse(pos, name, expected))
return false;
if (!open.ignore(pos))
return false;
if (!columns_p.parse(pos, columns, expected))
return false;
if (!close.ignore(pos))
return false;
2018-02-26 03:37:08 +00:00
auto func = std::make_shared<ASTFunction>();
2019-08-08 20:02:30 +00:00
tryGetIdentifierNameInto(name, func->name);
// FIXME(ilezhankin): func->no_empty_args = true; ?
func->arguments = columns;
func->children.push_back(columns);
node = func;
return true;
}
bool ParserIdentifierWithParameters::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
2020-06-18 12:52:05 +00:00
return ParserFunction().parse(pos, node, expected);
}
2011-08-18 18:48:00 +00:00
bool ParserNameTypePairList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserNameTypePair>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
bool ParserColumnDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserColumnDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
bool ParserNameList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserCompoundIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
2018-12-25 18:45:08 +00:00
bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_type("TYPE");
2019-01-02 16:04:44 +00:00
ParserKeyword s_granularity("GRANULARITY");
2018-12-25 18:45:08 +00:00
ParserIdentifier name_p;
2020-06-18 12:52:05 +00:00
ParserDataType data_type_p;
2019-01-20 15:02:19 +00:00
ParserExpression expression_p;
2019-01-03 08:10:25 +00:00
ParserUnsignedInteger granularity_p;
2018-12-25 18:45:08 +00:00
ASTPtr name;
ASTPtr expr;
ASTPtr type;
2019-01-02 16:04:44 +00:00
ASTPtr granularity;
2018-12-25 18:45:08 +00:00
if (!name_p.parse(pos, name, expected))
return false;
if (!expression_p.parse(pos, expr, expected))
return false;
if (!s_type.ignore(pos, expected))
return false;
2020-06-18 12:52:05 +00:00
if (!data_type_p.parse(pos, type, expected))
2018-12-25 18:45:08 +00:00
return false;
2019-01-02 16:04:44 +00:00
if (!s_granularity.ignore(pos, expected))
return false;
if (!granularity_p.parse(pos, granularity, expected))
return false;
2018-12-25 18:45:08 +00:00
auto index = std::make_shared<ASTIndexDeclaration>();
index->name = name->as<ASTIdentifier &>().name();
2020-02-14 18:20:20 +00:00
index->granularity = granularity->as<ASTLiteral &>().value.safeGet<UInt64>();
2018-12-25 18:45:08 +00:00
index->set(index->expr, expr);
index->set(index->type, type);
node = index;
return true;
}
2019-05-12 11:36:02 +00:00
bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_check("CHECK");
ParserIdentifier name_p;
2019-05-18 05:16:33 +00:00
ParserLogicalOrExpression expression_p;
2019-05-12 11:36:02 +00:00
ASTPtr name;
ASTPtr expr;
2019-01-20 09:19:34 +00:00
2019-05-12 11:36:02 +00:00
if (!name_p.parse(pos, name, expected))
return false;
if (!s_check.ignore(pos, expected))
return false;
if (!expression_p.parse(pos, expr, expected))
return false;
auto constraint = std::make_shared<ASTConstraintDeclaration>();
constraint->name = name->as<ASTIdentifier &>().name();
2019-05-12 11:36:02 +00:00
constraint->set(constraint->expr, expr);
node = constraint;
return true;
}
bool ParserTablePropertyDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
2019-01-20 09:19:34 +00:00
{
ParserKeyword s_index("INDEX");
2019-05-12 11:36:02 +00:00
ParserKeyword s_constraint("CONSTRAINT");
ParserKeyword s_primary_key("PRIMARY KEY");
2019-01-20 09:19:34 +00:00
ParserIndexDeclaration index_p;
2019-05-12 11:36:02 +00:00
ParserConstraintDeclaration constraint_p;
2020-06-14 16:17:22 +00:00
ParserColumnDeclaration column_p{true, true};
ParserExpression primary_key_p;
2019-01-20 09:19:34 +00:00
2019-01-20 15:02:19 +00:00
ASTPtr new_node = nullptr;
2019-01-20 09:19:34 +00:00
if (s_index.ignore(pos, expected))
{
2019-01-20 15:02:19 +00:00
if (!index_p.parse(pos, new_node, expected))
2019-01-20 09:19:34 +00:00
return false;
}
2019-05-12 11:36:02 +00:00
else if (s_constraint.ignore(pos, expected))
{
if (!constraint_p.parse(pos, new_node, expected))
return false;
}
else if (s_primary_key.ignore(pos, expected))
{
if (!primary_key_p.parse(pos, new_node, expected))
return false;
}
2019-01-20 09:19:34 +00:00
else
{
2019-01-20 15:02:19 +00:00
if (!column_p.parse(pos, new_node, expected))
2019-01-20 09:19:34 +00:00
return false;
}
2019-01-20 15:02:19 +00:00
node = new_node;
2019-01-20 09:19:34 +00:00
return true;
}
2019-01-08 11:04:25 +00:00
bool ParserIndexDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserIndexDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
2019-05-25 14:07:45 +00:00
.parse(pos, node, expected);
}
bool ParserConstraintDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserConstraintDeclaration>(), std::make_unique<ParserToken>(TokenType::Comma), false)
2019-01-08 11:04:25 +00:00
.parse(pos, node, expected);
}
2019-05-12 11:36:02 +00:00
bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
2019-01-20 09:19:34 +00:00
{
ASTPtr list;
2019-05-12 11:36:02 +00:00
if (!ParserList(
std::make_unique<ParserTablePropertyDeclaration>(),
std::make_unique<ParserToken>(TokenType::Comma), false)
2019-01-20 09:19:34 +00:00
.parse(pos, list, expected))
return false;
ASTPtr columns = std::make_shared<ASTExpressionList>();
ASTPtr indices = std::make_shared<ASTExpressionList>();
2019-05-12 11:36:02 +00:00
ASTPtr constraints = std::make_shared<ASTExpressionList>();
ASTPtr primary_key;
2019-01-20 09:19:34 +00:00
for (const auto & elem : list->children)
{
2019-03-11 13:22:51 +00:00
if (elem->as<ASTColumnDeclaration>())
2019-01-20 09:19:34 +00:00
columns->children.push_back(elem);
2019-03-11 13:22:51 +00:00
else if (elem->as<ASTIndexDeclaration>())
2019-01-20 09:19:34 +00:00
indices->children.push_back(elem);
2019-05-12 11:36:02 +00:00
else if (elem->as<ASTConstraintDeclaration>())
constraints->children.push_back(elem);
else if (elem->as<ASTIdentifier>() || elem->as<ASTFunction>())
{
if (primary_key)
{
2021-01-04 21:28:57 +00:00
/// Multiple primary keys are not allowed.
return false;
}
primary_key = elem;
}
2019-01-20 09:19:34 +00:00
else
return false;
}
auto res = std::make_shared<ASTColumns>();
if (!columns->children.empty())
res->set(res->columns, columns);
if (!indices->children.empty())
res->set(res->indices, indices);
2019-05-12 11:36:02 +00:00
if (!constraints->children.empty())
res->set(res->constraints, constraints);
if (primary_key)
res->set(res->primary_key, primary_key);
2019-01-20 09:19:34 +00:00
node = res;
return true;
}
bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
2011-11-01 15:16:04 +00:00
{
ParserKeyword s_engine("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");
2011-11-01 15:16:04 +00:00
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
ParserExpression expression_p;
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
2019-10-09 13:02:05 +00:00
ParserTTLExpressionList parser_ttl_list;
2011-11-01 15:16:04 +00:00
ASTPtr engine;
ASTPtr partition_by;
ASTPtr primary_key;
ASTPtr order_by;
ASTPtr sample_by;
ASTPtr ttl_table;
ASTPtr settings;
if (!s_engine.ignore(pos, expected))
return false;
s_eq.ignore(pos, expected);
if (!ident_with_optional_params_p.parse(pos, engine, expected))
return false;
2011-11-01 15:16:04 +00:00
while (true)
{
if (!partition_by && s_partition_by.ignore(pos, expected))
{
if (expression_p.parse(pos, partition_by, expected))
continue;
else
return false;
}
2011-11-01 15:16:04 +00:00
if (!primary_key && s_primary_key.ignore(pos, expected))
{
if (expression_p.parse(pos, primary_key, expected))
continue;
else
return false;
}
if (!order_by && s_order_by.ignore(pos, expected))
{
if (expression_p.parse(pos, order_by, expected))
continue;
else
return false;
}
if (!sample_by && s_sample_by.ignore(pos, expected))
{
if (expression_p.parse(pos, sample_by, expected))
continue;
else
return false;
}
if (!ttl_table && s_ttl.ignore(pos, expected))
{
2019-10-09 13:02:05 +00:00
if (parser_ttl_list.parse(pos, ttl_table, expected))
continue;
else
return false;
}
if (s_settings.ignore(pos, expected))
{
if (!settings_p.parse(pos, settings, expected))
return false;
}
break;
}
2011-11-01 15:16:04 +00:00
2018-02-26 03:37:08 +00:00
auto storage = std::make_shared<ASTStorage>();
storage->set(storage->engine, engine);
storage->set(storage->partition_by, partition_by);
storage->set(storage->primary_key, primary_key);
storage->set(storage->order_by, order_by);
storage->set(storage->sample_by, sample_by);
storage->set(storage->ttl_table, ttl_table);
2018-12-25 18:45:08 +00:00
storage->set(storage->settings, settings);
node = storage;
return true;
2011-11-01 15:16:04 +00:00
}
2019-10-09 20:09:10 +00:00
bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
2011-08-18 18:48:00 +00:00
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
2020-12-25 16:28:10 +00:00
ParserKeyword s_replace("REPLACE");
ParserKeyword s_or_replace("OR REPLACE");
2019-10-23 13:46:38 +00:00
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
2020-04-06 23:22:44 +00:00
ParserCompoundIdentifier table_name_p(true);
ParserKeyword s_from("FROM");
2019-10-23 13:46:38 +00:00
ParserKeyword s_on("ON");
ParserKeyword s_as("AS");
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p;
ParserIdentifier name_p;
2019-05-12 11:36:02 +00:00
ParserTablePropertiesDeclarationList table_properties_p;
2018-02-25 00:50:53 +00:00
ParserSelectWithUnionQuery select_p;
2019-07-18 18:29:49 +00:00
ParserFunction table_function_p;
ParserNameList names_p;
2020-04-06 23:22:44 +00:00
ASTPtr table;
2019-01-21 17:17:21 +00:00
ASTPtr columns_list;
ASTPtr storage;
ASTPtr as_database;
ASTPtr as_table;
2019-07-18 18:29:49 +00:00
ASTPtr as_table_function;
ASTPtr select;
ASTPtr from_path;
2017-04-21 12:39:28 +00:00
String cluster_str;
bool attach = false;
2020-12-25 16:28:10 +00:00
bool replace = false;
bool or_replace = false;
bool if_not_exists = false;
bool is_temporary = false;
2020-12-25 16:28:10 +00:00
if (s_create.ignore(pos, expected))
{
2020-12-25 16:28:10 +00:00
if (s_or_replace.ignore(pos, expected))
replace = or_replace = true;
}
2020-12-25 16:28:10 +00:00
else if (s_attach.ignore(pos, expected))
attach = true;
else if (s_replace.ignore(pos, expected))
replace = true;
else
return false;
2020-12-25 16:28:10 +00:00
if (!replace && !or_replace && s_temporary.ignore(pos, expected))
{
is_temporary = true;
}
2019-10-09 20:09:10 +00:00
if (!s_table.ignore(pos, expected))
return false;
2020-12-25 16:28:10 +00:00
if (!replace && !or_replace && s_if_not_exists.ignore(pos, expected))
2019-10-09 20:09:10 +00:00
if_not_exists = true;
2020-04-06 23:22:44 +00:00
if (!table_name_p.parse(pos, table, expected))
2019-10-09 20:09:10 +00:00
return false;
if (attach && s_from.ignore(pos, expected))
{
ParserLiteral from_path_p;
if (!from_path_p.parse(pos, from_path, expected))
return false;
}
2019-10-23 13:46:38 +00:00
if (s_on.ignore(pos, expected))
2019-10-09 20:09:10 +00:00
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
2020-04-07 14:05:51 +00:00
StorageID table_id = getTableIdentifier(table);
2020-04-06 23:22:44 +00:00
2019-10-09 20:09:10 +00:00
// Shortcut for ATTACH a previously detached table
bool short_attach = attach && !from_path;
if (short_attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
2019-10-09 20:09:10 +00:00
{
auto query = std::make_shared<ASTCreateQuery>();
node = query;
2019-10-09 20:09:10 +00:00
query->attach = attach;
query->if_not_exists = if_not_exists;
query->cluster = cluster_str;
2019-12-10 19:48:16 +00:00
query->database = table_id.database_name;
query->table = table_id.table_name;
query->uuid = table_id.uuid;
2019-10-09 20:09:10 +00:00
return true;
}
2019-10-09 20:09:10 +00:00
/// List of columns.
if (s_lparen.ignore(pos, expected))
{
if (!table_properties_p.parse(pos, columns_list, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!s_rparen.ignore(pos, expected))
return false;
auto storage_parse_result = storage_p.parse(pos, storage, expected);
if (storage_parse_result && s_as.ignore(pos, expected))
{
if (!select_p.parse(pos, select, expected))
return false;
}
if (!storage_parse_result && !is_temporary)
{
if (!s_as.ignore(pos, expected))
return false;
if (!table_function_p.parse(pos, as_table_function, expected))
{
return false;
}
}
2019-10-09 20:09:10 +00:00
}
else
{
storage_p.parse(pos, storage, expected);
2019-10-09 20:09:10 +00:00
if (!s_as.ignore(pos, expected))
return false;
2019-10-30 13:06:23 +00:00
if (!select_p.parse(pos, select, expected)) /// AS SELECT ...
2019-10-09 20:09:10 +00:00
{
/// ENGINE can not be specified for table functions.
if (storage || !table_function_p.parse(pos, as_table_function, expected))
{
2019-10-09 20:09:10 +00:00
/// AS [db.]table
if (!name_p.parse(pos, as_table, expected))
return false;
if (s_dot.ignore(pos, expected))
{
2019-10-09 20:09:10 +00:00
as_database = as_table;
if (!name_p.parse(pos, as_table, expected))
return false;
2019-07-18 18:29:49 +00:00
}
2019-10-09 20:09:10 +00:00
/// Optional - ENGINE can be specified.
if (!storage)
storage_p.parse(pos, storage, expected);
}
}
}
2019-10-09 20:09:10 +00:00
auto query = std::make_shared<ASTCreateQuery>();
node = query;
if (as_table_function)
query->as_table_function = as_table_function;
query->attach = attach;
2020-12-25 16:28:10 +00:00
query->replace_table = replace;
query->create_or_replace = or_replace;
2019-10-09 20:09:10 +00:00
query->if_not_exists = if_not_exists;
query->temporary = is_temporary;
2019-12-10 19:48:16 +00:00
query->database = table_id.database_name;
query->table = table_id.table_name;
query->uuid = table_id.uuid;
2019-10-09 20:09:10 +00:00
query->cluster = cluster_str;
query->set(query->columns_list, columns_list);
query->set(query->storage, storage);
if (query->storage && query->columns_list && query->columns_list->primary_key)
{
if (query->storage->primary_key)
{
throw Exception("Multiple primary keys are not allowed.", ErrorCodes::BAD_ARGUMENTS);
}
query->storage->primary_key = query->columns_list->primary_key;
}
2019-10-09 20:09:10 +00:00
tryGetIdentifierNameInto(as_database, query->as_database);
tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select);
if (from_path)
query->attach_from_path = from_path->as<ASTLiteral &>().value.get<String>();
2019-10-09 20:09:10 +00:00
return true;
}
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");
2020-04-06 23:22:44 +00:00
ParserCompoundIdentifier table_name_p(true);
2019-10-09 20:09:10 +00:00
ParserKeyword s_as("AS");
ParserKeyword s_view("VIEW");
ParserKeyword s_live("LIVE");
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserTablePropertiesDeclarationList table_properties_p;
ParserSelectWithUnionQuery select_p;
2020-04-06 23:22:44 +00:00
ASTPtr table;
ASTPtr to_table;
2019-10-09 20:09:10 +00:00
ASTPtr columns_list;
ASTPtr as_database;
ASTPtr as_table;
ASTPtr select;
2020-02-14 18:20:20 +00:00
ASTPtr live_view_timeout;
ASTPtr live_view_periodic_refresh;
2019-10-09 20:09:10 +00:00
String cluster_str;
bool attach = false;
bool if_not_exists = false;
bool with_and = false;
bool with_timeout = false;
bool with_periodic_refresh = false;
2019-10-09 20:09:10 +00:00
if (!s_create.ignore(pos, expected))
{
2019-10-09 20:09:10 +00:00
if (s_attach.ignore(pos, expected))
attach = true;
else
2019-10-09 20:09:10 +00:00
return false;
}
if (!s_live.ignore(pos, expected))
return false;
if (!s_view.ignore(pos, expected))
return false;
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
2020-04-06 23:22:44 +00:00
if (!table_name_p.parse(pos, table, expected))
2019-10-09 20:09:10 +00:00
return false;
if (ParserKeyword{"WITH"}.ignore(pos, expected))
2020-02-14 18:20:20 +00:00
{
if (ParserKeyword{"TIMEOUT"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
{
live_view_timeout = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC));
}
/// Optional - AND
if (ParserKeyword{"AND"}.ignore(pos, expected))
with_and = true;
with_timeout = true;
}
if (ParserKeyword{"REFRESH"}.ignore(pos, expected) || ParserKeyword{"PERIODIC REFRESH"}.ignore(pos, expected))
{
if (!ParserNumber{}.parse(pos, live_view_periodic_refresh, expected))
live_view_periodic_refresh = std::make_shared<ASTLiteral>(static_cast<UInt64>(DEFAULT_PERIODIC_LIVE_VIEW_REFRESH_SEC));
with_periodic_refresh = true;
}
else if (with_and)
return false;
if (!with_timeout && !with_periodic_refresh)
return false;
2020-02-14 18:20:20 +00:00
}
2019-10-09 20:09:10 +00:00
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
// TO [db.]table
if (ParserKeyword{"TO"}.ignore(pos, expected))
{
2020-04-06 23:22:44 +00:00
if (!table_name_p.parse(pos, to_table, expected))
2019-10-09 20:09:10 +00:00
return false;
}
2019-10-09 20:09:10 +00:00
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
if (s_lparen.ignore(pos, expected))
{
if (!table_properties_p.parse(pos, columns_list, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!s_rparen.ignore(pos, expected))
return false;
}
2019-08-23 05:08:06 +00:00
2019-10-09 20:09:10 +00:00
/// AS SELECT ...
if (!s_as.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!select_p.parse(pos, select, expected))
return false;
2019-10-09 20:09:10 +00:00
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->attach = attach;
query->if_not_exists = if_not_exists;
query->is_live_view = true;
2020-04-07 14:05:51 +00:00
StorageID table_id = getTableIdentifier(table);
2019-12-10 19:48:16 +00:00
query->database = table_id.database_name;
query->table = table_id.table_name;
query->uuid = table_id.uuid;
2019-10-09 20:09:10 +00:00
query->cluster = cluster_str;
2020-04-07 14:05:51 +00:00
if (to_table)
query->to_table_id = getTableIdentifier(to_table);
2019-10-09 20:09:10 +00:00
query->set(query->columns_list, columns_list);
tryGetIdentifierNameInto(as_database, query->as_database);
tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select);
2020-02-14 18:20:20 +00:00
if (live_view_timeout)
query->live_view_timeout.emplace(live_view_timeout->as<ASTLiteral &>().value.safeGet<UInt64>());
if (live_view_periodic_refresh)
query->live_view_periodic_refresh.emplace(live_view_periodic_refresh->as<ASTLiteral &>().value.safeGet<UInt64>());
2020-02-14 18:20:20 +00:00
2019-10-09 20:09:10 +00:00
return true;
}
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");
ParserStorage storage_p;
ParserIdentifier name_p;
ASTPtr database;
ASTPtr storage;
2020-07-02 20:39:31 +00:00
UUID uuid = UUIDHelpers::Nil;
2019-10-09 20:09:10 +00:00
String cluster_str;
bool attach = false;
bool if_not_exists = false;
2019-10-09 20:09:10 +00:00
if (!s_create.ignore(pos, expected))
{
if (s_attach.ignore(pos, expected))
attach = true;
else
2019-08-23 05:08:06 +00:00
return false;
}
2019-10-09 20:09:10 +00:00
if (!s_database.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!name_p.parse(pos, database, expected))
return false;
2020-07-02 20:39:31 +00:00
if (ParserKeyword("UUID").ignore(pos, expected))
{
ParserStringLiteral uuid_p;
ASTPtr ast_uuid;
if (!uuid_p.parse(pos, ast_uuid, expected))
return false;
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
}
2019-10-09 20:09:10 +00:00
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
2019-10-09 20:09:10 +00:00
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
storage_p.parse(pos, storage, expected);
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->attach = attach;
query->if_not_exists = if_not_exists;
tryGetIdentifierNameInto(database, query->database);
2020-07-02 20:39:31 +00:00
query->uuid = uuid;
2019-10-09 20:09:10 +00:00
query->cluster = cluster_str;
query->set(query->storage, storage);
return true;
}
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");
2020-04-06 23:22:44 +00:00
ParserCompoundIdentifier table_name_p(true);
2019-10-09 20:09:10 +00:00
ParserKeyword s_as("AS");
ParserKeyword s_view("VIEW");
ParserKeyword s_materialized("MATERIALIZED");
ParserKeyword s_populate("POPULATE");
ParserKeyword s_or_replace("OR REPLACE");
ParserToken s_dot(TokenType::Dot);
ParserToken s_lparen(TokenType::OpeningRoundBracket);
ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserStorage storage_p;
ParserIdentifier name_p;
ParserTablePropertiesDeclarationList table_properties_p;
ParserSelectWithUnionQuery select_p;
ParserNameList names_p;
2020-04-06 23:22:44 +00:00
ASTPtr table;
ASTPtr to_table;
2021-03-08 17:26:38 +00:00
ASTPtr to_inner_uuid;
2019-10-09 20:09:10 +00:00
ASTPtr columns_list;
ASTPtr storage;
ASTPtr as_database;
ASTPtr as_table;
ASTPtr select;
2019-10-09 20:09:10 +00:00
String cluster_str;
bool attach = false;
bool if_not_exists = false;
bool is_ordinary_view = false;
2019-10-09 20:09:10 +00:00
bool is_materialized_view = false;
bool is_populate = false;
bool replace_view = false;
if (!s_create.ignore(pos, expected))
{
if (s_attach.ignore(pos, expected))
attach = true;
else
return false;
2019-10-09 20:09:10 +00:00
}
2019-10-09 20:09:10 +00:00
/// VIEW or MATERIALIZED VIEW
if (s_or_replace.ignore(pos, expected))
{
replace_view = true;
}
2019-10-09 20:09:10 +00:00
if (!replace_view && s_materialized.ignore(pos, expected))
{
is_materialized_view = true;
}
else
is_ordinary_view = true;
2019-03-11 16:50:31 +00:00
2019-10-09 20:09:10 +00:00
if (!s_view.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!replace_view && s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
2020-04-06 23:22:44 +00:00
if (!table_name_p.parse(pos, table, expected))
2019-10-09 20:09:10 +00:00
return false;
2019-10-09 20:09:10 +00:00
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
2021-03-08 17:26:38 +00:00
if (ParserKeyword{"TO INNER UUID"}.ignore(pos, expected))
{
ParserLiteral literal_p;
if (!literal_p.parse(pos, to_inner_uuid, expected))
return false;
}
else if (ParserKeyword{"TO"}.ignore(pos, expected))
2019-10-09 20:09:10 +00:00
{
2021-03-08 17:26:38 +00:00
// TO [db.]table
2020-04-06 23:22:44 +00:00
if (!table_name_p.parse(pos, to_table, expected))
2019-10-09 20:09:10 +00:00
return false;
}
2019-10-09 20:09:10 +00:00
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
if (s_lparen.ignore(pos, expected))
{
if (!table_properties_p.parse(pos, columns_list, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!s_rparen.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
}
2020-04-06 23:22:44 +00:00
if (is_materialized_view && !to_table)
2019-10-09 20:09:10 +00:00
{
/// Internal ENGINE for MATERIALIZED VIEW must be specified.
if (!storage_p.parse(pos, storage, expected))
return false;
2019-10-09 20:09:10 +00:00
if (s_populate.ignore(pos, expected))
is_populate = true;
}
2019-10-09 20:09:10 +00:00
/// AS SELECT ...
if (!s_as.ignore(pos, expected))
return false;
if (!select_p.parse(pos, select, expected))
return false;
2018-02-26 03:37:08 +00:00
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->attach = attach;
query->if_not_exists = if_not_exists;
query->is_ordinary_view = is_ordinary_view;
query->is_materialized_view = is_materialized_view;
query->is_populate = is_populate;
2019-03-11 16:50:31 +00:00
query->replace_view = replace_view;
2020-04-07 14:05:51 +00:00
StorageID table_id = getTableIdentifier(table);
2019-12-10 19:48:16 +00:00
query->database = table_id.database_name;
query->table = table_id.table_name;
query->uuid = table_id.uuid;
2017-04-21 12:39:28 +00:00
query->cluster = cluster_str;
2020-04-07 14:05:51 +00:00
if (to_table)
query->to_table_id = getTableIdentifier(to_table);
2021-03-08 17:26:38 +00:00
if (to_inner_uuid)
query->to_inner_uuid = parseFromString<UUID>(to_inner_uuid->as<ASTLiteral>()->value.get<String>());
2019-01-21 17:17:21 +00:00
query->set(query->columns_list, columns_list);
query->set(query->storage, storage);
2019-08-08 20:02:30 +00:00
tryGetIdentifierNameInto(as_database, query->as_database);
tryGetIdentifierNameInto(as_table, query->as_table);
query->set(query->select, select);
2019-10-09 20:09:10 +00:00
return true;
}
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
2019-10-09 20:09:10 +00:00
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_on("ON");
2020-07-13 21:21:01 +00:00
ParserCompoundIdentifier dict_name_p(true);
2019-10-09 20:09:10 +00:00
ParserToken s_left_paren(TokenType::OpeningRoundBracket);
ParserToken s_right_paren(TokenType::ClosingRoundBracket);
ParserToken s_dot(TokenType::Dot);
ParserDictionaryAttributeDeclarationList attributes_p;
ParserDictionary dictionary_p;
bool if_not_exists = false;
ASTPtr name;
ASTPtr attributes;
ASTPtr dictionary;
String cluster_str;
2019-10-09 20:09:10 +00:00
bool attach = false;
2019-10-09 20:09:10 +00:00
if (!s_create.ignore(pos, expected))
{
if (s_attach.ignore(pos, expected))
attach = true;
else
return false;
}
2019-10-09 20:09:10 +00:00
if (!s_dictionary.ignore(pos, expected))
return false;
2019-12-04 15:11:36 +00:00
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
2020-07-13 21:21:01 +00:00
if (!dict_name_p.parse(pos, name, expected))
2019-10-09 20:09:10 +00:00
return false;
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!attach)
{
if (!s_left_paren.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!attributes_p.parse(pos, attributes, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!s_right_paren.ignore(pos, expected))
return false;
2019-10-09 20:09:10 +00:00
if (!dictionary_p.parse(pos, dictionary, expected))
return false;
}
2019-10-09 20:09:10 +00:00
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->is_dictionary = true;
query->attach = attach;
2019-10-09 20:09:10 +00:00
2020-07-13 21:21:01 +00:00
StorageID dict_id = getTableIdentifier(name);
query->database = dict_id.database_name;
query->table = dict_id.table_name;
query->uuid = dict_id.uuid;
2019-10-09 20:09:10 +00:00
query->if_not_exists = if_not_exists;
query->set(query->dictionary_attributes_list, attributes);
query->set(query->dictionary, dictionary);
query->cluster = cluster_str;
2019-10-09 20:09:10 +00:00
return true;
2011-08-18 18:48:00 +00:00
}
2019-10-09 20:09:10 +00:00
bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserCreateTableQuery table_p;
ParserCreateDatabaseQuery database_p;
ParserCreateViewQuery view_p;
ParserCreateDictionaryQuery dictionary_p;
ParserCreateLiveViewQuery live_view_p;
return table_p.parse(pos, node, expected)
|| database_p.parse(pos, node, expected)
|| view_p.parse(pos, node, expected)
|| dictionary_p.parse(pos, node, expected)
|| live_view_p.parse(pos, node, expected);
}
2011-08-18 18:48:00 +00:00
}