mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
allow arbitrary partition literals in ALTER PARTITION queries [#CLICKHOUSE-3000]
This commit is contained in:
parent
f0995f2741
commit
68cb592277
@ -48,7 +48,7 @@ public:
|
|||||||
setEmpty();
|
setEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
AutoArray(size_t size_)
|
explicit AutoArray(size_t size_)
|
||||||
{
|
{
|
||||||
init(size_, false);
|
init(size_, false);
|
||||||
}
|
}
|
||||||
|
@ -253,7 +253,7 @@ namespace ErrorCodes
|
|||||||
extern const int INVALID_NESTED_NAME = 245;
|
extern const int INVALID_NESTED_NAME = 245;
|
||||||
extern const int CORRUPTED_DATA = 246;
|
extern const int CORRUPTED_DATA = 246;
|
||||||
extern const int INCORRECT_MARK = 247;
|
extern const int INCORRECT_MARK = 247;
|
||||||
extern const int INVALID_PARTITION_NAME = 248;
|
extern const int INVALID_PARTITION_VALUE = 248;
|
||||||
extern const int NOT_ENOUGH_BLOCK_NUMBERS = 250;
|
extern const int NOT_ENOUGH_BLOCK_NUMBERS = 250;
|
||||||
extern const int NO_SUCH_REPLICA = 251;
|
extern const int NO_SUCH_REPLICA = 251;
|
||||||
extern const int TOO_MUCH_PARTS = 252;
|
extern const int TOO_MUCH_PARTS = 252;
|
||||||
|
@ -60,19 +60,19 @@ BlockIO InterpreterAlterQuery::execute()
|
|||||||
switch (command.type)
|
switch (command.type)
|
||||||
{
|
{
|
||||||
case PartitionCommand::DROP_PARTITION:
|
case PartitionCommand::DROP_PARTITION:
|
||||||
table->dropPartition(query_ptr, command.partition, command.detach, context.getSettingsRef());
|
table->dropPartition(query_ptr, command.partition, command.detach, context);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case PartitionCommand::ATTACH_PARTITION:
|
case PartitionCommand::ATTACH_PARTITION:
|
||||||
table->attachPartition(query_ptr, command.partition, command.part, context.getSettingsRef());
|
table->attachPartition(command.partition, command.part, context);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case PartitionCommand::FETCH_PARTITION:
|
case PartitionCommand::FETCH_PARTITION:
|
||||||
table->fetchPartition(command.partition, command.from, context.getSettingsRef());
|
table->fetchPartition(command.partition, command.from, context);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case PartitionCommand::FREEZE_PARTITION:
|
case PartitionCommand::FREEZE_PARTITION:
|
||||||
table->freezePartition(command.partition, command.with_name, context.getSettingsRef());
|
table->freezePartition(command.partition, command.with_name, context);
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case PartitionCommand::RESHARD_PARTITION:
|
case PartitionCommand::RESHARD_PARTITION:
|
||||||
@ -82,7 +82,7 @@ BlockIO InterpreterAlterQuery::execute()
|
|||||||
break;
|
break;
|
||||||
|
|
||||||
case PartitionCommand::CLEAR_COLUMN:
|
case PartitionCommand::CLEAR_COLUMN:
|
||||||
table->clearColumnInPartition(query_ptr, command.partition, command.column_name, context.getSettingsRef());
|
table->clearColumnInPartition(command.partition, command.column_name, context);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -136,10 +136,9 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
if (!params.clear_column)
|
if (!params.clear_column)
|
||||||
throw Exception("Can't DROP COLUMN from partition. It is possible only CLEAR COLUMN in partition", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("Can't DROP COLUMN from partition. It is possible only CLEAR COLUMN in partition", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
const Field & partition = typeid_cast<const ASTLiteral &>(*(params.partition)).value;
|
|
||||||
const Field & column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
|
const Field & column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
|
||||||
|
|
||||||
out_partition_commands.emplace_back(PartitionCommand::clearColumn(partition, column_name));
|
out_partition_commands.emplace_back(PartitionCommand::clearColumn(params.partition, column_name));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -185,30 +184,22 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
}
|
}
|
||||||
else if (params.type == ASTAlterQuery::DROP_PARTITION)
|
else if (params.type == ASTAlterQuery::DROP_PARTITION)
|
||||||
{
|
{
|
||||||
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
out_partition_commands.emplace_back(PartitionCommand::dropPartition(params.partition, params.detach));
|
||||||
out_partition_commands.emplace_back(PartitionCommand::dropPartition(partition, params.detach));
|
|
||||||
}
|
}
|
||||||
else if (params.type == ASTAlterQuery::ATTACH_PARTITION)
|
else if (params.type == ASTAlterQuery::ATTACH_PARTITION)
|
||||||
{
|
{
|
||||||
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
out_partition_commands.emplace_back(PartitionCommand::attachPartition(params.partition, params.part));
|
||||||
out_partition_commands.emplace_back(PartitionCommand::attachPartition(partition, params.part));
|
|
||||||
}
|
}
|
||||||
else if (params.type == ASTAlterQuery::FETCH_PARTITION)
|
else if (params.type == ASTAlterQuery::FETCH_PARTITION)
|
||||||
{
|
{
|
||||||
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
out_partition_commands.emplace_back(PartitionCommand::fetchPartition(params.partition, params.from));
|
||||||
out_partition_commands.emplace_back(PartitionCommand::fetchPartition(partition, params.from));
|
|
||||||
}
|
}
|
||||||
else if (params.type == ASTAlterQuery::FREEZE_PARTITION)
|
else if (params.type == ASTAlterQuery::FREEZE_PARTITION)
|
||||||
{
|
{
|
||||||
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
out_partition_commands.emplace_back(PartitionCommand::freezePartition(params.partition, params.with_name));
|
||||||
out_partition_commands.emplace_back(PartitionCommand::freezePartition(partition, params.with_name));
|
|
||||||
}
|
}
|
||||||
else if (params.type == ASTAlterQuery::RESHARD_PARTITION)
|
else if (params.type == ASTAlterQuery::RESHARD_PARTITION)
|
||||||
{
|
{
|
||||||
Field partition;
|
|
||||||
if (params.partition)
|
|
||||||
partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
|
||||||
|
|
||||||
WeightedZooKeeperPaths weighted_zookeeper_paths;
|
WeightedZooKeeperPaths weighted_zookeeper_paths;
|
||||||
|
|
||||||
const ASTs & ast_weighted_zookeeper_paths = typeid_cast<const ASTExpressionList &>(*params.weighted_zookeeper_paths).children;
|
const ASTs & ast_weighted_zookeeper_paths = typeid_cast<const ASTExpressionList &>(*params.weighted_zookeeper_paths).children;
|
||||||
@ -223,7 +214,7 @@ void InterpreterAlterQuery::parseAlter(
|
|||||||
coordinator = dynamic_cast<const ASTLiteral &>(*params.coordinator).value;
|
coordinator = dynamic_cast<const ASTLiteral &>(*params.coordinator).value;
|
||||||
|
|
||||||
out_partition_commands.emplace_back(PartitionCommand::reshardPartitions(
|
out_partition_commands.emplace_back(PartitionCommand::reshardPartitions(
|
||||||
partition, weighted_zookeeper_paths, params.sharding_key_expr,
|
params.partition, weighted_zookeeper_paths, params.sharding_key_expr,
|
||||||
params.do_copy, coordinator));
|
params.do_copy, coordinator));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -35,7 +35,7 @@ private:
|
|||||||
|
|
||||||
Type type;
|
Type type;
|
||||||
|
|
||||||
Field partition;
|
ASTPtr partition;
|
||||||
Field column_name;
|
Field column_name;
|
||||||
bool detach = false; /// true for DETACH PARTITION.
|
bool detach = false; /// true for DETACH PARTITION.
|
||||||
|
|
||||||
@ -52,7 +52,7 @@ private:
|
|||||||
/// For FREEZE PARTITION
|
/// For FREEZE PARTITION
|
||||||
String with_name;
|
String with_name;
|
||||||
|
|
||||||
static PartitionCommand dropPartition(const Field & partition, bool detach)
|
static PartitionCommand dropPartition(const ASTPtr & partition, bool detach)
|
||||||
{
|
{
|
||||||
PartitionCommand res;
|
PartitionCommand res;
|
||||||
res.type = DROP_PARTITION;
|
res.type = DROP_PARTITION;
|
||||||
@ -61,7 +61,7 @@ private:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
static PartitionCommand clearColumn(const Field & partition, const Field & column_name)
|
static PartitionCommand clearColumn(const ASTPtr & partition, const Field & column_name)
|
||||||
{
|
{
|
||||||
PartitionCommand res;
|
PartitionCommand res;
|
||||||
res.type = CLEAR_COLUMN;
|
res.type = CLEAR_COLUMN;
|
||||||
@ -70,7 +70,7 @@ private:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
static PartitionCommand attachPartition(const Field & partition, bool part)
|
static PartitionCommand attachPartition(const ASTPtr & partition, bool part)
|
||||||
{
|
{
|
||||||
PartitionCommand res;
|
PartitionCommand res;
|
||||||
res.type = ATTACH_PARTITION;
|
res.type = ATTACH_PARTITION;
|
||||||
@ -79,7 +79,7 @@ private:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
static PartitionCommand fetchPartition(const Field & partition, const String & from)
|
static PartitionCommand fetchPartition(const ASTPtr & partition, const String & from)
|
||||||
{
|
{
|
||||||
PartitionCommand res;
|
PartitionCommand res;
|
||||||
res.type = FETCH_PARTITION;
|
res.type = FETCH_PARTITION;
|
||||||
@ -88,7 +88,7 @@ private:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
static PartitionCommand freezePartition(const Field & partition, const String & with_name)
|
static PartitionCommand freezePartition(const ASTPtr & partition, const String & with_name)
|
||||||
{
|
{
|
||||||
PartitionCommand res;
|
PartitionCommand res;
|
||||||
res.type = FREEZE_PARTITION;
|
res.type = FREEZE_PARTITION;
|
||||||
@ -97,7 +97,7 @@ private:
|
|||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
static PartitionCommand reshardPartitions(const Field & partition_,
|
static PartitionCommand reshardPartitions(const ASTPtr & partition_,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths_, const ASTPtr & sharding_key_expr_,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths_, const ASTPtr & sharding_key_expr_,
|
||||||
bool do_copy_, const Field & coordinator_)
|
bool do_copy_, const Field & coordinator_)
|
||||||
{
|
{
|
||||||
|
@ -18,12 +18,12 @@ BlockIO InterpreterOptimizeQuery::execute()
|
|||||||
{
|
{
|
||||||
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
|
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
|
||||||
|
|
||||||
if (ast.final && ast.partition.empty())
|
if (ast.final && !ast.partition)
|
||||||
throw Exception("FINAL flag for OPTIMIZE query is meaningful only with specified PARTITION", ErrorCodes::BAD_ARGUMENTS);
|
throw Exception("FINAL flag for OPTIMIZE query is meaningful only with specified PARTITION", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
StoragePtr table = context.getTable(ast.database, ast.table);
|
StoragePtr table = context.getTable(ast.database, ast.table);
|
||||||
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
|
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
|
||||||
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context.getSettings());
|
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context);
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -60,9 +60,10 @@ public:
|
|||||||
*/
|
*/
|
||||||
ASTPtr primary_key;
|
ASTPtr primary_key;
|
||||||
|
|
||||||
/** In DROP PARTITION and RESHARD PARTITION queries, the name of the partition is stored here.
|
/** In DROP PARTITION and RESHARD PARTITION queries, the value or ID of the partition is stored here.
|
||||||
*/
|
*/
|
||||||
ASTPtr partition;
|
ASTPtr partition;
|
||||||
|
|
||||||
bool detach = false; /// true for DETACH PARTITION
|
bool detach = false; /// true for DETACH PARTITION
|
||||||
|
|
||||||
bool part = false; /// true for ATTACH PART
|
bool part = false; /// true for ATTACH PART
|
||||||
|
@ -16,7 +16,7 @@ public:
|
|||||||
String table;
|
String table;
|
||||||
|
|
||||||
/// The partition to optimize can be specified.
|
/// The partition to optimize can be specified.
|
||||||
String partition;
|
ASTPtr partition;
|
||||||
/// A flag can be specified - perform optimization "to the end" instead of one step.
|
/// A flag can be specified - perform optimization "to the end" instead of one step.
|
||||||
bool final;
|
bool final;
|
||||||
/// Do deduplicate (default: false)
|
/// Do deduplicate (default: false)
|
||||||
@ -26,9 +26,21 @@ public:
|
|||||||
ASTOptimizeQuery(const StringRange range_) : IAST(range_) {}
|
ASTOptimizeQuery(const StringRange range_) : IAST(range_) {}
|
||||||
|
|
||||||
/** Get the text that identifies this element. */
|
/** Get the text that identifies this element. */
|
||||||
String getID() const override { return "OptimizeQuery_" + database + "_" + table + "_" + partition + (final ? "_final" : "") + (deduplicate ? "_deduplicate" : ""); };
|
String getID() const override { return "OptimizeQuery_" + database + "_" + table + (final ? "_final" : "") + (deduplicate ? "_deduplicate" : ""); };
|
||||||
|
|
||||||
ASTPtr clone() const override { return std::make_shared<ASTOptimizeQuery>(*this); }
|
ASTPtr clone() const override
|
||||||
|
{
|
||||||
|
auto res = std::make_shared<ASTOptimizeQuery>(*this);
|
||||||
|
res->children.clear();
|
||||||
|
|
||||||
|
if (partition)
|
||||||
|
{
|
||||||
|
res->partition = partition->clone();
|
||||||
|
res->children.push_back(res->partition);
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||||
@ -36,9 +48,11 @@ protected:
|
|||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "")
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "")
|
||||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
||||||
|
|
||||||
if (!partition.empty())
|
if (partition)
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " PARTITION " << (settings.hilite ? hilite_none : "")
|
{
|
||||||
<< partition;
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " PARTITION " << (settings.hilite ? hilite_none : "");
|
||||||
|
partition->formatImpl(settings, state, frame);
|
||||||
|
}
|
||||||
|
|
||||||
if (final)
|
if (final)
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FINAL" << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FINAL" << (settings.hilite ? hilite_none : "");
|
||||||
|
44
dbms/src/Parsers/ASTPartition.cpp
Normal file
44
dbms/src/Parsers/ASTPartition.cpp
Normal file
@ -0,0 +1,44 @@
|
|||||||
|
#include <Parsers/ASTPartition.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
String ASTPartition::getID() const
|
||||||
|
{
|
||||||
|
if (value)
|
||||||
|
return "Partition";
|
||||||
|
else
|
||||||
|
return "Partition_ID_" + id;
|
||||||
|
}
|
||||||
|
|
||||||
|
ASTPtr ASTPartition::clone() const
|
||||||
|
{
|
||||||
|
auto res = std::make_shared<ASTPartition>(*this);
|
||||||
|
res->children.clear();
|
||||||
|
|
||||||
|
if (value)
|
||||||
|
{
|
||||||
|
res->value = value->clone();
|
||||||
|
res->children.push_back(res->value);
|
||||||
|
}
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||||
|
{
|
||||||
|
if (value)
|
||||||
|
{
|
||||||
|
value->formatImpl(settings, state, frame);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
|
||||||
|
WriteBufferFromOwnString id_buf;
|
||||||
|
writeQuoted(id, id_buf);
|
||||||
|
settings.ostr << id_buf.str();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
29
dbms/src/Parsers/ASTPartition.h
Normal file
29
dbms/src/Parsers/ASTPartition.h
Normal file
@ -0,0 +1,29 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <common/StringRef.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Either a (possibly compound) expression representing a partition value or a partition ID.
|
||||||
|
class ASTPartition : public IAST
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
ASTPtr value;
|
||||||
|
StringRef fields_str; /// The extent of comma-separated partition expression fields without parentheses.
|
||||||
|
size_t fields_count = 0;
|
||||||
|
|
||||||
|
String id;
|
||||||
|
|
||||||
|
ASTPartition() = default;
|
||||||
|
ASTPartition(StringRange range_) : IAST(range_) {}
|
||||||
|
String getID() const override;
|
||||||
|
ASTPtr clone() const override;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -498,6 +498,8 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
|||||||
if (pos->type != TokenType::StringLiteral)
|
if (pos->type != TokenType::StringLiteral)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
|
Pos begin = pos;
|
||||||
|
|
||||||
String s;
|
String s;
|
||||||
ReadBufferFromMemory in(pos->begin, pos->size());
|
ReadBufferFromMemory in(pos->begin, pos->size());
|
||||||
|
|
||||||
@ -518,7 +520,7 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
|||||||
}
|
}
|
||||||
|
|
||||||
++pos;
|
++pos;
|
||||||
node = std::make_shared<ASTLiteral>(StringRange(pos->begin, pos->end), s);
|
node = std::make_shared<ASTLiteral>(StringRange(begin, pos), s);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2,8 +2,8 @@
|
|||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
|
#include <Parsers/ParserPartition.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
|
||||||
#include <Parsers/ASTAlterQuery.h>
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
|
||||||
@ -46,8 +46,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
ParserIdentifier table_parser;
|
ParserIdentifier table_parser;
|
||||||
ParserCompoundIdentifier parser_name;
|
ParserCompoundIdentifier parser_name;
|
||||||
ParserCompoundColumnDeclaration parser_col_decl;
|
ParserCompoundColumnDeclaration parser_col_decl;
|
||||||
ParserLiteral parser_literal;
|
ParserPartition parser_partition;
|
||||||
ParserUnsignedInteger parser_uint;
|
|
||||||
ParserStringLiteral parser_string_literal;
|
ParserStringLiteral parser_string_literal;
|
||||||
|
|
||||||
ASTPtr table;
|
ASTPtr table;
|
||||||
@ -106,7 +105,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
else if (s_drop_partition.ignore(pos, expected))
|
else if (s_drop_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
params.type = ASTAlterQuery::DROP_PARTITION;
|
params.type = ASTAlterQuery::DROP_PARTITION;
|
||||||
@ -130,13 +129,13 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
if (s_in_partition.ignore(pos, expected))
|
if (s_in_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else if (s_detach_partition.ignore(pos, expected))
|
else if (s_detach_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
params.type = ASTAlterQuery::DROP_PARTITION;
|
params.type = ASTAlterQuery::DROP_PARTITION;
|
||||||
@ -144,14 +143,14 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
else if (s_attach_partition.ignore(pos, expected))
|
else if (s_attach_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
params.type = ASTAlterQuery::ATTACH_PARTITION;
|
params.type = ASTAlterQuery::ATTACH_PARTITION;
|
||||||
}
|
}
|
||||||
else if (s_attach_part.ignore(pos, expected))
|
else if (s_attach_part.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_string_literal.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
params.part = true;
|
params.part = true;
|
||||||
@ -159,7 +158,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
else if (s_fetch_partition.ignore(pos, expected))
|
else if (s_fetch_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (!s_from.ignore(pos, expected))
|
if (!s_from.ignore(pos, expected))
|
||||||
@ -174,7 +173,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
}
|
}
|
||||||
else if (s_freeze_partition.ignore(pos, expected))
|
else if (s_freeze_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_literal.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
/// WITH NAME 'name' - place local backup to directory with specified name
|
/// WITH NAME 'name' - place local backup to directory with specified name
|
||||||
@ -225,7 +224,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
if (s_partition.ignore(pos, expected))
|
if (s_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (!parser_uint.parse(pos, params.partition, expected))
|
if (!parser_partition.parse(pos, params.partition, expected))
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,8 +1,9 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTOptimizeQuery.h>
|
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
|
||||||
#include <Parsers/ParserOptimizeQuery.h>
|
#include <Parsers/ParserOptimizeQuery.h>
|
||||||
|
#include <Parsers/ParserPartition.h>
|
||||||
|
#include <Parsers/CommonParsers.h>
|
||||||
|
|
||||||
|
#include <Parsers/ASTOptimizeQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
@ -22,7 +23,7 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
|||||||
ParserKeyword s_deduplicate("DEDUPLICATE");
|
ParserKeyword s_deduplicate("DEDUPLICATE");
|
||||||
ParserToken s_dot(TokenType::Dot);
|
ParserToken s_dot(TokenType::Dot);
|
||||||
ParserIdentifier name_p;
|
ParserIdentifier name_p;
|
||||||
ParserLiteral partition_p;
|
ParserPartition partition_p;
|
||||||
|
|
||||||
ASTPtr database;
|
ASTPtr database;
|
||||||
ASTPtr table;
|
ASTPtr table;
|
||||||
@ -62,8 +63,7 @@ bool ParserOptimizeQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte
|
|||||||
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
|
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
|
||||||
if (table)
|
if (table)
|
||||||
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
|
query->table = typeid_cast<const ASTIdentifier &>(*table).name;
|
||||||
if (partition)
|
query->partition = partition;
|
||||||
query->partition = applyVisitor(FieldVisitorToString(), typeid_cast<const ASTLiteral &>(*partition).value);
|
|
||||||
query->final = final;
|
query->final = final;
|
||||||
query->deduplicate = deduplicate;
|
query->deduplicate = deduplicate;
|
||||||
|
|
||||||
|
81
dbms/src/Parsers/ParserPartition.cpp
Normal file
81
dbms/src/Parsers/ParserPartition.cpp
Normal file
@ -0,0 +1,81 @@
|
|||||||
|
#include <Parsers/ParserPartition.h>
|
||||||
|
#include <Parsers/CommonParsers.h>
|
||||||
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
|
#include <Parsers/ASTPartition.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
ParserKeyword s_id("ID");
|
||||||
|
ParserStringLiteral parser_string_literal;
|
||||||
|
ParserExpression parser_expr;
|
||||||
|
|
||||||
|
Pos begin = pos;
|
||||||
|
|
||||||
|
auto partition = std::make_shared<ASTPartition>();
|
||||||
|
|
||||||
|
if (s_id.ignore(pos, expected))
|
||||||
|
{
|
||||||
|
ASTPtr partition_id;
|
||||||
|
if (!parser_string_literal.parse(pos, partition_id, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
partition->id = dynamic_cast<const ASTLiteral &>(*partition_id).value.get<String>();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
ASTPtr value;
|
||||||
|
if (!parser_expr.parse(pos, value, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
size_t fields_count;
|
||||||
|
StringRef fields_str;
|
||||||
|
|
||||||
|
const auto * tuple_ast = typeid_cast<const ASTFunction *>(value.get());
|
||||||
|
if (tuple_ast && tuple_ast->name == "tuple")
|
||||||
|
{
|
||||||
|
const auto * arguments_ast = dynamic_cast<const ASTExpressionList *>(tuple_ast->arguments.get());
|
||||||
|
if (arguments_ast)
|
||||||
|
fields_count = arguments_ast->children.size();
|
||||||
|
else
|
||||||
|
fields_count = 0;
|
||||||
|
|
||||||
|
Pos left_paren = begin;
|
||||||
|
Pos right_paren = pos;
|
||||||
|
|
||||||
|
while (left_paren != right_paren && left_paren->type != TokenType::OpeningRoundBracket)
|
||||||
|
++left_paren;
|
||||||
|
if (left_paren->type != TokenType::OpeningRoundBracket)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
while (right_paren != left_paren && right_paren->type != TokenType::ClosingRoundBracket)
|
||||||
|
--right_paren;
|
||||||
|
if (right_paren->type != TokenType::ClosingRoundBracket)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
fields_str = StringRef(left_paren->end, right_paren->begin - left_paren->end);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
fields_count = 1;
|
||||||
|
fields_str = StringRef(begin->begin, pos->begin - begin->begin);
|
||||||
|
}
|
||||||
|
|
||||||
|
partition->value = value;
|
||||||
|
partition->children.push_back(value);
|
||||||
|
partition->fields_str = fields_str;
|
||||||
|
partition->fields_count = fields_count;
|
||||||
|
}
|
||||||
|
|
||||||
|
partition->range = StringRange(begin, pos);
|
||||||
|
node = partition;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
17
dbms/src/Parsers/ParserPartition.h
Normal file
17
dbms/src/Parsers/ParserPartition.h
Normal file
@ -0,0 +1,17 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IParserBase.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Parse either a partition value as a (possibly compound) literal or a partition ID.
|
||||||
|
/// Produce ASTPartition.
|
||||||
|
class ParserPartition : public IParserBase
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
const char * getName() const override { return "partition"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -69,6 +69,7 @@ public:
|
|||||||
bool operator< (const TokenIterator & rhs) const { return index < rhs.index; }
|
bool operator< (const TokenIterator & rhs) const { return index < rhs.index; }
|
||||||
bool operator<= (const TokenIterator & rhs) const { return index <= rhs.index; }
|
bool operator<= (const TokenIterator & rhs) const { return index <= rhs.index; }
|
||||||
bool operator== (const TokenIterator & rhs) const { return index == rhs.index; }
|
bool operator== (const TokenIterator & rhs) const { return index == rhs.index; }
|
||||||
|
bool operator!= (const TokenIterator & rhs) const { return index != rhs.index; }
|
||||||
|
|
||||||
bool isValid() { return get().type < TokenType::EndOfStream; }
|
bool isValid() { return get().type < TokenType::EndOfStream; }
|
||||||
|
|
||||||
|
@ -221,35 +221,35 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/** Execute CLEAR COLUMN ... IN PARTITION query which removes column from given partition. */
|
/** Execute CLEAR COLUMN ... IN PARTITION query which removes column from given partition. */
|
||||||
virtual void clearColumnInPartition(const ASTPtr & query, const Field & partition, const Field & column_name, const Settings & settings)
|
virtual void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method dropColumnFromPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method dropColumnFromPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Run the query (DROP|DETACH) PARTITION.
|
/** Run the query (DROP|DETACH) PARTITION.
|
||||||
*/
|
*/
|
||||||
virtual void dropPartition(const ASTPtr & query, const Field & partition, bool detach, const Settings & settings)
|
virtual void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Run the ATTACH request (PART|PARTITION).
|
/** Run the ATTACH request (PART|PARTITION).
|
||||||
*/
|
*/
|
||||||
virtual void attachPartition(const ASTPtr & query, const Field & partition, bool part, const Settings & settings)
|
virtual void attachPartition(const ASTPtr & partition, bool part, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Run the FETCH PARTITION query.
|
/** Run the FETCH PARTITION query.
|
||||||
*/
|
*/
|
||||||
virtual void fetchPartition(const Field & partition, const String & from, const Settings & settings)
|
virtual void fetchPartition(const ASTPtr & partition, const String & from, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method fetchPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method fetchPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Run the FREEZE PARTITION request. That is, create a local backup (snapshot) of data using the `localBackup` function (see localBackup.h)
|
/** Run the FREEZE PARTITION request. That is, create a local backup (snapshot) of data using the `localBackup` function (see localBackup.h)
|
||||||
*/
|
*/
|
||||||
virtual void freezePartition(const Field & partition, const String & with_name, const Settings & settings)
|
virtual void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method freezePartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method freezePartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
@ -258,7 +258,7 @@ public:
|
|||||||
*/
|
*/
|
||||||
virtual void reshardPartitions(
|
virtual void reshardPartitions(
|
||||||
const ASTPtr & query, const String & database_name,
|
const ASTPtr & query, const String & database_name,
|
||||||
const Field & partition,
|
const ASTPtr & partition,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
||||||
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
||||||
const Context & context)
|
const Context & context)
|
||||||
@ -269,7 +269,7 @@ public:
|
|||||||
/** Perform any background work. For example, combining parts in a MergeTree type table.
|
/** Perform any background work. For example, combining parts in a MergeTree type table.
|
||||||
* Returns whether any work has been done.
|
* Returns whether any work has been done.
|
||||||
*/
|
*/
|
||||||
virtual bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings)
|
virtual bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
{
|
{
|
||||||
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
}
|
}
|
||||||
|
@ -6,13 +6,19 @@
|
|||||||
#include <Storages/AlterCommands.h>
|
#include <Storages/AlterCommands.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTNameTypePair.h>
|
#include <Parsers/ASTNameTypePair.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTPartition.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <DataStreams/ExpressionBlockInputStream.h>
|
#include <DataStreams/ExpressionBlockInputStream.h>
|
||||||
|
#include <DataStreams/ValuesRowInputStream.h>
|
||||||
#include <DataStreams/copyData.h>
|
#include <DataStreams/copyData.h>
|
||||||
#include <IO/WriteBufferFromFile.h>
|
#include <IO/WriteBufferFromFile.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <IO/CompressedReadBuffer.h>
|
#include <IO/CompressedReadBuffer.h>
|
||||||
|
#include <IO/ReadBufferFromMemory.h>
|
||||||
|
#include <IO/ConcatReadBuffer.h>
|
||||||
#include <IO/HexWriteBuffer.h>
|
#include <IO/HexWriteBuffer.h>
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <DataTypes/DataTypeDate.h>
|
#include <DataTypes/DataTypeDate.h>
|
||||||
@ -63,6 +69,7 @@ namespace ErrorCodes
|
|||||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||||
extern const int SYNTAX_ERROR;
|
extern const int SYNTAX_ERROR;
|
||||||
extern const int CORRUPTED_DATA;
|
extern const int CORRUPTED_DATA;
|
||||||
|
extern const int INVALID_PARTITION_VALUE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -1714,8 +1721,21 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void MergeTreeData::freezePartition(const std::string & prefix, const String & with_name)
|
void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context)
|
||||||
{
|
{
|
||||||
|
String prefix;
|
||||||
|
if (format_version == 0)
|
||||||
|
{
|
||||||
|
const auto & partition = dynamic_cast<const ASTPartition &>(*partition_ast);
|
||||||
|
/// Month-partitioning specific - allow partition ID can be passed in the partition value.
|
||||||
|
if (const auto * partition_lit = dynamic_cast<const ASTLiteral *>(partition.value.get()))
|
||||||
|
prefix = partition_lit->value.getType() == Field::Types::UInt64
|
||||||
|
? toString(partition_lit->value.get<UInt64>())
|
||||||
|
: partition_lit->value.safeGet<String>();
|
||||||
|
}
|
||||||
|
else
|
||||||
|
prefix = getPartitionIDFromQuery(partition_ast, context);
|
||||||
|
|
||||||
LOG_DEBUG(log, "Freezing parts with prefix " + prefix);
|
LOG_DEBUG(log, "Freezing parts with prefix " + prefix);
|
||||||
|
|
||||||
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();
|
String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString();
|
||||||
@ -1777,18 +1797,61 @@ size_t MergeTreeData::getPartitionSize(const std::string & partition_id) const
|
|||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
String MergeTreeData::getPartitionIDFromQuery(const Field & partition)
|
String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & partition_ast, const Context & context)
|
||||||
{
|
{
|
||||||
/// Month-partitioning specific, TODO: generalize.
|
const auto & partition = typeid_cast<const ASTPartition &>(*partition_ast);
|
||||||
String partition_id = partition.getType() == Field::Types::UInt64
|
|
||||||
? toString(partition.get<UInt64>())
|
|
||||||
: partition.safeGet<String>();
|
|
||||||
|
|
||||||
if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII))
|
if (!partition.value)
|
||||||
throw Exception("Invalid partition format: " + partition_id + ". Partition should consist of 6 digits: YYYYMM",
|
return partition.id;
|
||||||
ErrorCodes::INVALID_PARTITION_NAME);
|
|
||||||
|
|
||||||
return partition_id;
|
if (format_version == 0)
|
||||||
|
{
|
||||||
|
/// Month-partitioning specific - allow partition ID can be passed in the partition value.
|
||||||
|
const auto * partition_lit = typeid_cast<const ASTLiteral *>(partition.value.get());
|
||||||
|
if (partition_lit && partition_lit->value.getType() == Field::Types::String)
|
||||||
|
{
|
||||||
|
String partition_id = partition_lit->value.get<String>();
|
||||||
|
if (partition_id.size() != 6 || !std::all_of(partition_id.begin(), partition_id.end(), isNumericASCII))
|
||||||
|
throw Exception(
|
||||||
|
"Invalid partition format: " + partition_id + ". Partition should consist of 6 digits: YYYYMM",
|
||||||
|
ErrorCodes::INVALID_PARTITION_VALUE);
|
||||||
|
return partition_id;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Re-parse partition key fields using the information about expected field types.
|
||||||
|
|
||||||
|
size_t fields_count = partition_expr_column_types.size();
|
||||||
|
if (partition.fields_count != fields_count)
|
||||||
|
throw Exception(
|
||||||
|
"Wrong number of fields in the partition expression: " + toString(partition.fields_count) +
|
||||||
|
", must be: " + toString(fields_count),
|
||||||
|
ErrorCodes::INVALID_PARTITION_VALUE);
|
||||||
|
|
||||||
|
Row partition_row(fields_count);
|
||||||
|
|
||||||
|
if (fields_count)
|
||||||
|
{
|
||||||
|
ReadBufferFromMemory left_paren_buf("(", 1);
|
||||||
|
ReadBufferFromMemory fields_buf(partition.fields_str.data, partition.fields_str.size);
|
||||||
|
ReadBufferFromMemory right_paren_buf(")", 1);
|
||||||
|
ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf});
|
||||||
|
|
||||||
|
ValuesRowInputStream input_stream(buf, context, /* interpret_expressions = */true);
|
||||||
|
Block block;
|
||||||
|
for (size_t i = 0; i < fields_count; ++i)
|
||||||
|
block.insert(ColumnWithTypeAndName(partition_expr_column_types[i], partition_expr_columns[i]));
|
||||||
|
|
||||||
|
if (!input_stream.read(block))
|
||||||
|
throw Exception(
|
||||||
|
"Could not parse partition value: `" + partition.fields_str.toString() + "`",
|
||||||
|
ErrorCodes::INVALID_PARTITION_VALUE);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < fields_count; ++i)
|
||||||
|
block.getByPosition(i).column->get(0, partition_row[i]);
|
||||||
|
}
|
||||||
|
|
||||||
|
return MergeTreeDataPart::Partition(std::move(partition_row)).getID(*this);
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeData::Transaction::rollback()
|
void MergeTreeData::Transaction::rollback()
|
||||||
|
@ -410,7 +410,7 @@ public:
|
|||||||
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
|
* Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number,
|
||||||
* or if 'with_name' is specified - backup is created in directory with specified name.
|
* or if 'with_name' is specified - backup is created in directory with specified name.
|
||||||
*/
|
*/
|
||||||
void freezePartition(const std::string & prefix, const String & with_name);
|
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context);
|
||||||
|
|
||||||
/// Returns the size of partition in bytes.
|
/// Returns the size of partition in bytes.
|
||||||
size_t getPartitionSize(const std::string & partition_id) const;
|
size_t getPartitionSize(const std::string & partition_id) const;
|
||||||
@ -461,7 +461,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// For ATTACH/DETACH/DROP/RESHARD PARTITION.
|
/// For ATTACH/DETACH/DROP/RESHARD PARTITION.
|
||||||
String getPartitionIDFromQuery(const Field & partition);
|
String getPartitionIDFromQuery(const ASTPtr & partition, const Context & context);
|
||||||
|
|
||||||
MergeTreeDataFormatVersion format_version;
|
MergeTreeDataFormatVersion format_version;
|
||||||
|
|
||||||
|
@ -351,7 +351,18 @@ String MergeTreeDataPart::Partition::getID(const MergeTreeData & storage) const
|
|||||||
|
|
||||||
void MergeTreeDataPart::Partition::serializeTextQuoted(const MergeTreeData & storage, WriteBuffer & out) const
|
void MergeTreeDataPart::Partition::serializeTextQuoted(const MergeTreeData & storage, WriteBuffer & out) const
|
||||||
{
|
{
|
||||||
for (size_t i = 0; i < storage.partition_expr_column_types.size(); ++i)
|
size_t key_size = storage.partition_expr_column_types.size();
|
||||||
|
|
||||||
|
if (key_size == 0)
|
||||||
|
{
|
||||||
|
writeCString("tuple()", out);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (key_size > 1)
|
||||||
|
writeChar('(', out);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < key_size; ++i)
|
||||||
{
|
{
|
||||||
if (i > 0)
|
if (i > 0)
|
||||||
writeCString(", ", out);
|
writeCString(", ", out);
|
||||||
@ -361,6 +372,9 @@ void MergeTreeDataPart::Partition::serializeTextQuoted(const MergeTreeData & sto
|
|||||||
column->insert(value[i]);
|
column->insert(value[i]);
|
||||||
type->serializeTextQuoted(*column, 0, out);
|
type->serializeTextQuoted(*column, 0, out);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (key_size > 1)
|
||||||
|
writeChar(')', out);
|
||||||
}
|
}
|
||||||
|
|
||||||
void MergeTreeDataPart::Partition::load(const MergeTreeData & storage, const String & part_path)
|
void MergeTreeDataPart::Partition::load(const MergeTreeData & storage, const String & part_path)
|
||||||
|
@ -346,7 +346,7 @@ void StorageBuffer::shutdown()
|
|||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
optimize(nullptr /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context.getSettings());
|
optimize(nullptr /*query*/, {} /*partition*/, false /*final*/, false /*deduplicate*/, context);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
@ -365,9 +365,9 @@ void StorageBuffer::shutdown()
|
|||||||
*
|
*
|
||||||
* This kind of race condition make very hard to implement proper tests.
|
* This kind of race condition make very hard to implement proper tests.
|
||||||
*/
|
*/
|
||||||
bool StorageBuffer::optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings)
|
bool StorageBuffer::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
{
|
{
|
||||||
if (!partition_id.empty())
|
if (partition)
|
||||||
throw Exception("Partition cannot be specified when optimizing table of type Buffer", ErrorCodes::NOT_IMPLEMENTED);
|
throw Exception("Partition cannot be specified when optimizing table of type Buffer", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
if (final)
|
if (final)
|
||||||
@ -593,7 +593,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
|
|||||||
auto lock = lockStructureForAlter(__PRETTY_FUNCTION__);
|
auto lock = lockStructureForAlter(__PRETTY_FUNCTION__);
|
||||||
|
|
||||||
/// So that no blocks of the old structure remain.
|
/// So that no blocks of the old structure remain.
|
||||||
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context.getSettings());
|
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context);
|
||||||
|
|
||||||
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
|
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
|
||||||
|
|
||||||
|
@ -69,7 +69,7 @@ public:
|
|||||||
void startup() override;
|
void startup() override;
|
||||||
/// Flush all buffers into the subordinate table and stop background thread.
|
/// Flush all buffers into the subordinate table and stop background thread.
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings) override;
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||||
|
|
||||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
|
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
|
||||||
|
|
||||||
|
@ -270,7 +270,7 @@ void StorageDistributed::shutdown()
|
|||||||
|
|
||||||
void StorageDistributed::reshardPartitions(
|
void StorageDistributed::reshardPartitions(
|
||||||
const ASTPtr & query, const String & database_name,
|
const ASTPtr & query, const String & database_name,
|
||||||
const Field & partition,
|
const ASTPtr & partition,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
||||||
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
||||||
const Context & context)
|
const Context & context)
|
||||||
@ -322,8 +322,7 @@ void StorageDistributed::reshardPartitions(
|
|||||||
ASTAlterQuery::Parameters & parameters = alter_query.parameters.back();
|
ASTAlterQuery::Parameters & parameters = alter_query.parameters.back();
|
||||||
|
|
||||||
parameters.type = ASTAlterQuery::RESHARD_PARTITION;
|
parameters.type = ASTAlterQuery::RESHARD_PARTITION;
|
||||||
if (!partition.isNull())
|
parameters.partition = partition->clone();
|
||||||
parameters.partition = std::make_shared<ASTLiteral>(StringRange(), partition);
|
|
||||||
|
|
||||||
ASTPtr expr_list = std::make_shared<ASTExpressionList>();
|
ASTPtr expr_list = std::make_shared<ASTExpressionList>();
|
||||||
for (const auto & entry : weighted_zookeeper_paths)
|
for (const auto & entry : weighted_zookeeper_paths)
|
||||||
|
@ -76,7 +76,7 @@ public:
|
|||||||
|
|
||||||
void reshardPartitions(
|
void reshardPartitions(
|
||||||
const ASTPtr & query, const String & database_name,
|
const ASTPtr & query, const String & database_name,
|
||||||
const Field & partition,
|
const ASTPtr & partition,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
||||||
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
||||||
const Context & context) override;
|
const Context & context) override;
|
||||||
|
@ -166,9 +166,9 @@ void StorageMaterializedView::drop()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool StorageMaterializedView::optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings)
|
bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
{
|
{
|
||||||
return getInnerTable()->optimize(query, partition_id, final, deduplicate, settings);
|
return getInnerTable()->optimize(query, partition, final, deduplicate, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr StorageMaterializedView::getInnerTable() const
|
StoragePtr StorageMaterializedView::getInnerTable() const
|
||||||
|
@ -35,7 +35,7 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
||||||
void drop() override;
|
void drop() override;
|
||||||
bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings) override;
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||||
|
|
||||||
BlockInputStreams read(
|
BlockInputStreams read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
#include <Interpreters/InterpreterAlterQuery.h>
|
#include <Interpreters/InterpreterAlterQuery.h>
|
||||||
#include <Interpreters/PartLog.h>
|
#include <Interpreters/PartLog.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||||
@ -396,7 +397,7 @@ bool StorageMergeTree::mergeTask()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageMergeTree::clearColumnInPartition(const ASTPtr & query, const Field & partition, const Field & column_name, const Settings &)
|
void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context)
|
||||||
{
|
{
|
||||||
/// Asks to complete merges and does not allow them to start.
|
/// Asks to complete merges and does not allow them to start.
|
||||||
/// This protects against "revival" of data for a removed partition after completion of merge.
|
/// This protects against "revival" of data for a removed partition after completion of merge.
|
||||||
@ -405,7 +406,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & query, const Field
|
|||||||
/// We don't change table structure, only data in some parts, parts are locked inside alterDataPart() function
|
/// We don't change table structure, only data in some parts, parts are locked inside alterDataPart() function
|
||||||
auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
|
auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
|
||||||
|
|
||||||
String partition_id = data.getPartitionIDFromQuery(partition);
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
MergeTreeData::DataParts parts = data.getDataParts();
|
MergeTreeData::DataParts parts = data.getDataParts();
|
||||||
|
|
||||||
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
std::vector<MergeTreeData::AlterDataPartTransactionPtr> transactions;
|
||||||
@ -446,7 +447,17 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & query, const Field
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageMergeTree::dropPartition(const ASTPtr & query, const Field & partition, bool detach, const Settings & settings)
|
bool StorageMergeTree::optimize(
|
||||||
|
const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
|
{
|
||||||
|
String partition_id;
|
||||||
|
if (partition)
|
||||||
|
partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
|
return merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, final, deduplicate);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void StorageMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
|
||||||
{
|
{
|
||||||
/// Asks to complete merges and does not allow them to start.
|
/// Asks to complete merges and does not allow them to start.
|
||||||
/// This protects against "revival" of data for a removed partition after completion of merge.
|
/// This protects against "revival" of data for a removed partition after completion of merge.
|
||||||
@ -454,7 +465,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & query, const Field & partiti
|
|||||||
/// Waits for completion of merge and does not start new ones.
|
/// Waits for completion of merge and does not start new ones.
|
||||||
auto lock = lockForAlter(__PRETTY_FUNCTION__);
|
auto lock = lockForAlter(__PRETTY_FUNCTION__);
|
||||||
|
|
||||||
String partition_id = data.getPartitionIDFromQuery(partition);
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
|
|
||||||
size_t removed_parts = 0;
|
size_t removed_parts = 0;
|
||||||
MergeTreeData::DataParts parts = data.getDataParts();
|
MergeTreeData::DataParts parts = data.getDataParts();
|
||||||
@ -473,18 +484,18 @@ void StorageMergeTree::dropPartition(const ASTPtr & query, const Field & partiti
|
|||||||
data.replaceParts({part}, {}, false);
|
data.replaceParts({part}, {}, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << applyVisitor(FieldVisitorToString(), partition) << ".");
|
LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " parts inside partition ID " << partition_id << ".");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageMergeTree::attachPartition(const ASTPtr & query, const Field & field, bool part, const Settings & settings)
|
void StorageMergeTree::attachPartition(const ASTPtr & partition, bool part, const Context & context)
|
||||||
{
|
{
|
||||||
String partition_id;
|
String partition_id;
|
||||||
|
|
||||||
if (part)
|
if (part)
|
||||||
partition_id = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();
|
partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
|
||||||
else
|
else
|
||||||
partition_id = data.getPartitionIDFromQuery(field);
|
partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
|
|
||||||
String source_dir = "detached/";
|
String source_dir = "detached/";
|
||||||
|
|
||||||
@ -532,12 +543,9 @@ void StorageMergeTree::attachPartition(const ASTPtr & query, const Field & field
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageMergeTree::freezePartition(const Field & partition, const String & with_name, const Settings & settings)
|
void StorageMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
|
||||||
{
|
{
|
||||||
/// The prefix can be arbitrary. Not necessarily a month - you can specify only a year.
|
data.freezePartition(partition, with_name, context);
|
||||||
data.freezePartition(partition.getType() == Field::Types::UInt64
|
|
||||||
? toString(partition.get<UInt64>())
|
|
||||||
: partition.safeGet<String>(), with_name);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -61,15 +61,12 @@ public:
|
|||||||
|
|
||||||
/** Perform the next step in combining the parts.
|
/** Perform the next step in combining the parts.
|
||||||
*/
|
*/
|
||||||
bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings) override
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||||
{
|
|
||||||
return merge(settings.min_bytes_to_use_direct_io, true, partition_id, final, deduplicate);
|
|
||||||
}
|
|
||||||
|
|
||||||
void dropPartition(const ASTPtr & query, const Field & partition, bool detach, const Settings & settings) override;
|
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) override;
|
||||||
void clearColumnInPartition(const ASTPtr & query, const Field & partition, const Field & column_name, const Settings & settings) override;
|
void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) override;
|
||||||
void attachPartition(const ASTPtr & query, const Field & partition, bool part, const Settings & settings) override;
|
void attachPartition(const ASTPtr & partition, bool part, const Context & context) override;
|
||||||
void freezePartition(const Field & partition, const String & with_name, const Settings & settings) override;
|
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context) override;
|
||||||
|
|
||||||
void drop() override;
|
void drop() override;
|
||||||
|
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTOptimizeQuery.h>
|
#include <Parsers/ASTOptimizeQuery.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
#include <IO/ReadBufferFromString.h>
|
#include <IO/ReadBufferFromString.h>
|
||||||
@ -2348,13 +2349,13 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & query, con
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings)
|
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
{
|
{
|
||||||
assertNotReadonly();
|
assertNotReadonly();
|
||||||
|
|
||||||
if (!is_leader_node)
|
if (!is_leader_node)
|
||||||
{
|
{
|
||||||
sendRequestToLeaderReplica(query, settings);
|
sendRequestToLeaderReplica(query, context.getSettingsRef());
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2375,13 +2376,14 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const String & p
|
|||||||
MergeTreeDataMerger::FuturePart future_merged_part;
|
MergeTreeDataMerger::FuturePart future_merged_part;
|
||||||
bool selected = false;
|
bool selected = false;
|
||||||
|
|
||||||
if (partition_id.empty())
|
if (!partition)
|
||||||
{
|
{
|
||||||
selected = merger.selectPartsToMerge(
|
selected = merger.selectPartsToMerge(
|
||||||
future_merged_part, false, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge);
|
future_merged_part, false, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
selected = merger.selectAllPartsToMergeWithinPartition(future_merged_part, disk_space, can_merge, partition_id, final);
|
selected = merger.selectAllPartsToMergeWithinPartition(future_merged_part, disk_space, can_merge, partition_id, final);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2619,13 +2621,13 @@ String StorageReplicatedMergeTree::getFakePartNameCoveringAllPartsInPartition(co
|
|||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::clearColumnInPartition(
|
void StorageReplicatedMergeTree::clearColumnInPartition(
|
||||||
const ASTPtr & query, const Field & partition, const Field & column_name, const Settings & settings)
|
const ASTPtr & partition, const Field & column_name, const Context & context)
|
||||||
{
|
{
|
||||||
assertNotReadonly();
|
assertNotReadonly();
|
||||||
|
|
||||||
/// We don't block merges, so anyone can manage this task (not only leader)
|
/// We don't block merges, so anyone can manage this task (not only leader)
|
||||||
|
|
||||||
String partition_id = data.getPartitionIDFromQuery(partition);
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
|
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
|
||||||
|
|
||||||
if (fake_part_name.empty())
|
if (fake_part_name.empty())
|
||||||
@ -2646,26 +2648,26 @@ void StorageReplicatedMergeTree::clearColumnInPartition(
|
|||||||
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||||
|
|
||||||
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
||||||
if (settings.replication_alter_partitions_sync != 0)
|
if (context.getSettingsRef().replication_alter_partitions_sync != 0)
|
||||||
{
|
{
|
||||||
if (settings.replication_alter_partitions_sync == 1)
|
if (context.getSettingsRef().replication_alter_partitions_sync == 1)
|
||||||
waitForReplicaToProcessLogEntry(replica_name, entry);
|
waitForReplicaToProcessLogEntry(replica_name, entry);
|
||||||
else
|
else
|
||||||
waitForAllReplicasToProcessLogEntry(entry);
|
waitForAllReplicasToProcessLogEntry(entry);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const Field & partition, bool detach, const Settings & settings)
|
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
|
||||||
{
|
{
|
||||||
assertNotReadonly();
|
assertNotReadonly();
|
||||||
|
|
||||||
if (!is_leader_node)
|
if (!is_leader_node)
|
||||||
{
|
{
|
||||||
sendRequestToLeaderReplica(query, settings);
|
sendRequestToLeaderReplica(query, context.getSettingsRef());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
String partition_id = data.getPartitionIDFromQuery(partition);
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
|
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
|
||||||
|
|
||||||
if (fake_part_name.empty())
|
if (fake_part_name.empty())
|
||||||
@ -2696,9 +2698,9 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const Field
|
|||||||
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
||||||
|
|
||||||
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
/// If necessary, wait until the operation is performed on itself or on all replicas.
|
||||||
if (settings.replication_alter_partitions_sync != 0)
|
if (context.getSettingsRef().replication_alter_partitions_sync != 0)
|
||||||
{
|
{
|
||||||
if (settings.replication_alter_partitions_sync == 1)
|
if (context.getSettingsRef().replication_alter_partitions_sync == 1)
|
||||||
waitForReplicaToProcessLogEntry(replica_name, entry);
|
waitForReplicaToProcessLogEntry(replica_name, entry);
|
||||||
else
|
else
|
||||||
waitForAllReplicasToProcessLogEntry(entry);
|
waitForAllReplicasToProcessLogEntry(entry);
|
||||||
@ -2706,16 +2708,16 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const Field
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & query, const Field & field, bool attach_part, const Settings & settings)
|
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
|
||||||
{
|
{
|
||||||
assertNotReadonly();
|
assertNotReadonly();
|
||||||
|
|
||||||
String partition_id;
|
String partition_id;
|
||||||
|
|
||||||
if (attach_part)
|
if (attach_part)
|
||||||
partition_id = field.safeGet<String>();
|
partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
|
||||||
else
|
else
|
||||||
partition_id = data.getPartitionIDFromQuery(field);
|
partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
|
|
||||||
String source_dir = "detached/";
|
String source_dir = "detached/";
|
||||||
|
|
||||||
@ -3219,9 +3221,9 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
|
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context)
|
||||||
{
|
{
|
||||||
String partition_id = data.getPartitionIDFromQuery(partition);
|
String partition_id = data.getPartitionIDFromQuery(partition, context);
|
||||||
|
|
||||||
String from = from_;
|
String from = from_;
|
||||||
if (from.back() == '/')
|
if (from.back() == '/')
|
||||||
@ -3373,20 +3375,15 @@ void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const S
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::freezePartition(const Field & partition, const String & with_name, const Settings & settings)
|
void StorageReplicatedMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
|
||||||
{
|
{
|
||||||
/// The prefix can be arbitrary. Not necessarily a month - you can specify only a year.
|
data.freezePartition(partition, with_name, context);
|
||||||
String prefix = partition.getType() == Field::Types::UInt64
|
|
||||||
? toString(partition.get<UInt64>())
|
|
||||||
: partition.safeGet<String>();
|
|
||||||
|
|
||||||
data.freezePartition(prefix, with_name);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::reshardPartitions(
|
void StorageReplicatedMergeTree::reshardPartitions(
|
||||||
const ASTPtr & query, const String & database_name,
|
const ASTPtr & query, const String & database_name,
|
||||||
const Field & partition,
|
const ASTPtr & partition,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
||||||
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
||||||
const Context & context)
|
const Context & context)
|
||||||
@ -3466,8 +3463,8 @@ void StorageReplicatedMergeTree::reshardPartitions(
|
|||||||
throw Exception{"Shard paths must be distinct", ErrorCodes::DUPLICATE_SHARD_PATHS};
|
throw Exception{"Shard paths must be distinct", ErrorCodes::DUPLICATE_SHARD_PATHS};
|
||||||
}
|
}
|
||||||
|
|
||||||
bool include_all = partition.isNull();
|
bool include_all = !partition;
|
||||||
String partition_id = !partition.isNull() ? data.getPartitionIDFromQuery(partition) : String();
|
String partition_id = partition ? data.getPartitionIDFromQuery(partition, context) : String();
|
||||||
|
|
||||||
/// Make a list of local partitions that need to be resharded.
|
/// Make a list of local partitions that need to be resharded.
|
||||||
std::set<std::string> unique_partition_list;
|
std::set<std::string> unique_partition_list;
|
||||||
|
@ -133,19 +133,19 @@ public:
|
|||||||
|
|
||||||
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
BlockOutputStreamPtr write(const ASTPtr & query, const Settings & settings) override;
|
||||||
|
|
||||||
bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings) override;
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||||
|
|
||||||
void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override;
|
void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override;
|
||||||
|
|
||||||
void clearColumnInPartition(const ASTPtr & query, const Field & partition, const Field & column_name, const Settings & settings) override;
|
void clearColumnInPartition(const ASTPtr & partition, const Field & column_name, const Context & context) override;
|
||||||
void dropPartition(const ASTPtr & query, const Field & partition, bool detach, const Settings & settings) override;
|
void dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context) override;
|
||||||
void attachPartition(const ASTPtr & query, const Field & partition, bool part, const Settings & settings) override;
|
void attachPartition(const ASTPtr & partition, bool part, const Context & context) override;
|
||||||
void fetchPartition(const Field & partition, const String & from, const Settings & settings) override;
|
void fetchPartition(const ASTPtr & partition, const String & from, const Context & context) override;
|
||||||
void freezePartition(const Field & partition, const String & with_name, const Settings & settings) override;
|
void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context) override;
|
||||||
|
|
||||||
void reshardPartitions(
|
void reshardPartitions(
|
||||||
const ASTPtr & query, const String & database_name,
|
const ASTPtr & query, const String & database_name,
|
||||||
const Field & partition,
|
const ASTPtr & partition,
|
||||||
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
||||||
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
||||||
const Context & context) override;
|
const Context & context) override;
|
||||||
|
@ -380,9 +380,9 @@ void StorageTrivialBuffer::shutdown()
|
|||||||
*
|
*
|
||||||
* This kind of race condition make very hard to implement proper tests.
|
* This kind of race condition make very hard to implement proper tests.
|
||||||
*/
|
*/
|
||||||
bool StorageTrivialBuffer::optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings)
|
bool StorageTrivialBuffer::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
|
||||||
{
|
{
|
||||||
if (!partition_id.empty())
|
if (partition)
|
||||||
throw Exception("Partition cannot be specified when optimizing table of type TrivialBuffer",
|
throw Exception("Partition cannot be specified when optimizing table of type TrivialBuffer",
|
||||||
ErrorCodes::NOT_IMPLEMENTED);
|
ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
|
@ -69,7 +69,7 @@ public:
|
|||||||
void startup() override;
|
void startup() override;
|
||||||
/// Writes all the blocks in buffer into the destination table. Stop flushing thread.
|
/// Writes all the blocks in buffer into the destination table. Stop flushing thread.
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
bool optimize(const ASTPtr & query, const String & partition_id, bool final, bool deduplicate, const Settings & settings) override;
|
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||||
|
|
||||||
void rename(const String & new_path_to_db, const String & new_database_name,
|
void rename(const String & new_path_to_db, const String & new_database_name,
|
||||||
const String & new_table_name) override { name = new_table_name; }
|
const String & new_table_name) override { name = new_table_name; }
|
||||||
|
Loading…
Reference in New Issue
Block a user