Merge pull request #7209 from ClickHouse/dictionaries_ddl_parser

Dictionaries ddl parser
This commit is contained in:
alesapin 2019-10-09 12:51:08 +03:00 committed by GitHub
commit ef0b2f5936
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
32 changed files with 1617 additions and 63 deletions

View File

@ -32,7 +32,7 @@ Block InterpreterExistsQuery::getSampleBlock()
BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
{
const auto & ast = query_ptr->as<ASTExistsQuery &>();
const auto & ast = query_ptr->as<ASTExistsTableQuery &>();
bool res = ast.temporary ? context.isExternalTableExist(ast.table) : context.isTableExist(ast.database, ast.table);
return std::make_shared<OneBlockInputStream>(Block{{

View File

@ -133,7 +133,7 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
throwIfNoAccess(context);
return std::make_unique<InterpreterOptimizeQuery>(query, context);
}
else if (query->as<ASTExistsQuery>())
else if (query->as<ASTExistsTableQuery>())
{
return std::make_unique<InterpreterExistsQuery>(query, context);
}

View File

@ -1,8 +1,8 @@
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
@ -187,6 +187,8 @@ ASTPtr ASTCreateQuery::clone() const
res->set(res->select, select->clone());
if (tables)
res->set(res->tables, tables->clone());
if (dictionary)
res->set(res->dictionary, dictionary->clone());
cloneOutputOptions(*res);
@ -212,6 +214,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
return;
}
if (!is_dictionary)
{
std::string what = "TABLE";
if (is_view)
@ -232,6 +235,14 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
formatOnCluster(settings);
}
else
{
/// Always CREATE and always DICTIONARY
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE DICTIONARY " << (if_not_exists ? "IF NOT EXISTS " : "")
<< (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "")
<< backQuoteIfNeed(table);
}
if (as_table_function)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
@ -260,13 +271,23 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
settings.ostr << (settings.one_line ? ")" : "\n)");
}
if (dictionary_attributes_list)
{
settings.ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;
++frame_nested.indent;
dictionary_attributes_list->formatImpl(settings, state, frame_nested);
settings.ostr << (settings.one_line ? ")" : "\n)");
}
if (storage)
storage->formatImpl(settings, state, frame);
if (dictionary)
dictionary->formatImpl(settings, state, frame);
if (is_populate)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : "");
}
if (select)
{
@ -282,4 +303,3 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
}
}

View File

@ -2,6 +2,8 @@
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTDictionary.h>
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
namespace DB
@ -58,9 +60,11 @@ public:
bool is_materialized_view{false};
bool is_live_view{false};
bool is_populate{false};
bool is_dictionary{false}; /// CREATE DICTIONARY
bool replace_view{false}; /// CREATE OR REPLACE VIEW
ASTColumns * columns_list = nullptr;
ASTExpressionList *tables = nullptr;
ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of dictionary
ASTExpressionList * tables = nullptr;
String to_database; /// For CREATE MATERIALIZED VIEW mv TO table.
String to_table;
ASTStorage * storage = nullptr;
@ -68,6 +72,7 @@ public:
String as_table;
ASTPtr as_table_function;
ASTSelectWithUnionQuery * select = nullptr;
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)
/** Get the text that identifies this element. */
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; }

View File

@ -0,0 +1,164 @@
#include <Parsers/ASTDictionary.h>
#include <Poco/String.h>
namespace DB
{
ASTPtr ASTDictionaryRange::clone() const
{
auto res = std::make_shared<ASTDictionaryRange>(*this);
res->children.clear();
res->min_attr_name = min_attr_name;
res->max_attr_name = max_attr_name;
return res;
}
void ASTDictionaryRange::formatImpl(const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "RANGE"
<< (settings.hilite ? hilite_none : "")
<< "("
<< (settings.hilite ? hilite_keyword : "")
<< "MIN "
<< (settings.hilite ? hilite_none : "")
<< min_attr_name << ", "
<< (settings.hilite ? hilite_keyword : "")
<< "MAX "
<< (settings.hilite ? hilite_none : "")
<< max_attr_name << ")";
}
ASTPtr ASTDictionaryLifetime::clone() const
{
auto res = std::make_shared<ASTDictionaryLifetime>(*this);
res->children.clear();
res->min_sec = min_sec;
res->max_sec = max_sec;
return res;
}
void ASTDictionaryLifetime::formatImpl(const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "LIFETIME"
<< (settings.hilite ? hilite_none : "")
<< "("
<< (settings.hilite ? hilite_keyword : "")
<< "MIN "
<< (settings.hilite ? hilite_none : "")
<< min_sec << ", "
<< (settings.hilite ? hilite_keyword : "")
<< "MAX "
<< (settings.hilite ? hilite_none : "")
<< max_sec << ")";
}
ASTPtr ASTDictionaryLayout::clone() const
{
auto res = std::make_shared<ASTDictionaryLayout>(*this);
res->children.clear();
res->layout_type = layout_type;
if (parameter.has_value())
{
res->parameter.emplace(parameter->first, nullptr);
res->set(res->parameter->second, parameter->second->clone());
}
return res;
}
void ASTDictionaryLayout::formatImpl(const FormatSettings & settings,
FormatState & state,
FormatStateStacked expected) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "LAYOUT"
<< (settings.hilite ? hilite_none : "")
<< "("
<< (settings.hilite ? hilite_keyword : "")
<< Poco::toUpper(layout_type)
<< (settings.hilite ? hilite_none : "");
settings.ostr << "(";
if (parameter)
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< Poco::toUpper(parameter->first)
<< (settings.hilite ? hilite_none : "")
<< " ";
parameter->second->formatImpl(settings, state, expected);
}
settings.ostr << ")";
settings.ostr << ")";
}
ASTPtr ASTDictionary::clone() const
{
auto res = std::make_shared<ASTDictionary>(*this);
res->children.clear();
if (source)
res->set(res->source, source->clone());
if (primary_key)
res->set(res->primary_key, primary_key->clone());
if (lifetime)
res->set(res->lifetime, lifetime->clone());
if (layout)
res->set(res->layout, layout->clone());
if (range)
res->set(res->range, range->clone());
return res;
}
void ASTDictionary::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (primary_key)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "PRIMARY KEY "
<< (settings.hilite ? hilite_none : "");
primary_key->formatImpl(settings, state, frame);
}
if (source)
settings.ostr << settings.nl_or_ws;
source->formatImpl(settings, state, frame);
if (lifetime)
{
settings.ostr << settings.nl_or_ws;
lifetime->formatImpl(settings, state, frame);
}
if (layout)
{
settings.ostr << settings.nl_or_ws;
layout->formatImpl(settings, state, frame);
}
if (range)
{
settings.ostr << settings.nl_or_ws;
range->formatImpl(settings, state, frame);
}
}
}

