mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
multiple TTL with GROUP BY
This commit is contained in:
parent
5822ee1f01
commit
61d6a323dd
@ -20,7 +20,7 @@ ASTPtr ASTTTLElement::clone() const
|
|||||||
|
|
||||||
for (auto & expr : clone->group_by_key)
|
for (auto & expr : clone->group_by_key)
|
||||||
expr = expr->clone();
|
expr = expr->clone();
|
||||||
for (auto & [name, expr] : clone->group_by_aggregations)
|
for (auto & expr : clone->group_by_assignments)
|
||||||
expr = expr->clone();
|
expr = expr->clone();
|
||||||
|
|
||||||
return clone;
|
return clone;
|
||||||
@ -46,15 +46,15 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st
|
|||||||
settings.ostr << ", ";
|
settings.ostr << ", ";
|
||||||
(*it)->formatImpl(settings, state, frame);
|
(*it)->formatImpl(settings, state, frame);
|
||||||
}
|
}
|
||||||
if (!group_by_aggregations.empty())
|
|
||||||
|
if (!group_by_assignments.empty())
|
||||||
{
|
{
|
||||||
settings.ostr << " SET ";
|
settings.ostr << " SET ";
|
||||||
for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it)
|
for (auto it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it)
|
||||||
{
|
{
|
||||||
if (it != group_by_aggregations.begin())
|
if (it != group_by_assignments.begin())
|
||||||
settings.ostr << ", ";
|
settings.ostr << ", ";
|
||||||
settings.ostr << it->first << " = ";
|
(*it)->formatImpl(settings, state, frame);
|
||||||
it->second->formatImpl(settings, state, frame);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -18,7 +18,7 @@ public:
|
|||||||
String destination_name;
|
String destination_name;
|
||||||
|
|
||||||
ASTs group_by_key;
|
ASTs group_by_key;
|
||||||
std::vector<std::pair<String, ASTPtr>> group_by_aggregations;
|
ASTs group_by_assignments;
|
||||||
|
|
||||||
ASTPtr recompression_codec;
|
ASTPtr recompression_codec;
|
||||||
|
|
||||||
|
@ -23,6 +23,7 @@
|
|||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||||
#include <Parsers/ASTColumnsTransformers.h>
|
#include <Parsers/ASTColumnsTransformers.h>
|
||||||
|
#include <Parsers/ASTAssignment.h>
|
||||||
|
|
||||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||||
#include <Parsers/parseIntervalKind.h>
|
#include <Parsers/parseIntervalKind.h>
|
||||||
@ -1875,9 +1876,12 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
ParserIdentifier parser_identifier;
|
ParserIdentifier parser_identifier;
|
||||||
ParserStringLiteral parser_string_literal;
|
ParserStringLiteral parser_string_literal;
|
||||||
ParserExpression parser_exp;
|
ParserExpression parser_exp;
|
||||||
ParserExpressionList parser_expression_list(false);
|
ParserExpressionList parser_keys_list(false);
|
||||||
ParserCodec parser_codec;
|
ParserCodec parser_codec;
|
||||||
|
|
||||||
|
ParserList parser_assignment_list(
|
||||||
|
std::make_unique<ParserAssignment>(), std::make_unique<ParserToken>(TokenType::Comma));
|
||||||
|
|
||||||
ASTPtr ttl_expr;
|
ASTPtr ttl_expr;
|
||||||
if (!parser_exp.parse(pos, ttl_expr, expected))
|
if (!parser_exp.parse(pos, ttl_expr, expected))
|
||||||
return false;
|
return false;
|
||||||
@ -1911,9 +1915,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
|
|
||||||
ASTPtr where_expr;
|
ASTPtr where_expr;
|
||||||
ASTPtr ast_group_by_key;
|
ASTPtr group_by_key;
|
||||||
ASTPtr recompression_codec;
|
ASTPtr recompression_codec;
|
||||||
std::vector<std::pair<String, ASTPtr>> group_by_aggregations;
|
ASTPtr group_by_assignments;
|
||||||
|
|
||||||
if (mode == TTLMode::MOVE)
|
if (mode == TTLMode::MOVE)
|
||||||
{
|
{
|
||||||
@ -1925,30 +1929,13 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
else if (mode == TTLMode::GROUP_BY)
|
else if (mode == TTLMode::GROUP_BY)
|
||||||
{
|
{
|
||||||
if (!parser_expression_list.parse(pos, ast_group_by_key, expected))
|
if (!parser_keys_list.parse(pos, group_by_key, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (s_set.ignore(pos))
|
if (s_set.ignore(pos))
|
||||||
{
|
{
|
||||||
while (true)
|
if (!parser_assignment_list.parse(pos, group_by_assignments, expected))
|
||||||
{
|
|
||||||
if (!group_by_aggregations.empty() && !s_comma.ignore(pos))
|
|
||||||
break;
|
|
||||||
|
|
||||||
ASTPtr name;
|
|
||||||
ASTPtr value;
|
|
||||||
if (!parser_identifier.parse(pos, name, expected))
|
|
||||||
return false;
|
return false;
|
||||||
if (!s_eq.ignore(pos))
|
|
||||||
return false;
|
|
||||||
if (!parser_exp.parse(pos, value, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
String name_str;
|
|
||||||
if (!tryGetIdentifierNameInto(name, name_str))
|
|
||||||
return false;
|
|
||||||
group_by_aggregations.emplace_back(name_str, std::move(value));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (mode == TTLMode::DELETE && s_where.ignore(pos))
|
else if (mode == TTLMode::DELETE && s_where.ignore(pos))
|
||||||
@ -1972,8 +1959,8 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
if (mode == TTLMode::GROUP_BY)
|
if (mode == TTLMode::GROUP_BY)
|
||||||
{
|
{
|
||||||
ttl_element->group_by_key = std::move(ast_group_by_key->children);
|
ttl_element->group_by_key = std::move(group_by_key->children);
|
||||||
ttl_element->group_by_aggregations = std::move(group_by_aggregations);
|
ttl_element->group_by_assignments = std::move(group_by_assignments->children);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (mode == TTLMode::RECOMPRESS)
|
if (mode == TTLMode::RECOMPRESS)
|
||||||
@ -2008,4 +1995,31 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node,
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
auto assignment = std::make_shared<ASTAssignment>();
|
||||||
|
node = assignment;
|
||||||
|
|
||||||
|
ParserIdentifier p_identifier;
|
||||||
|
ParserToken s_equals(TokenType::Equals);
|
||||||
|
ParserExpression p_expression;
|
||||||
|
|
||||||
|
ASTPtr column;
|
||||||
|
if (!p_identifier.parse(pos, column, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (!s_equals.ignore(pos, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
ASTPtr expression;
|
||||||
|
if (!p_expression.parse(pos, expression, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
tryGetIdentifierNameInto(column, assignment->column_name);
|
||||||
|
if (expression)
|
||||||
|
assignment->children.push_back(expression);
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -468,4 +468,12 @@ protected:
|
|||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/// Part of the UPDATE command or TTL with GROUP BY of the form: col_name = expr
|
||||||
|
class ParserAssignment : public IParserBase
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
const char * getName() const override{ return "column assignment"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -11,7 +11,6 @@
|
|||||||
#include <Parsers/ASTIndexDeclaration.h>
|
#include <Parsers/ASTIndexDeclaration.h>
|
||||||
#include <Parsers/ASTAlterQuery.h>
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTAssignment.h>
|
|
||||||
#include <Parsers/parseDatabaseAndTableName.h>
|
#include <Parsers/parseDatabaseAndTableName.h>
|
||||||
|
|
||||||
|
|
||||||
@ -651,34 +650,6 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|
||||||
{
|
|
||||||
auto assignment = std::make_shared<ASTAssignment>();
|
|
||||||
node = assignment;
|
|
||||||
|
|
||||||
ParserIdentifier p_identifier;
|
|
||||||
ParserToken s_equals(TokenType::Equals);
|
|
||||||
ParserExpression p_expression;
|
|
||||||
|
|
||||||
ASTPtr column;
|
|
||||||
if (!p_identifier.parse(pos, column, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (!s_equals.ignore(pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
ASTPtr expression;
|
|
||||||
if (!p_expression.parse(pos, expression, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
tryGetIdentifierNameInto(column, assignment->column_name);
|
|
||||||
if (expression)
|
|
||||||
assignment->children.push_back(expression);
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
auto query = std::make_shared<ASTAlterQuery>();
|
auto query = std::make_shared<ASTAlterQuery>();
|
||||||
|
@ -63,12 +63,4 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/// Part of the UPDATE command of the form: col_name = expr
|
|
||||||
class ParserAssignment : public IParserBase
|
|
||||||
{
|
|
||||||
protected:
|
|
||||||
const char * getName() const override{ return "column assignment"; }
|
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Storages/TTLDescription.h>
|
#include <Storages/TTLDescription.h>
|
||||||
|
|
||||||
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <Functions/IFunction.h>
|
#include <Functions/IFunction.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
@ -7,12 +8,13 @@
|
|||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTTTLElement.h>
|
#include <Parsers/ASTTTLElement.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
#include <Parsers/ASTAssignment.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Storages/ColumnsDescription.h>
|
#include <Storages/ColumnsDescription.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
|
|
||||||
#include <DataTypes/DataTypeDate.h>
|
#include <DataTypes/DataTypeDate.h>
|
||||||
#include <DataTypes/DataTypeDateTime.h>
|
#include <DataTypes/DataTypeDateTime.h>
|
||||||
|
|
||||||
@ -197,16 +199,31 @@ TTLDescription TTLDescription::getTTLFromAST(
|
|||||||
used_primary_key_columns_set.insert(pk_columns[i]);
|
used_primary_key_columns_set.insert(pk_columns[i]);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (const auto & [name, _] : ttl_element->group_by_aggregations)
|
std::vector<std::pair<String, ASTPtr>> aggregations;
|
||||||
|
for (const auto & ast : ttl_element->group_by_assignments)
|
||||||
|
{
|
||||||
|
const auto assignment = ast->as<const ASTAssignment &>();
|
||||||
|
auto expression = assignment.expression();
|
||||||
|
|
||||||
|
const auto * expression_func = expression->as<const ASTFunction>();
|
||||||
|
if (!expression_func || !AggregateFunctionFactory::instance().isAggregateFunctionName(expression_func->name))
|
||||||
|
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION,
|
||||||
|
"Invalid expression for assignment of column {}. Should be an aggregate function", assignment.column_name);
|
||||||
|
|
||||||
|
auto type_literal = std::make_shared<ASTLiteral>(columns.getPhysical(assignment.column_name).type->getName());
|
||||||
|
expression = makeASTFunction("cast", expression->clone(), type_literal);
|
||||||
|
aggregations.emplace_back(assignment.column_name, std::move(expression));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (const auto & [name, _] : aggregations)
|
||||||
aggregation_columns_set.insert(name);
|
aggregation_columns_set.insert(name);
|
||||||
|
|
||||||
if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size())
|
if (aggregation_columns_set.size() != ttl_element->group_by_assignments.size())
|
||||||
throw Exception(
|
throw Exception(
|
||||||
"Multiple aggregations set for one column in TTL Expression",
|
"Multiple aggregations set for one column in TTL Expression",
|
||||||
ErrorCodes::BAD_TTL_EXPRESSION);
|
ErrorCodes::BAD_TTL_EXPRESSION);
|
||||||
|
|
||||||
result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size());
|
result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size());
|
||||||
auto aggregations = ttl_element->group_by_aggregations;
|
|
||||||
|
|
||||||
const auto & primary_key_expressions = primary_key.expression_list_ast->children;
|
const auto & primary_key_expressions = primary_key.expression_list_ast->children;
|
||||||
for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i)
|
for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i)
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
TTL WHERE
|
||||||
1970-10-10 2
|
1970-10-10 2
|
||||||
1970-10-10 5
|
1970-10-10 5
|
||||||
1970-10-10 8
|
1970-10-10 8
|
||||||
@ -7,3 +8,15 @@
|
|||||||
2000-10-10 5
|
2000-10-10 5
|
||||||
2000-10-10 7
|
2000-10-10 7
|
||||||
2000-10-10 8
|
2000-10-10 8
|
||||||
|
TTL GROUP BY
|
||||||
|
1970-10-01 0 4950
|
||||||
|
2000-10-01 0 450
|
||||||
|
2000-10-01 1 460
|
||||||
|
2000-10-01 2 470
|
||||||
|
2000-10-01 3 480
|
||||||
|
2000-10-01 4 490
|
||||||
|
2000-10-01 5 500
|
||||||
|
2000-10-01 6 510
|
||||||
|
2000-10-01 7 520
|
||||||
|
2000-10-01 8 530
|
||||||
|
2000-10-01 9 540
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
SELECT 'TTL WHERE';
|
||||||
DROP TABLE IF EXISTS ttl_where;
|
DROP TABLE IF EXISTS ttl_where;
|
||||||
|
|
||||||
CREATE TABLE ttl_where
|
CREATE TABLE ttl_where
|
||||||
@ -10,11 +11,34 @@ ORDER BY tuple()
|
|||||||
TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0,
|
TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0,
|
||||||
d + toIntervalYear(40) DELETE WHERE i % 3 = 1;
|
d + toIntervalYear(40) DELETE WHERE i % 3 = 1;
|
||||||
|
|
||||||
|
-- This test will fail at 2040-10-10
|
||||||
|
|
||||||
INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10);
|
INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10);
|
||||||
INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10);
|
INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10);
|
||||||
|
|
||||||
OPTIMIZE TABLE ttl_where FINAL;
|
OPTIMIZE TABLE ttl_where FINAL;
|
||||||
|
|
||||||
SELECT * FROM ttl_where ORDER BY d, i;
|
SELECT * FROM ttl_where ORDER BY d, i;
|
||||||
|
|
||||||
DROP TABLE ttl_where;
|
DROP TABLE ttl_where;
|
||||||
|
|
||||||
|
SELECT 'TTL GROUP BY';
|
||||||
|
DROP TABLE IF EXISTS ttl_group_by;
|
||||||
|
|
||||||
|
CREATE TABLE ttl_group_by
|
||||||
|
(
|
||||||
|
`d` Date,
|
||||||
|
`i` UInt32,
|
||||||
|
`v` UInt64
|
||||||
|
)
|
||||||
|
ENGINE = MergeTree
|
||||||
|
ORDER BY (toStartOfMonth(d), i % 10)
|
||||||
|
TTL d + toIntervalYear(10) GROUP BY toStartOfMonth(d), i % 10 SET d = any(toStartOfMonth(d)), i = any(i % 10), v = sum(v),
|
||||||
|
d + toIntervalYear(40) GROUP BY toStartOfMonth(d) SET d = any(toStartOfMonth(d)), v = sum(v);
|
||||||
|
|
||||||
|
INSERT INTO ttl_group_by SELECT toDate('2000-10-10'), number, number FROM numbers(100);
|
||||||
|
INSERT INTO ttl_group_by SELECT toDate('1970-10-10'), number, number FROM numbers(100);
|
||||||
|
OPTIMIZE TABLE ttl_group_by FINAL;
|
||||||
|
|
||||||
|
SELECT * FROM ttl_group_by ORDER BY d, i;
|
||||||
|
|
||||||
|
DROP TABLE ttl_group_by;
|
||||||
|
Loading…
Reference in New Issue
Block a user