mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-11 18:14:03 +00:00
parse PARTITION BY, ORDER BY, SAMPLE BY, SETTINGS clauses in a storage definition [#CLICKHOUSE-3000]
This commit is contained in:
parent
48dea685d0
commit
6b9ead4bd8
@ -87,12 +87,14 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
const ASTFunction & engine = *create.storage->engine;
|
const ASTStorage & storage = *create.storage;
|
||||||
|
const ASTFunction & engine = *storage.engine;
|
||||||
/// Currently, there are no database engines, that support any arguments.
|
/// Currently, there are no database engines, that support any arguments.
|
||||||
if (engine.arguments || engine.parameters)
|
if (engine.arguments || engine.parameters
|
||||||
|
|| storage.partition_by || storage.order_by || storage.sample_by || storage.settings)
|
||||||
{
|
{
|
||||||
std::stringstream ostr;
|
std::stringstream ostr;
|
||||||
formatAST(*create.storage, ostr, 0, false, false);
|
formatAST(storage, ostr, 0, false, false);
|
||||||
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -15,6 +15,10 @@ class ASTStorage : public IAST
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ASTFunction * engine = nullptr;
|
ASTFunction * engine = nullptr;
|
||||||
|
IAST * partition_by = nullptr;
|
||||||
|
IAST * order_by = nullptr;
|
||||||
|
IAST * sample_by = nullptr;
|
||||||
|
ASTSetQuery * settings = nullptr;
|
||||||
|
|
||||||
ASTStorage() = default;
|
ASTStorage() = default;
|
||||||
ASTStorage(StringRange range_) : IAST(range_) {}
|
ASTStorage(StringRange range_) : IAST(range_) {}
|
||||||
@ -27,6 +31,14 @@ public:
|
|||||||
|
|
||||||
if (engine)
|
if (engine)
|
||||||
res->set(res->engine, engine->clone());
|
res->set(res->engine, engine->clone());
|
||||||
|
if (partition_by)
|
||||||
|
res->set(res->partition_by, partition_by->clone());
|
||||||
|
if (order_by)
|
||||||
|
res->set(res->order_by, order_by->clone());
|
||||||
|
if (sample_by)
|
||||||
|
res->set(res->sample_by, sample_by->clone());
|
||||||
|
if (settings)
|
||||||
|
res->set(res->settings, settings->clone());
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
@ -38,6 +50,27 @@ public:
|
|||||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ENGINE" << (s.hilite ? hilite_none : "") << " = ";
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ENGINE" << (s.hilite ? hilite_none : "") << " = ";
|
||||||
engine->formatImpl(s, state, frame);
|
engine->formatImpl(s, state, frame);
|
||||||
}
|
}
|
||||||
|
if (partition_by)
|
||||||
|
{
|
||||||
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "PARTITION BY " << (s.hilite ? hilite_none : "");
|
||||||
|
partition_by->formatImpl(s, state, frame);
|
||||||
|
}
|
||||||
|
if (order_by)
|
||||||
|
{
|
||||||
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "ORDER BY " << (s.hilite ? hilite_none : "");
|
||||||
|
order_by->formatImpl(s, state, frame);
|
||||||
|
}
|
||||||
|
if (sample_by)
|
||||||
|
{
|
||||||
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SAMPLE BY " << (s.hilite ? hilite_none : "");
|
||||||
|
sample_by->formatImpl(s, state, frame);
|
||||||
|
}
|
||||||
|
if (settings)
|
||||||
|
{
|
||||||
|
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : "");
|
||||||
|
settings->formatImpl(s, state, frame);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/ParserSelectQuery.h>
|
#include <Parsers/ParserSelectQuery.h>
|
||||||
|
#include <Parsers/ParserSetQuery.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -110,12 +111,22 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
{
|
{
|
||||||
ParserKeyword s_engine("ENGINE");
|
ParserKeyword s_engine("ENGINE");
|
||||||
ParserToken s_eq(TokenType::Equals);
|
ParserToken s_eq(TokenType::Equals);
|
||||||
|
ParserKeyword s_partition_by("PARTITION BY");
|
||||||
|
ParserKeyword s_order_by("ORDER BY");
|
||||||
|
ParserKeyword s_sample_by("SAMPLE BY");
|
||||||
|
ParserKeyword s_settings("SETTINGS");
|
||||||
|
|
||||||
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
|
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
|
||||||
|
ParserExpression expression_p;
|
||||||
|
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
|
||||||
|
|
||||||
Pos begin = pos;
|
Pos begin = pos;
|
||||||
|
|
||||||
ASTPtr engine;
|
ASTPtr engine;
|
||||||
|
ASTPtr partition_by;
|
||||||
|
ASTPtr order_by;
|
||||||
|
ASTPtr sample_by;
|
||||||
|
ASTPtr settings;
|
||||||
|
|
||||||
if (!s_engine.ignore(pos, expected))
|
if (!s_engine.ignore(pos, expected))
|
||||||
return false;
|
return false;
|
||||||
@ -125,8 +136,47 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
if (!ident_with_optional_params_p.parse(pos, engine, expected))
|
if (!ident_with_optional_params_p.parse(pos, engine, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
while (true)
|
||||||
|
{
|
||||||
|
if (!partition_by && s_partition_by.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
if (expression_p.parse(pos, partition_by, 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 (s_settings.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
if (!settings_p.parse(pos, settings, expected))
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
auto storage = std::make_shared<ASTStorage>(StringRange(begin, pos));
|
auto storage = std::make_shared<ASTStorage>(StringRange(begin, pos));
|
||||||
storage->set(storage->engine, engine);
|
storage->set(storage->engine, engine);
|
||||||
|
storage->set(storage->partition_by, partition_by);
|
||||||
|
storage->set(storage->order_by, order_by);
|
||||||
|
storage->set(storage->sample_by, sample_by);
|
||||||
|
storage->set(storage->settings, settings);
|
||||||
|
|
||||||
node = storage;
|
node = storage;
|
||||||
return true;
|
return true;
|
||||||
|
@ -187,7 +187,7 @@ protected:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/** ENGINE = name. */
|
/** ENGINE = name [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name = value, ...] */
|
||||||
class ParserStorage : public IParserBase
|
class ParserStorage : public IParserBase
|
||||||
{
|
{
|
||||||
protected:
|
protected:
|
||||||
|
@ -275,10 +275,28 @@ StoragePtr StorageFactory::get(
|
|||||||
|
|
||||||
if (engine_def.parameters)
|
if (engine_def.parameters)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Engine definition cannot take the form of a parametric function.", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
|
"Engine definition cannot take the form of a parametric function", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS);
|
||||||
|
|
||||||
|
if ((storage_def.partition_by || storage_def.order_by || storage_def.sample_by || storage_def.settings)
|
||||||
|
&& !endsWith(name, "MergeTree"))
|
||||||
|
{
|
||||||
|
throw Exception(
|
||||||
|
"Engine " + name + " doesn't support PARTITION BY, ORDER BY, SAMPLE BY or SETTINGS clauses. "
|
||||||
|
"Currently only the MergeTree family of engines supports them", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
}
|
||||||
|
|
||||||
|
auto check_arguments_empty = [&]
|
||||||
|
{
|
||||||
|
if (engine_def.arguments)
|
||||||
|
throw Exception(
|
||||||
|
"Engine " + name + " doesn't support any arguments (" + toString(engine_def.arguments->children.size()) + " given)",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
if (name == "Log")
|
if (name == "Log")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageLog::create(
|
return StorageLog::create(
|
||||||
data_path, table_name, columns,
|
data_path, table_name, columns,
|
||||||
materialized_columns, alias_columns, column_defaults,
|
materialized_columns, alias_columns, column_defaults,
|
||||||
@ -286,12 +304,14 @@ StoragePtr StorageFactory::get(
|
|||||||
}
|
}
|
||||||
else if (name == "View")
|
else if (name == "View")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageView::create(
|
return StorageView::create(
|
||||||
table_name, database_name, context, query, columns,
|
table_name, database_name, context, query, columns,
|
||||||
materialized_columns, alias_columns, column_defaults);
|
materialized_columns, alias_columns, column_defaults);
|
||||||
}
|
}
|
||||||
else if (name == "MaterializedView")
|
else if (name == "MaterializedView")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageMaterializedView::create(
|
return StorageMaterializedView::create(
|
||||||
table_name, database_name, context, query, columns,
|
table_name, database_name, context, query, columns,
|
||||||
materialized_columns, alias_columns, column_defaults,
|
materialized_columns, alias_columns, column_defaults,
|
||||||
@ -305,6 +325,7 @@ StoragePtr StorageFactory::get(
|
|||||||
}
|
}
|
||||||
else if (name == "TinyLog")
|
else if (name == "TinyLog")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageTinyLog::create(
|
return StorageTinyLog::create(
|
||||||
data_path, table_name, columns,
|
data_path, table_name, columns,
|
||||||
materialized_columns, alias_columns, column_defaults,
|
materialized_columns, alias_columns, column_defaults,
|
||||||
@ -312,6 +333,7 @@ StoragePtr StorageFactory::get(
|
|||||||
}
|
}
|
||||||
else if (name == "StripeLog")
|
else if (name == "StripeLog")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageStripeLog::create(
|
return StorageStripeLog::create(
|
||||||
data_path, table_name, columns,
|
data_path, table_name, columns,
|
||||||
materialized_columns, alias_columns, column_defaults,
|
materialized_columns, alias_columns, column_defaults,
|
||||||
@ -369,6 +391,7 @@ StoragePtr StorageFactory::get(
|
|||||||
}
|
}
|
||||||
else if (name == "Set")
|
else if (name == "Set")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageSet::create(
|
return StorageSet::create(
|
||||||
data_path, table_name, columns,
|
data_path, table_name, columns,
|
||||||
materialized_columns, alias_columns, column_defaults);
|
materialized_columns, alias_columns, column_defaults);
|
||||||
@ -432,6 +455,7 @@ StoragePtr StorageFactory::get(
|
|||||||
}
|
}
|
||||||
else if (name == "Memory")
|
else if (name == "Memory")
|
||||||
{
|
{
|
||||||
|
check_arguments_empty();
|
||||||
return StorageMemory::create(table_name, columns, materialized_columns, alias_columns, column_defaults);
|
return StorageMemory::create(table_name, columns, materialized_columns, alias_columns, column_defaults);
|
||||||
}
|
}
|
||||||
else if (name == "Null")
|
else if (name == "Null")
|
||||||
|
Loading…
Reference in New Issue
Block a user