View File

@ -0,0 +1,86 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
namespace DB
{
/// AST for external dictionary lifetime:
/// lifetime(min 10 max 100)
class ASTDictionaryLifetime : public IAST
{
public:
UInt64 min_sec = 0;
UInt64 max_sec = 0;
String getID(char) const override { return "Dictionary lifetime"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// AST for external dictionary layout. Has name and contain single parameter
/// layout(type()) or layout(type(param value))
class ASTDictionaryLayout : public IAST
{
using KeyValue = std::pair<std::string, ASTLiteral *>;
public:
/// flat, cache, hashed, etc.
String layout_type;
/// optional parameter (size_in_cells)
std::optional<KeyValue> parameter;
String getID(char) const override { return "Dictionary layout"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// AST for external range-hashed dictionary
/// Range bounded with two attributes from minimum to maximum
/// RANGE(min attr1 max attr2)
class ASTDictionaryRange : public IAST
{
public:
String min_attr_name;
String max_attr_name;
String getID(char) const override { return "Dictionary range"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// AST contains all parts of external dictionary definition except attributes
class ASTDictionary : public IAST
{
public:
/// Dictionary keys -- one or more
ASTExpressionList * primary_key;
/// Dictionary external source, doesn't have own AST, because
/// source parameters absolutely different for different sources
ASTFunctionWithKeyValueArguments * source;
/// Lifetime of dictionary (required part)
ASTDictionaryLifetime * lifetime;
/// Layout of dictionary (required part)
ASTDictionaryLayout * layout;
/// Range for dictionary (only for range-hashed dictionaries)
ASTDictionaryRange * range;
String getID(char) const override { return "Dictionary definition"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -0,0 +1,68 @@
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
namespace DB
{
ASTPtr ASTDictionaryAttributeDeclaration::clone() const
{
const auto res = std::make_shared<ASTDictionaryAttributeDeclaration>(*this);
res->children.clear();
if (type)
{
res->type = type->clone();
res->children.push_back(res->type);
}
if (default_value)
{
res->default_value = default_value;
res->children.push_back(res->default_value);
}
if (expression)
{
res->expression = expression->clone();
res->children.push_back(res->expression);
}
return res;
}
void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
if (!settings.one_line)
settings.ostr << settings.nl_or_ws << std::string(4 * frame.indent, ' ');
settings.ostr << backQuote(name);
if (type)
{
settings.ostr << ' ';
type->formatImpl(settings, state, frame);
}
if (default_value)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "DEFAULT" << (settings.hilite ? hilite_none : "") << ' ';
default_value->formatImpl(settings, state, frame);
}
if (expression)
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "EXPRESSION" << (settings.hilite ? hilite_none : "") << ' ';
expression->formatImpl(settings, state, frame);
}
if (hierarchical)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "HIERARCHICAL";
if (injective)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "INJECTIVE";
if (is_object_id)
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "IS_OBJECT_ID";
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTExpressionList.h>
namespace DB
{
/// AST for single dictionary attribute in dictionary DDL query
class ASTDictionaryAttributeDeclaration : public IAST
{
public:
/// Attribute name
String name;
/// Attribute type
ASTPtr type;
/// Attribute default value
ASTPtr default_value;
/// Attribute expression
ASTPtr expression;
/// Is atribute mirrored to the parent identifier
bool hierarchical;
/// Flag that shows whether the id->attribute image is injective
bool injective;
/// MongoDB object ID
bool is_object_id;
String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -44,7 +44,12 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
if (temporary)
settings.ostr << "TEMPORARY ";
settings.ostr << ((table.empty() && !database.empty()) ? "DATABASE " : "TABLE ");
if (table.empty() && !database.empty())
settings.ostr << "DATABASE ";
else if (!is_dictionary)
settings.ostr << "TABLE ";
else
settings.ostr << "DICTIONARY ";
if (if_exists)
settings.ostr << "IF EXISTS ";
@ -60,4 +65,3 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
}
}

View File

@ -25,6 +25,9 @@ public:
/// Useful if we already have a DDL lock
bool no_ddl_lock{false};
/// We dropping dictionary, so print correct word
bool is_dictionary{false};
/** Get the text that identifies this element. */
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -16,7 +16,11 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << ", ";
{
if (separator)
settings.ostr << separator;
settings.ostr << ' ';
}
(*it)->formatImpl(settings, state, frame);
}
@ -30,7 +34,12 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
settings.ostr << ", ";
{
if (separator)
settings.ostr << separator;
settings.ostr << ' ';
}
if (children.size() > 1)
settings.ostr << indent_str;

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/Lexer.h>
namespace DB
@ -11,11 +12,14 @@ namespace DB
class ASTExpressionList : public IAST
{
public:
explicit ASTExpressionList(char separator_ = ',') : separator(separator_) {}
String getID(char) const override { return "ExpressionList"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
void formatImplMultiline(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
char separator;
};
}

View File

@ -0,0 +1,73 @@
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Poco/String.h>
namespace DB
{
String ASTPair::getID(char) const
{
return "pair";
}
ASTPtr ASTPair::clone() const
{
auto res = std::make_shared<ASTPair>(*this);
res->children.clear();
if (second)
{
res->second = second;
res->children.push_back(second);
}
return res;
}
void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(first) << " " << (settings.hilite ? hilite_none : "");
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "(";
second->formatImpl(settings, state, frame);
if (second_with_brackets)
settings.ostr << (settings.hilite ? hilite_keyword : "") << ")";
settings.ostr << (settings.hilite ? hilite_none : "");
}
String ASTFunctionWithKeyValueArguments::getID(char delim) const
{
return "FunctionWithKeyValueArguments " + (delim + name);
}
ASTPtr ASTFunctionWithKeyValueArguments::clone() const
{
auto res = std::make_shared<ASTFunctionWithKeyValueArguments>(*this);
res->children.clear();
if (elements)
{
res->elements->clone();
res->children.push_back(res->elements);
}
return res;
}
void ASTFunctionWithKeyValueArguments::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << Poco::toUpper(name) << (settings.hilite ? hilite_none : "") << "(";
elements->formatImpl(settings, state, frame);
settings.ostr << ")";
settings.ostr << (settings.hilite ? hilite_none : "");
}
}

View File

@ -0,0 +1,56 @@
#pragma once
#include <Parsers/IAST.h>
#include <Core/Types.h>
namespace DB
{
/// Pair with name and value in lisp programming langugate style. It contain
/// string as key, but value either can be literal or list of
/// pairs.
class ASTPair : public IAST
{
public:
/// Name or key of pair
String first;
/// Value of pair, which can be also list of pairs
ASTPtr second;
/// Value is closed in brackets (HOST '127.0.0.1')
bool second_with_brackets;
public:
ASTPair(bool second_with_brackets_)
: second_with_brackets(second_with_brackets_)
{
}
String getID(char delim) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// Function with key-value arguments is a function which arguments consist of
/// pairs (see above). For example:
/// ->Pair with list of pairs as value<-
/// SOURCE(USER 'clickhouse' PORT 9000 REPLICA(HOST '127.0.0.1' PRIORITY 1) TABLE 'some_table')
class ASTFunctionWithKeyValueArguments : public IAST
{
public:
/// Name of function
String name;
/// Expression list
ASTPtr elements;
public:
String getID(char delim) const override;
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

View File

@ -140,4 +140,13 @@ protected:
}
};
// Parser always returns true and do nothing.
class ParserNothing : public IParserBase
{
public:
const char * getName() const override { return "nothing"; }
bool parseImpl(Pos & /*pos*/, ASTPtr & /*node*/, Expected & /*expected*/) override { return true; }
};
}

View File

@ -18,6 +18,7 @@
#include <Parsers/ASTQueryParameter.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionListParsers.h>
@ -1439,4 +1440,54 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return true;
}
bool ParserFunctionWithKeyValueArguments::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_parser;
ParserKeyValuePairsList pairs_list_parser;
ASTPtr identifier;
ASTPtr expr_list_args;
if (!id_parser.parse(pos, identifier, expected))
return false;
if (pos.get().type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (!pairs_list_parser.parse(pos, expr_list_args, expected))
return false;
if (pos.get().type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto function = std::make_shared<ASTFunctionWithKeyValueArguments>();
function->name = Poco::toLower(typeid_cast<ASTIdentifier &>(*identifier.get()).name);
function->elements = expr_list_args;
function->children.push_back(function->elements);
node = function;
return true;
}
bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier non_parametric;
ParserIdentifierWithParameters parametric;
if (parametric.parse(pos, node, expected))
return true;
ASTPtr ident;
if (non_parametric.parse(pos, ident, expected))
{
auto func = std::make_shared<ASTFunction>();
tryGetIdentifierNameInto(ident, func->name);
node = func;
return true;
}
return false;
}
}

View File

@ -298,4 +298,24 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
/** Parser for function with arguments like KEY VALUE (space separated)
* no commas alowed, just space-separated pairs.
*/
class ParserFunctionWithKeyValueArguments : public IParserBase
{
protected:
const char * getName() const override { return "function with key-value arguments"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** Data type or table engine, possibly with parameters. For example, UInt8 or see examples from ParserIdentifierWithParameters
* Parse result is ASTFunction, with or without arguments.
*/
class ParserIdentifierWithOptionalParameters : public IParserBase
{
protected:
const char * getName() const { return "identifier with optional parameters"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
}

View File

@ -5,6 +5,7 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Common/typeid_cast.h>
#include <Common/StringUtils/StringUtils.h>
@ -78,7 +79,7 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
bool first = true;
auto list = std::make_shared<ASTExpressionList>();
auto list = std::make_shared<ASTExpressionList>(result_separator);
node = list;
while (true)
@ -627,5 +628,47 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec
return true;
}
bool ParserKeyValuePair::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_parser;
ParserLiteral literal_parser;
ASTPtr identifier;
ASTPtr value;
bool with_brackets = false;
if (!id_parser.parse(pos, identifier, expected))
return false;
/// If it's not literal or identifier, than it's possible list of pairs
if (!literal_parser.parse(pos, value, expected) && !id_parser.parse(pos, value, expected))
{
ParserKeyValuePairsList kv_pairs_list;
ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket);
if (!open.ignore(pos))
return false;
if (!kv_pairs_list.parse(pos, value, expected))
return false;
if (!close.ignore(pos))
return false;
with_brackets = true;
}
auto pair = std::make_shared<ASTPair>(with_brackets);
pair->first = Poco::toLower(typeid_cast<ASTIdentifier &>(*identifier.get()).name);
pair->second = value;
node = pair;
return true;
}
bool ParserKeyValuePairsList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserList parser(std::make_unique<ParserKeyValuePair>(), std::make_unique<ParserNothing>(), true, 0);
return parser.parse(pos, node, expected);
}
}

View File

@ -19,8 +19,11 @@ using Operators_t = const char **;
class ParserList : public IParserBase
{
public:
ParserList(ParserPtr && elem_parser_, ParserPtr && separator_parser_, bool allow_empty_ = true)
: elem_parser(std::move(elem_parser_)), separator_parser(std::move(separator_parser_)), allow_empty(allow_empty_)
ParserList(ParserPtr && elem_parser_, ParserPtr && separator_parser_, bool allow_empty_ = true, char result_separator_ = ',')
: elem_parser(std::move(elem_parser_))
, separator_parser(std::move(separator_parser_))
, allow_empty(allow_empty_)
, result_separator(result_separator_)
{
}
protected:
@ -30,6 +33,7 @@ private:
ParserPtr elem_parser;
ParserPtr separator_parser;
bool allow_empty;
char result_separator;
};
@ -371,4 +375,20 @@ protected:
};
/// Parser for key-value pair, where value can be list of pairs.
class ParserKeyValuePair : public IParserBase
{
protected:
const char * getName() const override { return "key-value pair"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parser for list of key-value pairs.
class ParserKeyValuePairsList : public IParserBase
{
protected:
const char * getName() const override { return "list of pairs"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -114,9 +114,9 @@ public:
return true;
}
/* The same, but never move the position and do not write the result to node.
/** The same, but doesn't move the position even if parsing was successful.
*/
bool check_without_moving(Pos pos, Expected & expected)
bool checkWithoutMoving(Pos pos, Expected & expected)
{
ASTPtr node;
return parse(pos, node, expected);

View File

@ -62,27 +62,6 @@ bool ParserIdentifierWithParameters::parseImpl(Pos & pos, ASTPtr & node, Expecte
return false;
}
bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier non_parametric;
ParserIdentifierWithParameters parametric;
if (parametric.parse(pos, node, expected))
return true;
ASTPtr ident;
if (non_parametric.parse(pos, ident, expected))
{
auto func = std::make_shared<ASTFunction>();
tryGetIdentifierNameInto(ident, func->name);
node = func;
return true;
}
return false;
}
bool ParserNameTypePairList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserNameTypePair>(), std::make_unique<ParserToken>(TokenType::Comma), false)

View File

@ -37,18 +37,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
/** Data type or table engine, possibly with parameters. For example, UInt8 or see examples from ParserIdentifierWithParameters
* Parse result is ASTFunction, with or without arguments.
*/
class ParserIdentifierWithOptionalParameters : public IParserBase
{
protected:
const char * getName() const { return "identifier with optional parameters"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
template <typename NameParser>
class IParserNameTypePair : public IParserBase
{
@ -152,11 +140,11 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ASTPtr codec_expression;
ASTPtr ttl_expression;
if (!s_default.check_without_moving(pos, expected) &&
!s_materialized.check_without_moving(pos, expected) &&
!s_alias.check_without_moving(pos, expected) &&
!s_comment.check_without_moving(pos, expected) &&
!s_codec.check_without_moving(pos, expected))
if (!s_default.checkWithoutMoving(pos, expected) &&
!s_materialized.checkWithoutMoving(pos, expected) &&
!s_alias.checkWithoutMoving(pos, expected) &&
!s_comment.checkWithoutMoving(pos, expected) &&
!s_codec.checkWithoutMoving(pos, expected))
{
if (!type_parser.parse(pos, type, expected))
return false;

View File

@ -0,0 +1,324 @@
#include <Parsers/ParserDictionary.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTDictionary.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
#include <Poco/String.h>
namespace DB
{
bool ParserDictionaryLifetime::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserLiteral literal_p;
ParserKeyValuePairsList key_value_pairs_p;
ASTPtr ast_lifetime;
auto res = std::make_shared<ASTDictionaryLifetime>();
/// simple lifetime with only maximum value e.g. LIFETIME(300)
if (literal_p.parse(pos, ast_lifetime, expected))
{
auto literal = ast_lifetime->as<const ASTLiteral &>();
if (literal.value.getType() != Field::Types::UInt64)
return false;
res->max_sec = literal.value.get<UInt64>();
node = res;
return true;
}
if (!key_value_pairs_p.parse(pos, ast_lifetime, expected))
return false;
const ASTExpressionList & expr_list = ast_lifetime->as<const ASTExpressionList &>();
if (expr_list.children.size() != 2)
return false;
bool initialized_max = false;
/// should contain both min and max
for (const auto & elem : expr_list.children)
{
const ASTPair & pair = elem->as<const ASTPair &>();
const ASTLiteral * literal = dynamic_cast<const ASTLiteral *>(pair.second.get());
if (literal == nullptr)
return false;
if (literal->value.getType() != Field::Types::UInt64)
return false;
if (pair.first == "min")
res->min_sec = literal->value.get<UInt64>();
else if (pair.first == "max")
{
res->max_sec = literal->value.get<UInt64>();
initialized_max = true;
}
else
return false;
}
if (!initialized_max)
return false;
node = res;
return true;
}
bool ParserDictionaryRange::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyValuePairsList key_value_pairs_p;
ASTPtr ast_range;
if (!key_value_pairs_p.parse(pos, ast_range, expected))
return false;
const ASTExpressionList & expr_list = ast_range->as<const ASTExpressionList &>();
if (expr_list.children.size() != 2)
return false;
auto res = std::make_shared<ASTDictionaryRange>();
for (const auto & elem : expr_list.children)
{
const ASTPair & pair = elem->as<const ASTPair &>();
const ASTIdentifier * identifier = dynamic_cast<const ASTIdentifier *>(pair.second.get());
if (identifier == nullptr)
return false;
if (pair.first == "min")
res->min_attr_name = identifier->name;
else if (pair.first == "max")
res->max_attr_name = identifier->name;
else
return false;
}
if (res->min_attr_name.empty() || res->max_attr_name.empty())
return false;
node = res;
return true;
}
bool ParserDictionaryLayout::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserFunctionWithKeyValueArguments key_value_func_p;
ASTPtr ast_func;
if (!key_value_func_p.parse(pos, ast_func, expected))
return false;
const ASTFunctionWithKeyValueArguments & func = ast_func->as<const ASTFunctionWithKeyValueArguments &>();
auto res = std::make_shared<ASTDictionaryLayout>();
/// here must be exactly one argument - layout_type
if (func.children.size() > 1)
return false;
res->layout_type = func.name;
const ASTExpressionList & type_expr_list = func.elements->as<const ASTExpressionList &>();
/// there are no layout with more than 1 parameter
if (type_expr_list.children.size() > 1)
return false;
if (type_expr_list.children.size() == 1)
{
const ASTPair * pair = dynamic_cast<const ASTPair *>(type_expr_list.children.at(0).get());
if (pair == nullptr)
return false;
const ASTLiteral * literal = dynamic_cast<const ASTLiteral *>(pair->second.get());
if (literal == nullptr || literal->value.getType() != Field::Types::UInt64)
return false;
res->parameter.emplace(pair->first, nullptr);
res->set(res->parameter->second, literal->clone());
}
node = res;
return true;
}
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");
ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket);
ParserFunctionWithKeyValueArguments key_value_pairs_p;
ParserList expression_list_p(std::make_unique<ParserIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserDictionaryLifetime lifetime_p;
ParserDictionaryRange range_p;
ParserDictionaryLayout layout_p;
ASTPtr primary_key;
ASTPtr ast_source;
ASTPtr ast_lifetime;
ASTPtr ast_layout;
ASTPtr ast_range;
/// Primary is required to be the first in dictionary definition
if (primary_key_keyword.ignore(pos) && !expression_list_p.parse(pos, primary_key, expected))
return false;
/// Loop is used to avoid strict order of dictionary properties
while (true)
{
if (!ast_source && source_keyword.ignore(pos, expected))
{
if (!open.ignore(pos))
return false;
if (!key_value_pairs_p.parse(pos, ast_source, expected))
return false;
if (!close.ignore(pos))
return false;
continue;
}
if (!ast_lifetime && lifetime_keyword.ignore(pos, expected))
{
if (!open.ignore(pos))
return false;
if (!lifetime_p.parse(pos, ast_lifetime, expected))
return false;
if (!close.ignore(pos))
return false;
continue;
}
if (!ast_layout && layout_keyword.ignore(pos, expected))
{
if (!open.ignore(pos))
return false;
if (!layout_p.parse(pos, ast_layout, expected))
return false;
if (!close.ignore(pos))
return false;
continue;
}
if (!ast_range && range_keyword.ignore(pos, expected))
{
if (!open.ignore(pos))
return false;
if (!range_p.parse(pos, ast_range, expected))
return false;
if (!close.ignore(pos))
return false;
continue;
}
break;
}
auto query = std::make_shared<ASTDictionary>();
node = query;
if (primary_key)
query->set(query->primary_key, primary_key);
if (ast_source)
query->set(query->source, ast_source);
if (ast_lifetime)
query->set(query->lifetime, ast_lifetime);
if (ast_layout)
query->set(query->layout, ast_layout);
if (ast_range)
query->set(query->range, ast_range);
return true;
}
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserIdentifier name_p;
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 database;
ASTPtr name;
ASTPtr attributes;
ASTPtr dictionary;
if (!s_create.ignore(pos, expected))
return false;
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
if (!s_dictionary.ignore(pos, expected))
return false;
if (!name_p.parse(pos, name, expected))
return false;
if (s_dot.ignore(pos))
{
database = name;
if (!name_p.parse(pos, name, expected))
return false;
}
if (!s_left_paren.ignore(pos, expected))
return false;
if (!attributes_p.parse(pos, attributes, expected))
return false;
if (!s_right_paren.ignore(pos, expected))
return false;
if (!dictionary_p.parse(pos, dictionary, expected))
return false;
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->is_dictionary = true;
if (database)
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
query->table = typeid_cast<const ASTIdentifier &>(*name).name;
query->if_not_exists = if_not_exists;
query->set(query->dictionary_attributes_list, attributes);
query->set(query->dictionary, dictionary);
return true;
}
}

View File

@ -0,0 +1,67 @@
#pragma once
#include <Parsers/IParser.h>
#include <Parsers/IParserBase.h>
namespace DB
{
/// Parser for dictionary lifetime part. It should contain "lifetime" keyword,
/// opening bracket, literal value or two pairs and closing bracket:
/// lifetime(300), lifetime(min 100 max 200). Produces ASTDictionaryLifetime.
class ParserDictionaryLifetime : public IParserBase
{
protected:
const char * getName() const override { return "lifetime definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parser for dictionary range part. It should contain "range" keyword opening
/// bracket, two pairs and closing bracket: range(min attr1 max attr2). Produces
/// ASTDictionaryRange.
class ParserDictionaryRange : public IParserBase
{
protected:
const char * getName() const override { return "range definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parser for dictionary layout part. It should contain "layout" keyword,
/// opening bracket, possible pair with param value and closing bracket:
/// layout(type()) or layout(type(param value)). Produces ASTDictionaryLayout.
class ParserDictionaryLayout : public IParserBase
{
protected:
const char * getName() const override { return "layout definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Combines together all parsers from above and also parses primary key and
/// dictionary source, which consists of custom key-value pairs:
///
/// PRIMARY KEY key_column1, key_column2
/// SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))
/// LAYOUT(CACHE(size_in_cells 50))
/// LIFETIME(MIN 1 MAX 10)
/// RANGE(MIN second_column MAX third_column)
///
/// Produces ASTDictionary.
class ParserDictionary : public IParserBase
{
protected:
const char * getName() const override { return "dictionary definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Parses complete dictionary create query. Uses ParserDictionary and
/// ParserDictionaryAttributeDeclaration. Produces ASTCreateQuery.
class ParserCreateDictionaryQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE DICTIONARY"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -0,0 +1,113 @@
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
namespace DB
{
bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier name_parser;
ParserIdentifierWithOptionalParameters type_parser;
ParserKeyword s_default{"DEFAULT"};
ParserKeyword s_expression{"EXPRESSION"};
ParserKeyword s_hierarchical{"HIERARCHICAL"};
ParserKeyword s_injective{"INJECTIVE"};
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
ParserLiteral default_parser;
ParserTernaryOperatorExpression expression_parser;
/// mandatory attribute name
ASTPtr name;
if (!name_parser.parse(pos, name, expected))
return false;
ASTPtr type;
ASTPtr default_value;
ASTPtr expression;
bool hierarchical = false;
bool injective = false;
bool is_object_id = false;
/// attribute name should be followed by type name if it
if (!type_parser.parse(pos, type, expected))
return false;
/// loop to avoid strict order of attribute properties
while (true)
{
if (!default_value && s_default.ignore(pos, expected))
{
if (!default_parser.parse(pos, default_value, expected))
return false;
continue;
}
if (!expression && s_expression.ignore(pos, expected))
{
if (!expression_parser.parse(pos, expression, expected))
return false;
continue;
}
/// just single keyword, we don't use "true" or "1" for value
if (!hierarchical && s_hierarchical.ignore(pos, expected))
{
hierarchical = true;
continue;
}
if (!injective && s_injective.ignore(pos, expected))
{
injective = true;
continue;
}
if (!is_object_id && s_is_object_id.ignore(pos, expected))
{
is_object_id = true;
continue;
}
break;
}
auto attribute_declaration = std::make_shared<ASTDictionaryAttributeDeclaration>();
node = attribute_declaration;
tryGetIdentifierNameInto(name, attribute_declaration->name);
if (type)
{
attribute_declaration->type = type;
attribute_declaration->children.push_back(std::move(type));
}
if (default_value)
{
attribute_declaration->default_value = default_value;
attribute_declaration->children.push_back(std::move(default_value));
}
if (expression)
{
attribute_declaration->expression = expression;
attribute_declaration->children.push_back(std::move(expression));
}
attribute_declaration->hierarchical = hierarchical;
attribute_declaration->injective = injective;
attribute_declaration->is_object_id = is_object_id;
return true;
}
bool ParserDictionaryAttributeDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserList(std::make_unique<ParserDictionaryAttributeDeclaration>(),
std::make_unique<ParserToken>(TokenType::Comma), false)
.parse(pos, node, expected);
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Parsers/IParserBase.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
/// Parser for dictionary attribute declaration, similar with parser for table
/// column, but attributes has less parameters. Produces
/// ASTDictionaryAttributeDeclaration.
class ParserDictionaryAttributeDeclaration : public IParserBase
{
protected:
const char * getName() const override { return "attribute declaration"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Creates ASTExpressionList consists of dictionary attributes declaration.
class ParserDictionaryAttributeDeclarationList : public IParserBase
{
protected:
const char * getName() const { return "attribute declaration list"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
}

View File

@ -56,6 +56,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
{
ParserKeyword s_temporary("TEMPORARY");
ParserKeyword s_table("TABLE");
ParserKeyword s_dictionary("DICTIONARY");
ParserKeyword s_database("DATABASE");
ParserToken s_dot(TokenType::Dot);
ParserKeyword s_if_exists("IF EXISTS");
@ -66,6 +67,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
String cluster_str;
bool if_exists = false;
bool temporary = false;
bool is_dictionary = false;
if (s_database.ignore(pos, expected))
{
@ -87,7 +89,11 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
temporary = true;
if (!s_table.ignore(pos, expected))
return false;
{
if (!s_dictionary.ignore(pos, expected))
return false;
is_dictionary = true;
}
if (s_if_exists.ignore(pos, expected))
if_exists = true;
@ -115,6 +121,7 @@ bool ParserDropQuery::parseDropQuery(Pos & pos, ASTPtr & node, Expected & expect
query->kind = ASTDropQuery::Kind::Drop;
query->if_exists = if_exists;
query->temporary = temporary;
query->is_dictionary = is_dictionary;
tryGetIdentifierNameInto(database, query->database);
tryGetIdentifierNameInto(table, query->table);

View File

@ -12,6 +12,9 @@ namespace DB
*
* Or:
* DROP DATABASE [IF EXISTS] db
*
* Or:
* DROP DICTIONARY [IF EXISTS] [db.]name
*/
class ParserDropQuery : public IParserBase
{

View File

@ -21,6 +21,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserKeyword s_create("CREATE");
ParserKeyword s_database("DATABASE");
ParserKeyword s_table("TABLE");
ParserKeyword s_dictionary("DICTIONARY");
ParserToken s_dot(TokenType::Dot);
ParserIdentifier name_p;
@ -30,9 +31,16 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
bool parse_only_database_name = false;
bool temporary = false;
if (s_exists.ignore(pos, expected))
{
query = std::make_shared<ASTExistsQuery>();
if (s_temporary.ignore(pos, expected))
temporary = true;
if (s_table.checkWithoutMoving(pos, expected))
query = std::make_shared<ASTExistsTableQuery>();
else if (s_dictionary.checkWithoutMoving(pos, expected))
query = std::make_shared<ASTExistsDictionaryQuery>();
}
else if (s_show.ignore(pos, expected))
{
@ -44,6 +52,8 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
parse_only_database_name = true;
query = std::make_shared<ASTShowCreateDatabaseQuery>();
}
else if (s_dictionary.checkWithoutMoving(pos, expected))
query = std::make_shared<ASTShowCreateDictionaryQuery>();
else
query = std::make_shared<ASTShowCreateTableQuery>();
}
@ -59,10 +69,11 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
}
else
{
if (s_temporary.ignore(pos, expected))
if (temporary || s_temporary.ignore(pos, expected))
query->temporary = true;
s_table.ignore(pos, expected);
if (!s_table.ignore(pos, expected))
s_dictionary.ignore(pos, expected);
if (!name_p.parse(pos, table, expected))
return false;

View File

@ -7,7 +7,7 @@
namespace DB
{
/** Query (EXISTS | SHOW CREATE) [TABLE] [db.]name [FORMAT format]
/** Query (EXISTS | SHOW CREATE) [TABLE|DICTIONARY] [db.]name [FORMAT format]
*/
class ParserTablePropertiesQuery : public IParserBase
{

View File

@ -6,13 +6,21 @@
namespace DB
{
struct ASTExistsQueryIDAndQueryNames
struct ASTExistsTableQueryIDAndQueryNames
{
static constexpr auto ID = "ExistsQuery";
static constexpr auto ID = "ExistsTableQuery";
static constexpr auto Query = "EXISTS TABLE";
static constexpr auto QueryTemporary = "EXISTS TEMPORARY TABLE";
};
struct ASTExistsDictionaryQueryIDAndQueryNames
{
static constexpr auto ID = "ExistsDictionaryQuery";
static constexpr auto Query = "EXISTS DICTIONARY";
/// No temporary dictionaries are supported, just for parsing
static constexpr auto QueryTemporary = "EXISTS TEMPORARY DICTIONARY";
};
struct ASTShowCreateTableQueryIDAndQueryNames
{
static constexpr auto ID = "ShowCreateTableQuery";
@ -27,6 +35,14 @@ struct ASTShowCreateDatabaseQueryIDAndQueryNames
static constexpr auto QueryTemporary = "SHOW CREATE TEMPORARY DATABASE";
};
struct ASTShowCreateDictionaryQueryIDAndQueryNames
{
static constexpr auto ID = "ShowCreateDictionaryQuery";
static constexpr auto Query = "SHOW CREATE DICTIONARY";
/// No temporary dictionaries are supported, just for parsing
static constexpr auto QueryTemporary = "SHOW CREATE TEMPORARY DICTIONARY";
};
struct ASTDescribeQueryExistsQueryIDAndQueryNames
{
static constexpr auto ID = "DescribeQuery";
@ -34,8 +50,10 @@ struct ASTDescribeQueryExistsQueryIDAndQueryNames
static constexpr auto QueryTemporary = "DESCRIBE TEMPORARY TABLE";
};
using ASTExistsQuery = ASTQueryWithTableAndOutputImpl<ASTExistsQueryIDAndQueryNames>;
using ASTExistsTableQuery = ASTQueryWithTableAndOutputImpl<ASTExistsTableQueryIDAndQueryNames>;
using ASTExistsDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTExistsDictionaryQueryIDAndQueryNames>;
using ASTShowCreateTableQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateTableQueryIDAndQueryNames>;
using ASTShowCreateDictionaryQuery = ASTQueryWithTableAndOutputImpl<ASTShowCreateDictionaryQueryIDAndQueryNames>;
class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl<ASTShowCreateDatabaseQueryIDAndQueryNames>
{

View File

@ -0,0 +1,343 @@
#include <iostream>
#include <Core/Types.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/ParserTablePropertiesQuery.h>
#include <sstream>
#include <gtest/gtest.h>
using namespace DB;
String astToString(IAST * ast)
{
std::ostringstream oss;
dumpAST(*ast, oss);
return oss.str();
}
/// Tests for external dictionaries DDL parser
TEST(ParserDictionaryDDL, SimpleDictionary)
{
String input = " CREATE DICTIONARY test.dict1"
" ("
" key_column UInt64 DEFAULT 0,"
" second_column UInt8 DEFAULT 1,"
" third_column UInt8 DEFAULT 2"
" )"
" PRIMARY KEY key_column"
" SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))"
" LAYOUT(FLAT())"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
EXPECT_EQ(create->table, "dict1");
EXPECT_EQ(create->database, "test");
EXPECT_EQ(create->is_dictionary, true);
EXPECT_NE(create->dictionary, nullptr);
EXPECT_NE(create->dictionary->lifetime, nullptr);
EXPECT_NE(create->dictionary->source, nullptr);
EXPECT_NE(create->dictionary->layout, nullptr);
EXPECT_NE(create->dictionary->primary_key, nullptr);
EXPECT_NE(create->dictionary->range, nullptr);
/// source test
EXPECT_EQ(create->dictionary->source->name, "clickhouse");
auto children = create->dictionary->source->elements->children;
EXPECT_EQ(children[0]->as<ASTPair>() -> first, "host");
EXPECT_EQ(children[0]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "localhost");
EXPECT_EQ(children[1]->as<ASTPair>()->first, "port");
EXPECT_EQ(children[1]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<UInt64>(), 9000);
EXPECT_EQ(children[2]->as<ASTPair>()->first, "user");
EXPECT_EQ(children[2]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "default");
EXPECT_EQ(children[3]->as<ASTPair>()->first, "password");
EXPECT_EQ(children[3]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "");
EXPECT_EQ(children[4]->as<ASTPair>()->first, "db");
EXPECT_EQ(children[4]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "test");
EXPECT_EQ(children[5]->as<ASTPair>()->first, "table");
EXPECT_EQ(children[5]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "table_for_dict");
/// layout test
auto layout = create->dictionary->layout;
EXPECT_EQ(layout->layout_type, "flat");
EXPECT_EQ(layout->children.size(), 0);
/// lifetime test
auto lifetime = create->dictionary->lifetime;
EXPECT_EQ(lifetime->min_sec, 1);
EXPECT_EQ(lifetime->max_sec, 10);
/// primary key test
auto primary_key = create->dictionary->primary_key;
EXPECT_EQ(primary_key->children.size(), 1);
EXPECT_EQ(primary_key->children[0]->as<ASTIdentifier>()->name, "key_column");
/// range test
auto range = create->dictionary->range;
EXPECT_EQ(range->min_attr_name, "second_column");
EXPECT_EQ(range->max_attr_name, "third_column");
/// test attributes
EXPECT_NE(create->dictionary_attributes_list, nullptr);
auto attributes_children = create->dictionary_attributes_list->children;
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->name, "key_column");
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->name, "second_column");
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->name, "third_column");
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 0);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 1);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 2);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
}
TEST(ParserDictionaryDDL, AttributesWithMultipleProperties)
{
String input = " CREATE DICTIONARY dict2"
" ("
" key_column UInt64 IS_OBJECT_ID,"
" second_column UInt8 DEFAULT 1 HIERARCHICAL INJECTIVE,"
" third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77"
" )"
" PRIMARY KEY key_column"
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
EXPECT_EQ(create->table, "dict2");
EXPECT_EQ(create->database, "");
/// test attributes
EXPECT_NE(create->dictionary_attributes_list, nullptr);
auto attributes_children = create->dictionary_attributes_list->children;
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->name, "key_column");
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->name, "second_column");
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->name, "third_column");
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->default_value, nullptr);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 1);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 2);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression, true), "(rand() % 100) * 77");
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, true);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->injective, true);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, true);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
}
TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder)
{
String input = " CREATE DICTIONARY dict3"
" ("
" key_column UInt64 IS_OBJECT_ID DEFAULT 100,"
" second_column UInt8 INJECTIVE HIERARCHICAL DEFAULT 1,"
" third_column UInt8 EXPRESSION rand() % 100 * 77 DEFAULT 2 INJECTIVE HIERARCHICAL"
" )"
" PRIMARY KEY key_column"
" SOURCE(CLICKHOUSE(REPLICA(HOST '127.0.0.1' PRIORITY 1)))"
" LIFETIME(300)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
/// test attributes
EXPECT_NE(create->dictionary_attributes_list, nullptr);
auto attributes_children = create->dictionary_attributes_list->children;
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->name, "key_column");
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->name, "second_column");
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->name, "third_column");
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 100);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 1);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->default_value->as<ASTLiteral>()->value.get<UInt64>(), 2);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression, true), "(rand() % 100) * 77");
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, true);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, true);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->injective, false);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->injective, true);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->injective, true);
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, true);
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
EXPECT_EQ(attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->is_object_id, false);
/// lifetime test
auto lifetime = create->dictionary->lifetime;
EXPECT_EQ(lifetime->min_sec, 0);
EXPECT_EQ(lifetime->max_sec, 300);
}
TEST(ParserDictionaryDDL, NestedSource)
{
String input = " CREATE DICTIONARY dict4"
" ("
" key_column UInt64,"
" second_column UInt8,"
" third_column UInt8"
" )"
" PRIMARY KEY key_column"
" SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))"
" LAYOUT(CACHE(size_in_cells 50))"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
EXPECT_EQ(create->table, "dict4");
EXPECT_EQ(create->database, "");
/// source test
EXPECT_EQ(create->dictionary->source->name, "mysql");
auto children = create->dictionary->source->elements->children;
EXPECT_EQ(children[0]->as<ASTPair>()->first, "host");
EXPECT_EQ(children[0]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "localhost");
EXPECT_EQ(children[1]->as<ASTPair>()->first, "port");
EXPECT_EQ(children[1]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<UInt64>(), 9000);
EXPECT_EQ(children[2]->as<ASTPair>()->first, "user");
EXPECT_EQ(children[2]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "default");
EXPECT_EQ(children[3]->as<ASTPair>()->first, "replica");
auto replica = children[3]->as<ASTPair>()->second->children;
EXPECT_EQ(replica[0]->as<ASTPair>()->first, "host");
EXPECT_EQ(replica[0]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "127.0.0.1");
EXPECT_EQ(replica[1]->as<ASTPair>()->first, "priority");
EXPECT_EQ(replica[1]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<UInt64>(), 1);
EXPECT_EQ(children[4]->as<ASTPair>()->first, "password");
EXPECT_EQ(children[4]->as<ASTPair>()->second->as<ASTLiteral>()->value.get<String>(), "");
}
TEST(ParserDictionaryDDL, Formatting)
{
String input = " CREATE DICTIONARY test.dict5"
" ("
" key_column1 UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE,"
" key_column2 String DEFAULT '',"
" second_column UInt8 EXPRESSION intDiv(50, rand() % 1000),"
" third_column UInt8"
" )"
" PRIMARY KEY key_column1, key_column2"
" SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))"
" LAYOUT(CACHE(size_in_cells 50))"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
auto str = serializeAST(*create, true);
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '') LIFETIME(MIN 1, MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column, MAX third_column)");
}
TEST(ParserDictionaryDDL, ParseDropQuery)
{
String input1 = "DROP DICTIONARY test.dict1";
ParserDropQuery parser;
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0);
ASTDropQuery * drop1 = ast1->as<ASTDropQuery>();
EXPECT_TRUE(drop1->is_dictionary);
EXPECT_EQ(drop1->database, "test");
EXPECT_EQ(drop1->table, "dict1");
auto str1 = serializeAST(*drop1, true);
EXPECT_EQ(input1, str1);
String input2 = "DROP DICTIONARY IF EXISTS dict2";
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0);
ASTDropQuery * drop2 = ast2->as<ASTDropQuery>();
EXPECT_TRUE(drop2->is_dictionary);
EXPECT_EQ(drop2->database, "");
EXPECT_EQ(drop2->table, "dict2");
auto str2 = serializeAST(*drop2, true);
EXPECT_EQ(input2, str2);
}
TEST(ParserDictionaryDDL, ParsePropertiesQueries)
{
String input1 = "SHOW CREATE DICTIONARY test.dict1";
ParserTablePropertiesQuery parser;
ASTPtr ast1 = parseQuery(parser, input1.data(), input1.data() + input1.size(), "", 0);
ASTShowCreateDictionaryQuery * show1 = ast1->as<ASTShowCreateDictionaryQuery>();
EXPECT_EQ(show1->table, "dict1");
EXPECT_EQ(show1->database, "test");
EXPECT_EQ(serializeAST(*show1), input1);
String input2 = "EXISTS DICTIONARY dict2";
ASTPtr ast2 = parseQuery(parser, input2.data(), input2.data() + input2.size(), "", 0);
ASTExistsDictionaryQuery * show2 = ast2->as<ASTExistsDictionaryQuery>();
EXPECT_EQ(show2->table, "dict2");
EXPECT_EQ(show2->database, "");
EXPECT_EQ(serializeAST(*show2), input2);
}