Merge pull request #8701 from ClickHouse/alter_on_top_of_mutations

Alter on top of mutations for ReplicatedMergeTree
This commit is contained in:
alesapin 2020-03-03 20:15:08 +03:00 committed by GitHub
commit 2b390a9b73
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
59 changed files with 2025 additions and 878 deletions

View File

@ -488,6 +488,7 @@ namespace ErrorCodes
extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND = 514;
extern const int INCORRECT_ACCESS_ENTITY_DEFINITION = 515;
extern const int AUTHENTICATION_FAILED = 516;
extern const int CANNOT_ASSIGN_ALTER = 517;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -165,4 +165,13 @@ bool NamesAndTypesList::contains(const String & name) const
return false;
}
std::optional<NameAndTypePair> NamesAndTypesList::tryGetByName(const std::string & name) const
{
for (const NameAndTypePair & column : *this)
{
if (column.name == name)
return column;
}
return {};
}
}

View File

@ -73,7 +73,11 @@ public:
/// Unlike `filter`, returns columns in the order in which they go in `names`.
NamesAndTypesList addTypes(const Names & names) const;
/// Check that column contains in list
bool contains(const String & name) const;
/// Try to get column by name, return empty optional if column not found
std::optional<NameAndTypePair> tryGetByName(const std::string & name) const;
};
}

View File

@ -1,7 +1,7 @@
#include <Common/typeid_cast.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <Columns/ColumnsNumber.h>

View File

@ -1,6 +1,6 @@
#include <DataStreams/TTLBlockInputStream.h>
#include <DataTypes/DataTypeDate.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>

View File

@ -284,6 +284,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
if (commands.empty())
throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR);
const ColumnsDescription & columns_desc = storage->getColumns();
const IndicesDescription & indices_desc = storage->getIndices();
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
@ -292,7 +293,9 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
for (const MutationCommand & command : commands)
{
for (const auto & kv : command.column_to_update_expression)
{
updated_columns.insert(kv.first);
}
}
/// We need to know which columns affect which MATERIALIZED columns and data skipping indices
@ -436,6 +439,15 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run)
}
}
}
else if (command.type == MutationCommand::READ_COLUMN)
{
if (stages.empty() || !stages.back().column_to_updated.empty())
stages.emplace_back(context);
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
stages.emplace_back(context);
stages.back().column_to_updated.emplace(command.column_name, std::make_shared<ASTIdentifier>(command.column_name));
}
else
throw Exception("Unknown mutation command type: " + DB::toString<int>(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND);
}
@ -506,6 +518,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &
{
NamesAndTypesList all_columns = storage->getColumns().getAllPhysical();
/// Next, for each stage calculate columns changed by this and previous stages.
for (size_t i = 0; i < prepared_stages.size(); ++i)
{
@ -518,8 +531,6 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &
if (i > 0)
prepared_stages[i].output_columns = prepared_stages[i - 1].output_columns;
else if (!commands.additional_columns.empty())
prepared_stages[i].output_columns.insert(commands.additional_columns.begin(), commands.additional_columns.end());
if (prepared_stages[i].output_columns.size() < all_columns.size())
{

View File

@ -4,7 +4,7 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnArray.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Core/Block.h>
#include <Storages/ColumnDefault.h>

View File

@ -1,12 +1,14 @@
#include "evaluateMissingDefaults.h"
#include "inplaceBlockConversions.h"
#include <Core/Block.h>
#include <Parsers/queryToString.h>
#include <Storages/ColumnDefault.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTWithAlias.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <utility>
@ -16,7 +18,10 @@
namespace DB
{
static ASTPtr requiredExpressions(Block & block, const NamesAndTypesList & required_columns, const ColumnDefaults & column_defaults)
namespace
{
ASTPtr defaultRequiredExpressions(Block & block, const NamesAndTypesList & required_columns, const ColumnDefaults & column_defaults)
{
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
@ -40,22 +45,41 @@ static ASTPtr requiredExpressions(Block & block, const NamesAndTypesList & requi
return default_expr_list;
}
void evaluateMissingDefaults(Block & block,
const NamesAndTypesList & required_columns,
const ColumnDefaults & column_defaults,
const Context & context, bool save_unneeded_columns)
ASTPtr convertRequiredExpressions(Block & block, const NamesAndTypesList & required_columns)
{
if (column_defaults.empty())
return;
ASTPtr conversion_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & required_column : required_columns)
{
if (!block.has(required_column.name))
continue;
ASTPtr default_expr_list = requiredExpressions(block, required_columns, column_defaults);
if (!default_expr_list)
auto column_in_block = block.getByName(required_column.name);
if (column_in_block.type->equals(*required_column.type))
continue;
auto cast_func = makeASTFunction(
"CAST", std::make_shared<ASTIdentifier>(required_column.name), std::make_shared<ASTLiteral>(required_column.type->getName()));
conversion_expr_list->children.emplace_back(setAlias(cast_func, required_column.name));
}
return conversion_expr_list;
}
void executeExpressionsOnBlock(
Block & block,
ASTPtr expr_list,
bool save_unneeded_columns,
const NamesAndTypesList & required_columns,
const Context & context)
{
if (!expr_list)
return;
if (!save_unneeded_columns)
{
auto syntax_result = SyntaxAnalyzer(context).analyze(default_expr_list, block.getNamesAndTypesList());
ExpressionAnalyzer{default_expr_list, syntax_result, context}.getActions(true)->execute(block);
auto syntax_result = SyntaxAnalyzer(context).analyze(expr_list, block.getNamesAndTypesList());
ExpressionAnalyzer{expr_list, syntax_result, context}.getActions(true)->execute(block);
return;
}
@ -63,8 +87,8 @@ void evaluateMissingDefaults(Block & block,
* we are going to operate on a copy instead of the original block */
Block copy_block{block};
auto syntax_result = SyntaxAnalyzer(context).analyze(default_expr_list, block.getNamesAndTypesList());
auto expression_analyzer = ExpressionAnalyzer{default_expr_list, syntax_result, context};
auto syntax_result = SyntaxAnalyzer(context).analyze(expr_list, block.getNamesAndTypesList());
auto expression_analyzer = ExpressionAnalyzer{expr_list, syntax_result, context};
auto required_source_columns = syntax_result->requiredSourceColumns();
auto rows_was = copy_block.rows();
@ -97,9 +121,34 @@ void evaluateMissingDefaults(Block & block,
auto evaluated_col = copy_block.getByName(col->name);
evaluated_col.column = evaluated_col.column->convertToFullColumnIfConst();
block.insert(pos, std::move(evaluated_col));
if (block.has(col->name))
block.getByName(col->name) = std::move(evaluated_col);
else
block.insert(pos, std::move(evaluated_col));
}
}
}
}
void performRequiredConversions(Block & block, const NamesAndTypesList & required_columns, const Context & context)
{
ASTPtr conversion_expr_list = convertRequiredExpressions(block, required_columns);
if (conversion_expr_list->children.empty())
return;
executeExpressionsOnBlock(block, conversion_expr_list, true, required_columns, context);
}
void evaluateMissingDefaults(Block & block,
const NamesAndTypesList & required_columns,
const ColumnDefaults & column_defaults,
const Context & context, bool save_unneeded_columns)
{
if (column_defaults.empty())
return;
ASTPtr default_expr_list = defaultRequiredExpressions(block, required_columns, column_defaults);
executeExpressionsOnBlock(block, default_expr_list, save_unneeded_columns, required_columns, context);
}
}

View File

@ -12,9 +12,15 @@ class Context;
class NamesAndTypesList;
struct ColumnDefault;
/// Adds missing defaults to block according to required_columns
/// using column_defaults map
void evaluateMissingDefaults(Block & block,
const NamesAndTypesList & required_columns,
const std::unordered_map<std::string, ColumnDefault> & column_defaults,
const Context & context, bool save_unneeded_columns = true);
/// Tries to convert columns in block to required_columns
void performRequiredConversions(Block & block,
const NamesAndTypesList & required_columns,
const Context & context);
}

View File

@ -1,12 +1,16 @@
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Compression/CompressionFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
@ -16,9 +20,10 @@
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include <Compression/CompressionFactory.h>
#include <Parsers/queryToString.h>
@ -45,6 +50,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
if (command_ast->type == ASTAlterCommand::ADD_COLUMN)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::ADD_COLUMN;
const auto & ast_col_decl = command_ast->col_decl->as<ASTColumnDeclaration &>();
@ -85,6 +91,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::DROP_COLUMN;
command.column_name = getIdentifierName(command_ast->column);
command.if_exists = command_ast->if_exists;
@ -93,6 +100,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::MODIFY_COLUMN;
const auto & ast_col_decl = command_ast->col_decl->as<ASTColumnDeclaration &>();
@ -128,6 +136,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::COMMENT_COLUMN)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = COMMENT_COLUMN;
command.column_name = getIdentifierName(command_ast->column);
const auto & ast_comment = command_ast->comment->as<ASTLiteral &>();
@ -138,6 +147,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::MODIFY_ORDER_BY;
command.order_by = command_ast->order_by;
return command;
@ -145,6 +155,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::ADD_INDEX)
{
AlterCommand command;
command.ast = command_ast->clone();
command.index_decl = command_ast->index_decl;
command.type = AlterCommand::ADD_INDEX;
@ -162,6 +173,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT)
{
AlterCommand command;
command.ast = command_ast->clone();
command.constraint_decl = command_ast->constraint_decl;
command.type = AlterCommand::ADD_CONSTRAINT;
@ -179,6 +191,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
AlterCommand command;
command.ast = command_ast->clone();
command.if_exists = command_ast->if_exists;
command.type = AlterCommand::DROP_CONSTRAINT;
command.constraint_name = command_ast->constraint->as<ASTIdentifier &>().name;
@ -191,6 +204,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
throw Exception("\"ALTER TABLE table CLEAR INDEX index\" queries are not supported yet. Use \"CLEAR INDEX index IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::DROP_INDEX;
command.index_name = command_ast->index->as<ASTIdentifier &>().name;
command.if_exists = command_ast->if_exists;
@ -200,6 +214,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::MODIFY_TTL;
command.ttl = command_ast->ttl;
return command;
@ -207,6 +222,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
else if (command_ast->type == ASTAlterCommand::MODIFY_SETTING)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::MODIFY_SETTING;
command.settings_changes = command_ast->settings_changes->as<ASTSetQuery &>().changes;
return command;
@ -436,6 +452,82 @@ bool AlterCommand::isSettingsAlter() const
return type == MODIFY_SETTING;
}
namespace
{
/// If true, then in order to ALTER the type of the column from the type from to the type to
/// we don't need to rewrite the data, we only need to update metadata and columns.txt in part directories.
/// The function works for Arrays and Nullables of the same structure.
bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
{
if (from->getName() == to->getName())
return true;
static const std::unordered_multimap<std::type_index, const std::type_info &> ALLOWED_CONVERSIONS =
{
{ typeid(DataTypeEnum8), typeid(DataTypeEnum8) },
{ typeid(DataTypeEnum8), typeid(DataTypeInt8) },
{ typeid(DataTypeEnum16), typeid(DataTypeEnum16) },
{ typeid(DataTypeEnum16), typeid(DataTypeInt16) },
{ typeid(DataTypeDateTime), typeid(DataTypeUInt32) },
{ typeid(DataTypeUInt32), typeid(DataTypeDateTime) },
{ typeid(DataTypeDate), typeid(DataTypeUInt16) },
{ typeid(DataTypeUInt16), typeid(DataTypeDate) },
};
while (true)
{
auto it_range = ALLOWED_CONVERSIONS.equal_range(typeid(*from));
for (auto it = it_range.first; it != it_range.second; ++it)
{
if (it->second == typeid(*to))
return true;
}
const auto * arr_from = typeid_cast<const DataTypeArray *>(from);
const auto * arr_to = typeid_cast<const DataTypeArray *>(to);
if (arr_from && arr_to)
{
from = arr_from->getNestedType().get();
to = arr_to->getNestedType().get();
continue;
}
const auto * nullable_from = typeid_cast<const DataTypeNullable *>(from);
const auto * nullable_to = typeid_cast<const DataTypeNullable *>(to);
if (nullable_from && nullable_to)
{
from = nullable_from->getNestedType().get();
to = nullable_to->getNestedType().get();
continue;
}
return false;
}
}
}
bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metadata) const
{
if (ignore)
return false;
if (type == DROP_COLUMN)
return true;
if (type != MODIFY_COLUMN || data_type == nullptr)
return false;
for (const auto & column : metadata.columns.getAllPhysical())
{
if (column.name == column_name && !isMetadataOnlyConversion(column.type.get(), data_type.get()))
return true;
}
return false;
}
bool AlterCommand::isCommentAlter() const
{
if (type == COMMENT_COLUMN)
@ -453,6 +545,37 @@ bool AlterCommand::isCommentAlter() const
return false;
}
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(const StorageInMemoryMetadata & metadata) const
{
if (!isRequireMutationStage(metadata))
return {};
MutationCommand result;
if (type == MODIFY_COLUMN)
{
result.type = MutationCommand::Type::READ_COLUMN;
result.column_name = column_name;
result.data_type = data_type;
result.predicate = nullptr;
}
else if (type == DROP_COLUMN)
{
result.type = MutationCommand::Type::DROP_COLUMN;
result.column_name = column_name;
result.predicate = nullptr;
}
else if (type == DROP_INDEX)
{
result.type = MutationCommand::Type::DROP_INDEX;
result.column_name = column_name;
result.predicate = nullptr;
}
result.ast = ast->clone();
return result;
}
String alterTypeToString(const AlterCommand::Type type)
{
@ -680,4 +803,15 @@ bool AlterCommands::isCommentAlter() const
{
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isCommentAlter(); });
}
MutationCommands AlterCommands::getMutationCommands(const StorageInMemoryMetadata & metadata) const
{
MutationCommands result;
for (const auto & alter_cmd : *this)
if (auto mutation_cmd = alter_cmd.tryConvertToMutationCommand(metadata); mutation_cmd)
result.push_back(*mutation_cmd);
return result;
}
}

View File

@ -4,8 +4,8 @@
#include <Core/NamesAndTypes.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/StorageInMemoryMetadata.h>
#include <Storages/MutationCommands.h>
#include <Storages/ColumnsDescription.h>
#include <Common/SettingsChanges.h>
@ -18,6 +18,9 @@ class ASTAlterCommand;
/// Adding Nested columns is not expanded to add individual columns.
struct AlterCommand
{
/// The AST of the whole command
ASTPtr ast;
enum Type
{
ADD_COLUMN,
@ -100,11 +103,18 @@ struct AlterCommand
/// in each part on disk (it's not lightweight alter).
bool isModifyingData() const;
bool isRequireMutationStage(const StorageInMemoryMetadata & metadata) const;
/// Checks that only settings changed by alter
bool isSettingsAlter() const;
/// Checks that only comment changed by alter
bool isCommentAlter() const;
/// If possible, convert alter command to mutation command. In other case
/// return empty optional. Some storages may execute mutations after
/// metadata changes.
std::optional<MutationCommand> tryConvertToMutationCommand(const StorageInMemoryMetadata & metadata) const;
};
/// Return string representation of AlterCommand::Type
@ -151,6 +161,11 @@ public:
/// At least one command modify comments.
bool isCommentAlter() const;
/// Return mutation commands which some storages may execute as part of
/// alter. If alter can be performed is pure metadata update, than result is
/// empty.
MutationCommands getMutationCommands(const StorageInMemoryMetadata & metadata) const;
};
}

View File

@ -370,12 +370,7 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
{
return
{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
};
return StorageInMemoryMetadata(getColumns(), getIndices(), getConstraints());
}
void IStorage::alter(

View File

@ -87,7 +87,7 @@ private:
/// Acquires block number locks in all partitions.
class EphemeralLocksInAllPartitions
class EphemeralLocksInAllPartitions : private boost::noncopyable
{
public:
EphemeralLocksInAllPartitions(

View File

@ -3,7 +3,7 @@
#include <Common/escapeForFileName.h>
#include <Compression/CachedCompressedReadBuffer.h>
#include <Columns/ColumnArray.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Storages/MergeTree/IMergeTreeReader.h>
#include <Common/typeid_cast.h>
#include <Poco/File.h>
@ -182,4 +182,53 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns
}
}
void IMergeTreeReader::performRequiredConversions(Columns & res_columns)
{
try
{
size_t num_columns = columns.size();
if (res_columns.size() != num_columns)
{
throw Exception(
"Invalid number of columns passed to MergeTreeReader::performRequiredConversions. "
"Expected "
+ toString(num_columns)
+ ", "
"got "
+ toString(res_columns.size()),
ErrorCodes::LOGICAL_ERROR);
}
Block copy_block;
auto name_and_type = columns.begin();
for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type)
{
if (res_columns[pos] == nullptr)
continue;
if (columns_from_part.count(name_and_type->name))
copy_block.insert({res_columns[pos], columns_from_part[name_and_type->name], name_and_type->name});
else
copy_block.insert({res_columns[pos], name_and_type->type, name_and_type->name});
}
DB::performRequiredConversions(copy_block, columns, storage.global_context);
/// Move columns from block.
name_and_type = columns.begin();
for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type)
{
res_columns[pos] = std::move(copy_block.getByName(name_and_type->name).column);
}
}
catch (Exception & e)
{
/// Better diagnostics.
e.addMessage("(while reading from part " + path + ")");
throw;
}
}
}

View File

@ -44,6 +44,10 @@ public:
/// Evaluate defaulted columns if necessary.
void evaluateMissingDefaults(Block additional_columns, Columns & res_columns);
/// If part metadata is not equal to storage metadata, than
/// try to perform conversions of columns.
void performRequiredConversions(Columns & res_columns);
const NamesAndTypesList & getColumns() const { return columns; }
size_t numColumnsInResult() const { return columns.size(); }
@ -65,6 +69,8 @@ protected:
/// Columns that are read.
NamesAndTypesList columns;
std::unordered_map<String, DataTypePtr> columns_from_part;
UncompressedCache * uncompressed_cache;
MarkCache * mark_cache;

View File

@ -245,13 +245,6 @@ MergeTreeReadTaskColumns getReadTaskColumns(const MergeTreeData & storage, const
if (check_columns)
{
/// Under owned_data_part->columns_lock we check that all requested columns are of the same type as in the table.
/// This may be not true in case of ALTER MODIFY.
if (!pre_column_names.empty())
storage.check(data_part->getColumns(), pre_column_names);
if (!column_names.empty())
storage.check(data_part->getColumns(), column_names);
const NamesAndTypesList & physical_columns = storage.getColumns().getAllPhysical();
result.pre_columns = physical_columns.addTypes(pre_column_names);
result.columns = physical_columns.addTypes(column_names);

View File

@ -260,11 +260,7 @@ MergeTreeData::MergeTreeData(
StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const
{
StorageInMemoryMetadata metadata{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
};
StorageInMemoryMetadata metadata(getColumns(), getIndices(), getConstraints());
if (partition_by_ast)
metadata.partition_by_ast = partition_by_ast->clone();
@ -1727,6 +1723,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(
return createPart(name, type, part_info, disk, relative_path);
}
/// This code is not used anymore in StorageReplicatedMergeTree
/// soon it will be removed from StorageMergeTree as well
/// TODO(alesap)
void MergeTreeData::alterDataPart(
const NamesAndTypesList & new_columns,
const IndicesASTs & new_indices,

View File

@ -969,18 +969,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
context_for_reading.getSettingsRef().max_threads = 1;
MutationCommands commands_for_part;
std::copy_if(
std::cbegin(commands), std::cend(commands),
std::back_inserter(commands_for_part),
[&] (const MutationCommand & command)
{
return command.partition == nullptr ||
future_part.parts[0]->info.partition_id == data.getPartitionIDFromQuery(
command.partition, context_for_reading);
});
if (isCompactPart(source_part))
commands_for_part.additional_columns = source_part->getColumns().getNames();
for (const auto & command : commands)
{
if (command.partition == nullptr || future_part.parts[0]->info.partition_id == data.getPartitionIDFromQuery(
command.partition, context_for_reading))
commands_for_part.emplace_back(command);
}
if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading))
{
@ -988,34 +982,42 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info);
}
else
{
LOG_TRACE(log, "Mutating part " << source_part->name << " to mutation version " << future_part.part_info.mutation);
}
MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading, true);
auto in = mutations_interpreter.execute(table_lock_holder);
const auto & updated_header = mutations_interpreter.getUpdatedHeader();
BlockInputStreamPtr in = nullptr;
Block updated_header;
std::optional<MutationsInterpreter> interpreter;
const auto data_settings = data.getSettings();
MutationCommands for_interpreter, for_file_renames;
splitMutationCommands(source_part, commands_for_part, for_interpreter, for_file_renames);
UInt64 watch_prev_elapsed = 0;
MergeStageProgress stage_progress(1.0);
NamesAndTypesList all_columns = data.getColumns().getAllPhysical();
const auto & source_column_names = source_part->getColumns().getNames();
const auto & updated_column_names = updated_header.getNames();
if (!for_interpreter.empty())
{
interpreter.emplace(storage_from_source_part, for_interpreter, context_for_reading, true);
in = interpreter->execute(table_lock_holder);
updated_header = interpreter->getUpdatedHeader();
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
}
NameSet new_columns_set(source_column_names.begin(), source_column_names.end());
new_columns_set.insert(updated_column_names.begin(), updated_column_names.end());
auto new_columns = all_columns.filter(new_columns_set);
auto new_data_part = data.createPart(
future_part.name,
future_part.type,
future_part.part_info,
space_reservation->getDisk(),
"tmp_mut_" + future_part.name);
future_part.name, future_part.type, future_part.part_info, space_reservation->getDisk(), "tmp_mut_" + future_part.name);
new_data_part->is_temp = true;
new_data_part->ttl_infos = source_part->ttl_infos;
/// It shouldn't be changed by mutation.
new_data_part->index_granularity_info = source_part->index_granularity_info;
new_data_part->setColumns(new_columns);
new_data_part->setColumns(getColumnsForNewDataPart(source_part, updated_header, all_columns));
String new_part_tmp_path = new_data_part->getFullPath();
@ -1025,37 +1027,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
/// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus
/// deadlock is impossible.
auto compression_codec = context.chooseCompressionCodec(
source_part->bytes_on_disk,
static_cast<double>(source_part->bytes_on_disk) / data.getTotalActiveSizeInBytes());
source_part->bytes_on_disk, static_cast<double>(source_part->bytes_on_disk) / data.getTotalActiveSizeInBytes());
Poco::File(new_part_tmp_path).createDirectories();
const auto data_settings = data.getSettings();
/// Don't change granularity type while mutating subset of columns
auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType())
: getNonAdaptiveMrkExtension();
Block in_header = in->getHeader();
bool need_remove_expired_values = false;
UInt64 watch_prev_elapsed = 0;
MergeStageProgress stage_progress(1.0);
in->setProgressCallback(MergeProgressCallback(merge_entry, watch_prev_elapsed, stage_progress));
auto should_execute_ttl = [&](const auto & in_columns)
{
if (!data.hasAnyTTL())
return false;
for (const auto & command : commands_for_part)
if (command.type == MutationCommand::MATERIALIZE_TTL)
return true;
auto dependencies = data.getColumnDependencies(NameSet(in_columns.begin(), in_columns.end()));
for (const auto & dependency : dependencies)
if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET)
return true;
return false;
};
bool need_remove_expired_values = should_execute_ttl(in_header.getNamesAndTypesList().getNames());
if (in && shouldExecuteTTL(in->getHeader().getNamesAndTypesList().getNames(), commands_for_part))
need_remove_expired_values = true;
/// All columns from part are changed and may be some more that were missing before in part
if (source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList()))
@ -1102,7 +1086,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
/// Checks if columns used in skipping indexes modified.
std::set<MergeTreeIndexPtr> indices_to_recalc;
ASTPtr indices_recalc_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & col : in_header.getNames())
for (const auto & col : updated_header.getNames())
{
for (size_t i = 0; i < data.skip_indices.size(); ++i)
{
@ -1121,14 +1105,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
if (!indices_to_recalc.empty())
{
auto indices_recalc_syntax = SyntaxAnalyzer(context).analyze(
indices_recalc_expr_list, in_header.getNamesAndTypesList());
auto indices_recalc_syntax =
SyntaxAnalyzer(context).analyze(indices_recalc_expr_list, in->getHeader().getNamesAndTypesList());
auto indices_recalc_expr = ExpressionAnalyzer(
indices_recalc_expr_list,
indices_recalc_syntax, context).getActions(false);
/// We can update only one column, but some skip idx expression may depend on several
/// columns (c1 + c2 * c3). It works because in stream was created with help of
/// columns (c1 + c2 * c3). It works because this stream was created with help of
/// MutationsInterpreter which knows about skip indices and stream 'in' already has
/// all required columns.
/// TODO move this logic to single place.
@ -1136,36 +1120,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
std::make_shared<ExpressionBlockInputStream>(in, indices_recalc_expr));
}
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
/// Don't change granularity type while mutating subset of columns
auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension(new_data_part->getType()) : getNonAdaptiveMrkExtension();
for (const auto & entry : updated_header)
{
IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path)
{
String stream_name = IDataType::getFileNameForStream(entry.name, substream_path);
files_to_skip.insert(stream_name + ".bin");
files_to_skip.insert(stream_name + mrk_extension);
};
IDataType::SubstreamPath stream_path;
entry.type->enumerateStreams(callback, stream_path);
}
for (const auto & index : indices_to_recalc)
{
files_to_skip.insert(index->getFileName() + ".idx");
files_to_skip.insert(index->getFileName() + mrk_extension);
}
NameSet files_to_skip = collectFilesToSkip(updated_header, indices_to_recalc, mrk_extension);
NameSet files_to_remove = collectFilesToRemove(source_part, for_file_renames, mrk_extension);
if (need_remove_expired_values)
files_to_skip.insert("ttl.txt");
Poco::DirectoryIterator dir_end;
/// Create hardlinks for unchanged files
for (Poco::DirectoryIterator dir_it(source_part->getFullPath()); dir_it != dir_end; ++dir_it)
{
if (files_to_skip.count(dir_it.name()))
if (files_to_skip.count(dir_it.name()) || files_to_remove.count(dir_it.name()))
continue;
Poco::Path destination(new_part_tmp_path);
@ -1176,38 +1141,43 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
merge_entry->columns_written = all_columns.size() - updated_header.columns();
if (need_remove_expired_values)
in = std::make_shared<TTLBlockInputStream>(in, data, new_data_part, time_of_mutation, true);
MergedColumnOnlyOutputStream out(
new_data_part,
updated_header,
/* sync = */ false,
compression_codec,
/* skip_offsets = */ false,
std::vector<MergeTreeIndexPtr>(indices_to_recalc.begin(), indices_to_recalc.end()),
nullptr,
source_part->index_granularity,
&source_part->index_granularity_info
);
in->readPrefix();
out.writePrefix();
Block block;
while (check_not_cancelled() && (block = in->read()))
{
out.write(block);
merge_entry->rows_written += block.rows();
merge_entry->bytes_written_uncompressed += block.bytes();
}
in->readSuffix();
auto changed_checksums = out.writeSuffixAndGetChecksums();
new_data_part->checksums = source_part->checksums;
new_data_part->checksums.add(std::move(changed_checksums));
if (in)
{
if (need_remove_expired_values)
in = std::make_shared<TTLBlockInputStream>(in, data, new_data_part, time_of_mutation, true);
IMergedBlockOutputStream::WrittenOffsetColumns unused_written_offsets;
MergedColumnOnlyOutputStream out(
new_data_part,
updated_header,
/* sync = */ false,
compression_codec,
/* skip_offsets = */ false,
std::vector<MergeTreeIndexPtr>(indices_to_recalc.begin(), indices_to_recalc.end()),
nullptr,
source_part->index_granularity,
&source_part->index_granularity_info
);
in->readPrefix();
out.writePrefix();
Block block;
while (check_not_cancelled() && (block = in->read()))
{
out.write(block);
merge_entry->rows_written += block.rows();
merge_entry->bytes_written_uncompressed += block.bytes();
}
in->readSuffix();
auto changed_checksums = out.writeSuffixAndGetChecksums();
new_data_part->checksums.add(std::move(changed_checksums));
}
if (need_remove_expired_values)
{
@ -1219,16 +1189,21 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
new_data_part->checksums.files["ttl.txt"].file_hash = out_hashing.getHash();
}
for (const String & removed_file : files_to_remove)
if (new_data_part->checksums.files.count(removed_file))
new_data_part->checksums.files.erase(removed_file);
{
/// Write file with checksums.
WriteBufferFromFile out_checksums(new_part_tmp_path + "checksums.txt", 4096);
new_data_part->checksums.write(out_checksums);
}
} /// close fd
{
/// Write a file with a description of columns.
WriteBufferFromFile out_columns(new_part_tmp_path + "columns.txt", 4096);
new_data_part->getColumns().writeText(out_columns);
}
} /// close
new_data_part->rows_count = source_part->rows_count;
new_data_part->index_granularity = source_part->index_granularity;
@ -1340,4 +1315,184 @@ size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::
return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE);
}
void MergeTreeDataMergerMutator::splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames) const
{
NameSet removed_columns_from_compact_part;
NameSet already_changed_columns;
bool is_compact_part = isCompactPart(part);
for (const auto & command : commands)
{
if (command.type == MutationCommand::Type::DELETE
|| command.type == MutationCommand::Type::UPDATE
|| command.type == MutationCommand::Type::MATERIALIZE_INDEX
|| command.type == MutationCommand::Type::MATERIALIZE_TTL)
{
for_interpreter.push_back(command);
for (const auto & [column_name, expr] : command.column_to_update_expression)
already_changed_columns.emplace(column_name);
}
else if (command.type == MutationCommand::Type::READ_COLUMN)
{
/// If we don't have this column in source part, than we don't
/// need to materialize it
if (part->getColumns().contains(command.column_name))
{
for_interpreter.push_back(command);
if (!command.column_name.empty())
already_changed_columns.emplace(command.column_name);
}
else
for_file_renames.push_back(command);
}
else if (is_compact_part && command.type == MutationCommand::Type::DROP_COLUMN)
{
removed_columns_from_compact_part.emplace(command.column_name);
}
else
{
for_file_renames.push_back(command);
}
}
if (is_compact_part)
{
/// If it's compact part than we don't need to actually remove files from disk
/// we just don't read dropped columns
for (const auto & column : part->getColumns())
{
if (!removed_columns_from_compact_part.count(column.name) && !already_changed_columns.count(column.name))
{
for_interpreter.emplace_back(MutationCommand
{
.type = MutationCommand::Type::READ_COLUMN,
.column_name = column.name,
.data_type = column.type
});
}
}
}
}
NameSet MergeTreeDataMergerMutator::collectFilesToRemove(
MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension) const
{
/// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes.
std::map<String, size_t> stream_counts;
for (const NameAndTypePair & column : source_part->getColumns())
{
column.type->enumerateStreams(
[&](const IDataType::SubstreamPath & substream_path)
{
++stream_counts[IDataType::getFileNameForStream(column.name, substream_path)];
},
{});
}
NameSet remove_files;
/// Remove old indices
for (const auto & command : commands_for_removes)
{
if (command.type == MutationCommand::Type::DROP_INDEX)
{
remove_files.emplace("skp_idx_" + command.column_name + ".idx");
remove_files.emplace("skp_idx_" + command.column_name + mrk_extension);
}
else if (command.type == MutationCommand::Type::DROP_COLUMN)
{
IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path)
{
String stream_name = IDataType::getFileNameForStream(command.column_name, substream_path);
/// Delete files if they are no longer shared with another column.
if (--stream_counts[stream_name] == 0)
{
remove_files.emplace(stream_name + ".bin");
remove_files.emplace(stream_name + mrk_extension);
}
};
IDataType::SubstreamPath stream_path;
auto column = source_part->getColumns().tryGetByName(command.column_name);
if (column)
column->type->enumerateStreams(callback, stream_path);
}
}
return remove_files;
}
NameSet MergeTreeDataMergerMutator::collectFilesToSkip(
const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension) const
{
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
/// Skip updated files
for (const auto & entry : updated_header)
{
IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path)
{
String stream_name = IDataType::getFileNameForStream(entry.name, substream_path);
files_to_skip.insert(stream_name + ".bin");
files_to_skip.insert(stream_name + mrk_extension);
};
IDataType::SubstreamPath stream_path;
entry.type->enumerateStreams(callback, stream_path);
}
for (const auto & index : indices_to_recalc)
{
files_to_skip.insert(index->getFileName() + ".idx");
files_to_skip.insert(index->getFileName() + mrk_extension);
}
return files_to_skip;
}
NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart(
MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns) const
{
Names source_column_names = source_part->getColumns().getNames();
NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end());
for (auto it = all_columns.begin(); it != all_columns.end();)
{
if (updated_header.has(it->name))
{
auto updated_type = updated_header.getByName(it->name).type;
if (updated_type != it->type)
it->type = updated_type;
++it;
}
else if (source_columns_name_set.count(it->name))
{
++it;
}
else
it = all_columns.erase(it);
}
return all_columns;
}
bool MergeTreeDataMergerMutator::shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const
{
if (!data.hasAnyTTL())
return false;
for (const auto & command : commands)
if (command.type == MutationCommand::MATERIALIZE_TTL)
return true;
auto dependencies = data.getColumnDependencies(NameSet(columns.begin(), columns.end()));
for (const auto & dependency : dependencies)
if (dependency.kind == ColumnDependency::TTL_EXPRESSION || dependency.kind == ColumnDependency::TTL_TARGET)
return true;
return false;
}
}

View File

@ -134,7 +134,32 @@ private:
*/
MergeTreeData::DataPartsVector selectAllPartsFromPartition(const String & partition_id);
public:
/** Split mutation commands into two parts:
* First part should be executed by mutations interpreter.
* Other is just simple drop/renames, so they can be executed without interpreter.
*/
void splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames) const;
/// Apply commands to source_part i.e. remove some columns in source_part
/// and return set of files, that have to be removed from filesystem and checksums
NameSet collectFilesToRemove(MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension) const;
/// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt.
/// Because we will generate new versions of them after we perform mutation.
NameSet collectFilesToSkip(const Block & updated_header, const std::set<MergeTreeIndexPtr> & indices_to_recalc, const String & mrk_extension) const;
/// Get the columns list of the resulting part in the same order as all_columns.
NamesAndTypesList getColumnsForNewDataPart(MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList all_columns) const;
bool shouldExecuteTTL(const Names & columns, const MutationCommands & commands) const;
public :
/** Is used to cancel all merges and mutations. On cancel() call all currently running actions will throw exception soon.
* All new attempts to start a merge or mutation will throw an exception until all 'LockHolder' objects will be destroyed.
*/

View File

@ -640,6 +640,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
}
merge_tree_reader->evaluateMissingDefaults(block, columns);
merge_tree_reader->performRequiredConversions(columns);
}
read_result.columns.reserve(read_result.columns.size() + columns.size());
@ -659,6 +660,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults({}, read_result.columns);
merge_tree_reader->performRequiredConversions(read_result.columns);
}
else
read_result.columns.clear();

View File

@ -1,10 +1,12 @@
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeArray.h>
#include <Common/escapeForFileName.h>
#include <Columns/ColumnArray.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <Storages/MergeTree/MergeTreeReaderWide.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Storages/MergeTree/IMergeTreeReader.h>
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
@ -30,17 +32,27 @@ MergeTreeReaderWide::MergeTreeReaderWide(
MarkCache * mark_cache_,
const MarkRanges & mark_ranges_,
const MergeTreeReaderSettings & settings_,
const ValueSizeMap & avg_value_size_hints_,
const IMergeTreeDataPart::ValueSizeMap & avg_value_size_hints_,
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
clockid_t clock_type_)
: IMergeTreeReader(data_part_, columns_
, uncompressed_cache_, mark_cache_, mark_ranges_
, settings_, avg_value_size_hints_)
: IMergeTreeReader(
data_part_, columns_, uncompressed_cache_, mark_cache_,
mark_ranges_, settings_, avg_value_size_hints_)
{
try
{
for (const NameAndTypePair & column_from_part : data_part->getColumns())
{
columns_from_part[column_from_part.name] = column_from_part.type;
}
for (const NameAndTypePair & column : columns)
addStreams(column.name, *column.type, profile_callback_, clock_type_);
{
if (columns_from_part.count(column.name))
addStreams(column.name, *columns_from_part[column.name], profile_callback_, clock_type_);
else
addStreams(column.name, *column.type, profile_callback_, clock_type_);
}
}
catch (...)
{
@ -70,12 +82,17 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si
auto name_and_type = columns.begin();
for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type)
{
auto & [name, type] = *name_and_type;
String & name = name_and_type->name;
DataTypePtr type;
if (columns_from_part.count(name))
type = columns_from_part[name];
else
type = name_and_type->type;
/// The column is already present in the block so we will append the values to the end.
bool append = res_columns[pos] != nullptr;
if (!append)
res_columns[pos] = name_and_type->type->createColumn();
res_columns[pos] = type->createColumn();
/// To keep offsets shared. TODO Very dangerous. Get rid of this.
MutableColumnPtr column = res_columns[pos]->assumeMutable();

View File

@ -11,26 +11,6 @@ namespace ErrorCodes
extern const int MEMORY_LIMIT_EXCEEDED;
}
static Block replaceTypes(Block && header, const MergeTreeData::DataPartPtr & data_part)
{
/// Types may be different during ALTER (when this stream is used to perform an ALTER).
/// NOTE: We may use similar code to implement non blocking ALTERs.
for (const auto & name_type : data_part->getColumns())
{
if (header.has(name_type.name))
{
auto & elem = header.getByName(name_type.name);
if (!elem.type->equals(*name_type.type))
{
elem.type = name_type.type;
elem.column = elem.type->createColumn();
}
}
}
return std::move(header);
}
MergeTreeSelectProcessor::MergeTreeSelectProcessor(
const MergeTreeData & storage_,
const MergeTreeData::DataPartPtr & owned_data_part_,
@ -48,7 +28,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
bool quiet)
:
MergeTreeBaseSelectProcessor{
replaceTypes(storage_.getSampleBlockForColumns(required_columns_), owned_data_part_),
storage_.getSampleBlockForColumns(required_columns_),
storage_, prewhere_info_, max_block_size_rows_,
preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_,
reader_settings_, use_uncompressed_cache_, virt_column_names_},

View File

@ -35,7 +35,6 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream(
addTotalRowsApprox(data_part->rows_count);
header = storage.getSampleBlockForColumns(columns_to_read);
fixHeader(header);
/// Add columns because we don't want to read empty blocks
injectRequiredColumns(storage, data_part, columns_to_read);
@ -109,7 +108,11 @@ try
reader->fillMissingColumns(columns, should_evaluate_missing_defaults, rows_readed);
if (should_evaluate_missing_defaults)
{
reader->evaluateMissingDefaults({}, columns);
}
reader->performRequiredConversions(columns);
res = header.cloneEmpty();

View File

@ -1,203 +0,0 @@
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/setThreadName.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Databases/IDatabase.h>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_FOUND_NODE;
}
static const auto ALTER_ERROR_SLEEP_MS = 10 * 1000;
ReplicatedMergeTreeAlterThread::ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_)
: storage(storage_)
, zk_node_cache([&] { return storage.getZooKeeper(); })
, log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeAlterThread)")
, log(&Logger::get(log_name))
{
task = storage_.global_context.getSchedulePool().createTask(log_name, [this]{ run(); });
}
void ReplicatedMergeTreeAlterThread::run()
{
try
{
/** We have a description of columns in ZooKeeper, common for all replicas (Example: /clickhouse/tables/02-06/visits/columns),
* as well as a description of columns in local file with metadata (storage.getColumnsList()).
*
* If these descriptions are different - you need to do ALTER.
*
* If stored version of the node (columns_version) differs from the version in ZK,
* then the description of the columns in ZK does not necessarily differ from the local
* - this can happen with a loop from ALTER-s, which as a whole, does not change anything.
* In this case, you need to update the stored version number,
* and also check the structure of parts, and, if necessary, make ALTER.
*
* Recorded version number needs to be updated after updating the metadata, under lock.
* This version number is checked against the current one for INSERT.
* That is, we make sure to insert blocks with the correct structure.
*
* When the server starts, previous ALTER might not have been completed.
* Therefore, for the first time, regardless of the changes, we check the structure of all parts,
* (Example: /clickhouse/tables/02-06/visits/replicas/example02-06-1.yandex.ru/parts/20140806_20140831_131664_134988_3296/columns)
* and do ALTER if necessary.
*
* TODO: Too complicated, rewrite everything.
*/
auto zookeeper = storage.getZooKeeper();
String columns_path = storage.zookeeper_path + "/columns";
auto columns_znode = zk_node_cache.get(columns_path, task->getWatchCallback());
if (!columns_znode.exists)
throw Exception(columns_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
int32_t columns_version = columns_znode.stat.version;
String metadata_path = storage.zookeeper_path + "/metadata";
auto metadata_znode = zk_node_cache.get(metadata_path, task->getWatchCallback());
if (!metadata_znode.exists)
throw Exception(metadata_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
int32_t metadata_version = metadata_znode.stat.version;
const bool changed_columns_version = (columns_version != storage.columns_version);
const bool changed_metadata_version = (metadata_version != storage.metadata_version);
if (!(changed_columns_version || changed_metadata_version || force_recheck_parts))
return;
const String & columns_str = columns_znode.contents;
auto columns_in_zk = ColumnsDescription::parse(columns_str);
const String & metadata_str = metadata_znode.contents;
auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true);
/// If you need to lock table structure, then suspend merges and moves.
ActionLock merge_blocker = storage.merger_mutator.merges_blocker.cancel();
ActionLock moves_blocker = storage.parts_mover.moves_blocker.cancel();
MergeTreeData::DataParts parts;
/// If metadata nodes have changed, we will update table structure locally.
if (changed_columns_version || changed_metadata_version)
{
/// Temporarily cancel part checks to avoid locking for long time.
auto temporarily_stop_part_checks = storage.part_check_thread.temporarilyStop();
/// Temporarily cancel parts sending
ActionLock data_parts_exchange_blocker;
if (storage.data_parts_exchange_endpoint)
data_parts_exchange_blocker = storage.data_parts_exchange_endpoint->blocker.cancel();
/// Temporarily cancel part fetches
auto fetches_blocker = storage.fetcher.blocker.cancel();
LOG_INFO(log, "Version of metadata nodes in ZooKeeper changed. Waiting for structure write lock.");
auto table_lock = storage.lockExclusively(RWLockImpl::NO_QUERY);
if (columns_in_zk == storage.getColumns() && metadata_diff.empty())
{
LOG_INFO(log, "Metadata nodes changed in ZooKeeper, but their contents didn't change. "
"Most probably it is a cyclic ALTER.");
}
else
{
LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally.");
storage.setTableStructure(std::move(columns_in_zk), metadata_diff);
LOG_INFO(log, "Applied changes to the metadata of the table.");
}
/// You need to get a list of parts under table lock to avoid race condition with merge.
parts = storage.getDataParts();
storage.columns_version = columns_version;
storage.metadata_version = metadata_version;
}
/// Update parts.
if (changed_columns_version || force_recheck_parts)
{
auto table_lock = storage.lockStructureForShare(false, RWLockImpl::NO_QUERY);
if (changed_columns_version)
LOG_INFO(log, "ALTER-ing parts");
int changed_parts = 0;
if (!changed_columns_version)
parts = storage.getDataParts();
const auto columns_for_parts = storage.getColumns().getAllPhysical();
const auto indices_for_parts = storage.getIndices();
for (const MergeTreeData::DataPartPtr & part : parts)
{
/// Update the part and write result to temporary files.
/// TODO: You can skip checking for too large changes if ZooKeeper has, for example,
/// node /flags/force_alter.
MergeTreeData::AlterDataPartTransactionPtr transaction(new MergeTreeData::AlterDataPartTransaction(part));
storage.alterDataPart(columns_for_parts, indices_for_parts.indices, false, transaction);
if (!transaction->isValid())
continue;
storage.updatePartHeaderInZooKeeperAndCommit(zookeeper, *transaction);
++changed_parts;
}
/// Columns sizes could be quietly changed in case of MODIFY/ADD COLUMN
storage.recalculateColumnSizes();
if (changed_columns_version)
{
if (changed_parts != 0)
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
else
LOG_INFO(log, "No parts ALTER-ed");
}
}
/// Update metadata ZK nodes for a specific replica.
if (changed_columns_version || force_recheck_parts)
zookeeper->set(storage.replica_path + "/columns", columns_str);
if (changed_metadata_version || force_recheck_parts)
zookeeper->set(storage.replica_path + "/metadata", metadata_str);
force_recheck_parts = false;
}
catch (const Coordination::Exception & e)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code == Coordination::ZSESSIONEXPIRED)
return;
force_recheck_parts = true;
task->scheduleAfter(ALTER_ERROR_SLEEP_MS);
}
catch (...)
{
tryLogCurrentException(log, __PRETTY_FUNCTION__);
force_recheck_parts = true;
task->scheduleAfter(ALTER_ERROR_SLEEP_MS);
}
}
}

View File

@ -1,42 +0,0 @@
#pragma once
#include <thread>
#include <Core/BackgroundSchedulePool.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <Core/Types.h>
#include <common/logger_useful.h>
namespace DB
{
class StorageReplicatedMergeTree;
/** Keeps track of changing the table structure in ZooKeeper and performs the necessary conversions.
*
* NOTE This has nothing to do with manipulating partitions,
* which are processed through the replication queue.
*/
class ReplicatedMergeTreeAlterThread
{
public:
ReplicatedMergeTreeAlterThread(StorageReplicatedMergeTree & storage_);
void start() { task->activateAndSchedule(); }
void stop() { task->deactivate(); }
private:
void run();
StorageReplicatedMergeTree & storage;
zkutil::ZooKeeperNodeCache zk_node_cache;
String log_name;
Logger * log;
BackgroundSchedulePool::TaskHolder task;
bool force_recheck_parts = true;
};
}

View File

@ -0,0 +1,88 @@
#include <Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h>
#include <cassert>
namespace DB
{
int ReplicatedMergeTreeAltersSequence::getHeadAlterVersion(std::lock_guard<std::mutex> & /*state_lock*/) const
{
/// If queue empty, than we don't have version
if (!queue_state.empty())
return queue_state.begin()->first;
return -1;
}
void ReplicatedMergeTreeAltersSequence::addMutationForAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/)
{
/// Metadata alter can be added before, or
/// maybe already finished if we startup after metadata alter was finished.
if (!queue_state.count(alter_version))
queue_state.emplace(alter_version, AlterState{.metadata_finished=true, .data_finished=false});
else
queue_state[alter_version].data_finished = false;
}
void ReplicatedMergeTreeAltersSequence::addMetadataAlter(
int alter_version, bool have_mutation, std::lock_guard<std::mutex> & /*state_lock*/)
{
if (!queue_state.count(alter_version))
queue_state.emplace(alter_version, AlterState{.metadata_finished=false, .data_finished=!have_mutation});
else /// Data alter can be added before.
queue_state[alter_version].metadata_finished = false;
}
void ReplicatedMergeTreeAltersSequence::finishMetadataAlter(int alter_version, std::unique_lock<std::mutex> & /*state_lock*/)
{
/// Sequence must not be empty
assert(!queue_state.empty());
/// Alters have to be finished in order
assert(queue_state.begin()->first == alter_version);
/// If metadata stage finished (or was never added) than we can remove this alter
if (queue_state[alter_version].data_finished)
queue_state.erase(alter_version);
else
queue_state[alter_version].metadata_finished = true;
}
void ReplicatedMergeTreeAltersSequence::finishDataAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/)
{
/// Queue can be empty after load of finished mutation without move of mutation pointer
if (queue_state.empty())
return;
/// Mutations may finish multiple times (for example, after server restart, before update of mutation pointer)
if (alter_version >= queue_state.begin()->first)
{
/// All alter versions bigger than head must present in queue.
assert(queue_state.count(alter_version));
if (queue_state[alter_version].metadata_finished)
queue_state.erase(alter_version);
else
queue_state[alter_version].data_finished = true;
}
}
bool ReplicatedMergeTreeAltersSequence::canExecuteDataAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/) const
{
/// Queue maybe empty when we start after server shutdown
/// and have some MUTATE_PART records in queue
if (queue_state.empty())
return true;
/// All versions smaller than head, can be executed
if (alter_version < queue_state.begin()->first)
return true;
return queue_state.at(alter_version).metadata_finished;
}
bool ReplicatedMergeTreeAltersSequence::canExecuteMetaAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/) const
{
assert(!queue_state.empty());
/// We can execute only alters of metadata which are in head.
return queue_state.begin()->first == alter_version;
}
}

View File

@ -0,0 +1,63 @@
#pragma once
#include <deque>
#include <mutex>
#include <map>
namespace DB
{
/// ALTERs in StorageReplicatedMergeTree have to be executed sequentially (one
/// by one). But ReplicatedMergeTreeQueue execute all entries almost
/// concurrently. The only depency between entries is data parts, but they are
/// not suitable in alters case.
///
/// This class stores information about current alters in
/// ReplicatedMergeTreeQueue, and control their order of execution. Actually
/// it's a part of ReplicatedMergeTreeQueue and shouldn't be used directly by
/// other classes, also methods have to be called under ReplicatedMergeTreeQueue
/// state lock.
class ReplicatedMergeTreeAltersSequence
{
private:
/// In general case alter consist of two stages Alter data and alter
/// metadata. First we alter storage metadata and then we can apply
/// corresponding data changes (MUTATE_PART). After that, we can remove
/// alter from this sequence (alter is processed).
struct AlterState
{
bool metadata_finished = false;
bool data_finished = false;
};
private:
/// alter_version -> AlterState.
std::map<int, AlterState> queue_state;
public:
/// Add mutation for alter (alter data stage).
void addMutationForAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/);
/// Add metadata for alter (alter metadata stage). If have_mutation=true, than we expect, that
/// corresponding mutation will be added.
void addMetadataAlter(int alter_version, bool have_mutation, std::lock_guard<std::mutex> & /*state_lock*/);
/// Finish metadata alter. If corresponding data alter finished, than we can remove
/// alter from sequence.
void finishMetadataAlter(int alter_version, std::unique_lock <std::mutex> & /*state_lock*/);
/// Finish data alter. If corresponding metadata alter finished, than we can remove
/// alter from sequence.
void finishDataAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/);
/// Check that we can execute this data alter. If it's metadata stage finished.
bool canExecuteDataAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/) const;
/// Check that we can execute metadata alter with version.
bool canExecuteMetaAlter(int alter_version, std::lock_guard<std::mutex> & /*state_lock*/) const;
/// Just returns smallest alter version in sequence (first entry)
int getHeadAlterVersion(std::lock_guard<std::mutex> & /*state_lock*/) const;
};
}

View File

@ -1,6 +1,7 @@
#include <Common/ZooKeeper/Types.h>
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
@ -69,6 +70,22 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
<< source_parts.at(0) << "\n"
<< "to\n"
<< new_part_name;
out << "\nalter_version\n";
out << alter_version;
break;
case ALTER_METADATA: /// Just make local /metadata and /columns consistent with global
out << "alter\n";
out << "alter_version\n";
out << alter_version<< "\n";
out << "have_mutation\n";
out << have_mutation << "\n";
out << "columns_str_size:\n";
out << columns_str.size() << "\n";
out << columns_str << "\n";
out << "metadata_str_size:\n";
out << metadata_str.size() << "\n";
out << metadata_str;
break;
default:
@ -160,6 +177,25 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
>> "to\n"
>> new_part_name;
source_parts.push_back(source_part);
in >> "\nalter_version\n" >> alter_version;
}
else if (type_str == "alter")
{
type = ALTER_METADATA;
in >> "alter_version\n";
in >> alter_version;
in >> "\nhave_mutation\n";
in >> have_mutation;
in >> "\ncolumns_str_size:\n";
size_t columns_size;
in >> columns_size >> "\n";
columns_str.resize(columns_size);
in.readStrict(&columns_str[0], columns_size);
in >> "\nmetadata_str_size:\n";
size_t metadata_size;
in >> metadata_size >> "\n";
metadata_str.resize(metadata_size);
in.readStrict(&metadata_str[0], metadata_size);
}
in >> "\n";

View File

@ -36,6 +36,7 @@ struct ReplicatedMergeTreeLogEntryData
CLEAR_INDEX, /// Drop specific index from specified partition.
REPLACE_RANGE, /// Drop certain range of partitions and replace them by new ones
MUTATE_PART, /// Apply one or several mutations to the part.
ALTER_METADATA, /// Apply alter modification according to global /metadata and /columns paths
};
static String typeToString(Type type)
@ -49,6 +50,7 @@ struct ReplicatedMergeTreeLogEntryData
case ReplicatedMergeTreeLogEntryData::CLEAR_INDEX: return "CLEAR_INDEX";
case ReplicatedMergeTreeLogEntryData::REPLACE_RANGE: return "REPLACE_RANGE";
case ReplicatedMergeTreeLogEntryData::MUTATE_PART: return "MUTATE_PART";
case ReplicatedMergeTreeLogEntryData::ALTER_METADATA: return "ALTER_METADATA";
default:
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR);
}
@ -81,7 +83,7 @@ struct ReplicatedMergeTreeLogEntryData
String index_name;
/// Force filter by TTL in 'OPTIMIZE ... FINAL' query to remove expired values from old parts
/// without TTL infos or with outdated TTL infos, e.g. after 'ALTER ... MODIFY TTL' query.
/// without TTL infos or with outdated TTL infos, e.g. after 'ALTER ... MODIFY TTL' query.
bool force_ttl = false;
/// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory.
@ -105,10 +107,27 @@ struct ReplicatedMergeTreeLogEntryData
std::shared_ptr<ReplaceRangeEntry> replace_range_entry;
/// ALTER METADATA and MUTATE PART command
/// Version of metadata which will be set after this alter
/// Also present in MUTATE_PART command, to track mutations
/// required for complete alter execution.
int alter_version; /// May be equal to -1, if it's normal mutation, not metadata update.
/// only ALTER METADATA command
bool have_mutation; /// If this alter requires additional mutation step, for data update
String columns_str; /// New columns data corresponding to alter_version
String metadata_str; /// New metadata corresponding to alter_version
/// Returns a set of parts that will appear after executing the entry + parts to block
/// selection of merges. These parts are added to queue.virtual_parts.
Strings getVirtualPartNames() const
{
/// Doesn't produce any part
if (type == ALTER_METADATA)
return {};
/// DROP_RANGE does not add a real part, but we must disable merges in that range
if (type == DROP_RANGE)
return {new_part_name};
@ -156,6 +175,12 @@ struct ReplicatedMergeTreeLogEntryData
/// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled.
size_t quorum = 0;
/// If this MUTATE_PART entry caused by alter(modify/drop) query.
bool isAlterMutation() const
{
return type == MUTATE_PART && alter_version != -1;
}
};

View File

@ -23,6 +23,11 @@ void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const
out << "commands: ";
commands.writeText(out);
out << "\n";
out << "alter version: ";
out << alter_version;
}
void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in)
@ -47,6 +52,7 @@ void ReplicatedMergeTreeMutationEntry::readText(ReadBuffer & in)
in >> "commands: ";
commands.readText(in);
in >> "\nalter version: " >> alter_version;
}
String ReplicatedMergeTreeMutationEntry::toString() const

View File

@ -13,6 +13,11 @@ namespace DB
class ReadBuffer;
class WriteBuffer;
/// Mutation entry in /mutations path in zookeeper. This record contains information about blocks
/// in patitions. We will mutatate all parts with left number less than this numbers.
///
/// These entries processed separately from main replication /log, and produce other entries
/// -- MUTATE_PART in main replication log.
struct ReplicatedMergeTreeMutationEntry
{
void writeText(WriteBuffer & out) const;
@ -21,13 +26,27 @@ struct ReplicatedMergeTreeMutationEntry
String toString() const;
static ReplicatedMergeTreeMutationEntry parse(const String & str, String znode_name);
/// Name of znode (mutation-xxxxxxx)
String znode_name;
/// Create time of znode
time_t create_time = 0;
/// Replica which initiated mutation
String source_replica;
/// Accuired numbers of blocks
/// partition_id -> block_number
std::map<String, Int64> block_numbers;
/// Mutation commands which will give to MUTATE_PART entries
MutationCommands commands;
/// Version of metadata. Not equal to -1 only if this mutation
/// was created by ALTER MODIFY/DROP queries.
int alter_version = -1;
bool isAlterMutation() const { return alter_version != -1; }
};
using ReplicatedMergeTreeMutationEntryPtr = std::shared_ptr<const ReplicatedMergeTreeMutationEntry>;

View File

@ -125,7 +125,7 @@ void ReplicatedMergeTreeQueue::initialize(
void ReplicatedMergeTreeQueue::insertUnlocked(
const LogEntryPtr & entry, std::optional<time_t> & min_unprocessed_insert_time_changed,
std::lock_guard<std::mutex> & /* state_lock */)
std::lock_guard<std::mutex> & state_lock)
{
for (const String & virtual_part_name : entry->getVirtualPartNames())
{
@ -149,6 +149,11 @@ void ReplicatedMergeTreeQueue::insertUnlocked(
min_unprocessed_insert_time_changed = min_unprocessed_insert_time;
}
}
if (entry->type == LogEntry::ALTER_METADATA)
{
LOG_TRACE(log, "Adding alter metadata version " << entry->alter_version << " to the queue");
alter_sequence.addMetadataAlter(entry->alter_version, entry->have_mutation, state_lock);
}
}
@ -170,7 +175,7 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
bool is_successful,
std::optional<time_t> & min_unprocessed_insert_time_changed,
std::optional<time_t> & max_processed_insert_time_changed,
std::unique_lock<std::mutex> & /* queue_lock */)
std::unique_lock<std::mutex> & state_lock)
{
/// Update insert times.
if (entry->type == LogEntry::GET_PART)
@ -225,6 +230,12 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
current_parts.remove(drop_range_part_name);
virtual_parts.remove(drop_range_part_name);
}
if (entry->type == LogEntry::ALTER_METADATA)
{
LOG_TRACE(log, "Finishing metadata alter with version " << entry->alter_version);
alter_sequence.finishMetadataAlter(entry->alter_version, state_lock);
}
}
else
{
@ -234,6 +245,7 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval(
/// so we won't need to mutate it.
removePartFromMutations(virtual_part_name);
}
}
}
@ -251,6 +263,7 @@ void ReplicatedMergeTreeQueue::removePartFromMutations(const String & part_name)
for (auto it = from_it; it != in_partition->second.end(); ++it)
{
MutationStatus & status = *it->second;
status.parts_to_do.removePartAndCoveredParts(part_name);
if (status.parts_to_do.size() == 0)
some_mutations_are_probably_done = true;
@ -537,6 +550,7 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, C
}
catch (...)
{
tryLogCurrentException(log);
/// If it fails, the data in RAM is incorrect. In order to avoid possible further corruption of data in ZK, we will kill ourselves.
/// This is possible only if there is an unknown logical error.
std::terminate();
@ -552,7 +566,10 @@ void ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, C
}
static Names getPartNamesToMutate(
namespace
{
Names getPartNamesToMutate(
const ReplicatedMergeTreeMutationEntry & mutation, const ActiveDataPartSet & parts)
{
Names result;
@ -576,6 +593,7 @@ static Names getPartNamesToMutate(
return result;
}
}
void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback)
{
@ -678,7 +696,16 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C
}
if (mutation.parts_to_do.size() == 0)
{
some_mutations_are_probably_done = true;
}
/// otherwise it's already done
if (entry->isAlterMutation() && entry->znode_name > mutation_pointer)
{
LOG_TRACE(log, "Adding mutation " << entry->znode_name << " with alter version " << entry->alter_version << " to the queue");
alter_sequence.addMutationForAlter(entry->alter_version, state_lock);
}
}
}
@ -719,6 +746,12 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation(
mutations_by_partition.erase(partition_and_block_num.first);
}
if (entry->isAlterMutation())
{
LOG_DEBUG(log, "Removed alter " << entry->alter_version << " because mutation " + entry->znode_name + " were killed.");
alter_sequence.finishDataAlter(entry->alter_version, state_lock);
}
mutations_by_znode.erase(it);
LOG_DEBUG(log, "Removed mutation " + entry->znode_name + " from local state.");
}
@ -938,15 +971,17 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
String & out_postpone_reason,
MergeTreeDataMergerMutator & merger_mutator,
MergeTreeData & data,
std::lock_guard<std::mutex> & queue_lock) const
std::lock_guard<std::mutex> & state_lock) const
{
/// If our entry produce part which is alredy covered by
/// some other entry which is currently executing, then we can postpone this entry.
if (entry.type == LogEntry::MERGE_PARTS
|| entry.type == LogEntry::GET_PART
|| entry.type == LogEntry::MUTATE_PART)
{
for (const String & new_part_name : entry.getBlockingPartNames())
{
if (!isNotCoveredByFuturePartsImpl(new_part_name, out_postpone_reason, queue_lock))
if (!isNotCoveredByFuturePartsImpl(new_part_name, out_postpone_reason, state_lock))
{
if (!out_postpone_reason.empty())
LOG_DEBUG(log, out_postpone_reason);
@ -1016,13 +1051,44 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
String range_name = (entry.type == LogEntry::REPLACE_RANGE) ? entry.replace_range_entry->drop_range_part_name : entry.new_part_name;
auto range = MergeTreePartInfo::fromPartName(range_name, format_version);
if (0 != getConflictsCountForRange(range, entry, &conflicts_description, queue_lock))
if (0 != getConflictsCountForRange(range, entry, &conflicts_description, state_lock))
{
LOG_DEBUG(log, conflicts_description);
return false;
}
}
/// Alters must be executed one by one. First metadata change, and after that data alter (MUTATE_PART entries with).
/// corresponding alter_version.
if (entry.type == LogEntry::ALTER_METADATA)
{
if (!alter_sequence.canExecuteMetaAlter(entry.alter_version, state_lock))
{
int head_alter = alter_sequence.getHeadAlterVersion(state_lock);
out_postpone_reason = "Cannot execute alter metadata with version: " + std::to_string(entry.alter_version)
+ " because another alter " + std::to_string(head_alter)
+ " must be executed before";
return false;
}
}
/// If this MUTATE_PART is part of alter modify/drop query, than we have to execute them one by one
if (entry.isAlterMutation())
{
if (!alter_sequence.canExecuteDataAlter(entry.alter_version, state_lock))
{
int head_alter = alter_sequence.getHeadAlterVersion(state_lock);
if (head_alter == entry.alter_version)
out_postpone_reason = "Cannot execute alter data with version: "
+ std::to_string(entry.alter_version) + " because metadata still not altered";
else
out_postpone_reason = "Cannot execute alter data with version: " + std::to_string(entry.alter_version)
+ " because another alter " + std::to_string(head_alter) + " must be executed before";
return false;
}
}
return true;
}
@ -1119,6 +1185,8 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP
if (shouldExecuteLogEntry(**it, (*it)->postpone_reason, merger_mutator, data, lock))
{
entry = *it;
/// We gave a chance for the entry, move it to the tail of the queue, after that
/// we move it to the end of the queue.
queue.splice(queue.end(), queue, it);
break;
}
@ -1145,6 +1213,8 @@ bool ReplicatedMergeTreeQueue::processEntry(
try
{
/// We don't have any backoff for failed entries
/// we just count amount of tries for each ot them.
if (func(entry))
removeProcessedEntry(get_zookeeper(), entry);
}
@ -1299,6 +1369,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
{
LOG_TRACE(log, "Marking mutation " << znode << " done because it is <= mutation_pointer (" << mutation_pointer << ")");
mutation.is_done = true;
alter_sequence.finishDataAlter(mutation.entry->alter_version, lock);
if (mutation.parts_to_do.size() != 0)
{
LOG_INFO(log, "Seems like we jumped over mutation " << znode << " when downloaded part with bigger mutation number."
@ -1341,6 +1412,11 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
{
LOG_TRACE(log, "Mutation " << entry->znode_name << " is done");
it->second.is_done = true;
if (entry->isAlterMutation())
{
LOG_TRACE(log, "Finishing data alter with version " << entry->alter_version << " for entry " << entry->znode_name);
alter_sequence.finishDataAlter(entry->alter_version, lock);
}
}
}
}
@ -1687,7 +1763,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()(
}
std::optional<Int64> ReplicatedMergeTreeMergePredicate::getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const
std::optional<std::pair<Int64, int>> ReplicatedMergeTreeMergePredicate::getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const
{
/// Assigning mutations is easier than assigning merges because mutations appear in the same order as
/// the order of their version numbers (see StorageReplicatedMergeTree::mutate).
@ -1712,10 +1788,22 @@ std::optional<Int64> ReplicatedMergeTreeMergePredicate::getDesiredMutationVersio
Int64 current_version = queue.getCurrentMutationVersionImpl(part->info.partition_id, part->info.getDataVersion(), lock);
Int64 max_version = in_partition->second.rbegin()->first;
int alter_version = -1;
for (auto [mutation_version, mutation_status] : in_partition->second)
{
max_version = mutation_version;
if (mutation_version > current_version && mutation_status->entry->alter_version != -1)
{
alter_version = mutation_status->entry->alter_version;
break;
}
}
if (current_version >= max_version)
return {};
return max_version;
return std::make_pair(max_version, alter_version);
}
@ -1795,4 +1883,10 @@ String padIndex(Int64 index)
return std::string(10 - index_str.size(), '0') + index_str;
}
void ReplicatedMergeTreeQueue::removeCurrentPartsFromMutations()
{
std::lock_guard state_lock(state_mutex);
for (const auto & part_name : current_parts.getParts())
removePartFromMutations(part_name);
}
}

View File

@ -9,6 +9,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumAddedParts.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAltersSequence.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Core/BackgroundSchedulePool.h>
@ -67,7 +68,8 @@ private:
ActiveDataPartSet current_parts;
/** The queue of what you need to do on this line to catch up. It is taken from ZooKeeper (/replicas/me/queue/).
* In ZK records in chronological order. Here it is not necessary.
* In ZK records in chronological order. Here they are executed in parallel and reorder after entry execution.
* Order of execution is not "queue" at all. Look at selectEntryToProcess.
*/
Queue queue;
@ -120,15 +122,19 @@ private:
String latest_fail_reason;
};
/// Mapping from znode path to Mutations Status
std::map<String, MutationStatus> mutations_by_znode;
/// Partition -> (block_number -> MutationStatus)
std::unordered_map<String, std::map<Int64, MutationStatus *>> mutations_by_partition;
/// Znode ID of the latest mutation that is done.
String mutation_pointer;
/// Provides only one simultaneous call to pullLogsToQueue.
std::mutex pull_logs_to_queue_mutex;
/// This sequence control ALTERs execution in replication queue.
/// We need it because alters have to be executed sequentially (one by one).
ReplicatedMergeTreeAltersSequence alter_sequence;
/// List of subscribers
/// A subscriber callback is called when an entry queue is deleted
@ -163,6 +169,7 @@ private:
/// Put a set of (already existing) parts in virtual_parts.
void addVirtualParts(const MergeTreeData::DataParts & parts);
/// Insert new entry from log into queue
void insertUnlocked(
const LogEntryPtr & entry, std::optional<time_t> & min_unprocessed_insert_time_changed,
std::lock_guard<std::mutex> & state_lock);
@ -272,7 +279,7 @@ public:
void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {});
/// Remove a mutation from ZooKeeper and from the local set. Returns the removed entry or nullptr
/// if it could not be found.
/// if it could not be found. Called during KILL MUTATION query execution.
ReplicatedMergeTreeMutationEntryPtr removeMutation(zkutil::ZooKeeperPtr zookeeper, const String & mutation_id);
/** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM).
@ -375,6 +382,8 @@ public:
void getInsertTimes(time_t & out_min_unprocessed_insert_time, time_t & out_max_processed_insert_time) const;
std::vector<MergeTreeMutationStatus> getMutationsStatus() const;
void removeCurrentPartsFromMutations();
};
class ReplicatedMergeTreeMergePredicate
@ -389,9 +398,12 @@ public:
const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right,
String * out_reason = nullptr) const;
/// Return nonempty optional if the part can and should be mutated.
/// Returned mutation version number is always the biggest possible.
std::optional<Int64> getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const;
/// Return nonempty optional of desired mutation version and alter version.
/// If we have no alter (modify/drop) mutations in mutations queue, than we return biggest possible
/// mutation version (and -1 as alter version). In other case, we return biggest mutation version with
/// smallest alter version. This required, because we have to execute alter mutations sequentially and
/// don't glue them together. Alter is rare operation, so it shouldn't affect performance.
std::optional<std::pair<Int64, int>> getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const;
bool isMutationFinished(const ReplicatedMergeTreeMutationEntry & mutation) const;

View File

@ -180,6 +180,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
/// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost);
/// because cleanup_thread doesn't delete log_pointer of active replicas.
storage.queue.pullLogsToQueue(zookeeper);
storage.queue.removeCurrentPartsFromMutations();
storage.last_queue_update_finish_time.store(time(nullptr));
updateQuorumIfWeHavePart();
@ -199,7 +200,6 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
storage.mutations_updating_task->activateAndSchedule();
storage.mutations_finalizing_task->activateAndSchedule();
storage.cleanup_thread.start();
storage.alter_thread.start();
storage.part_check_thread.start();
return true;
@ -346,7 +346,6 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown()
storage.mutations_finalizing_task->deactivate();
storage.cleanup_thread.stop();
storage.alter_thread.stop();
storage.part_check_thread.stop();
LOG_TRACE(log, "Threads finished");

View File

@ -150,11 +150,9 @@ ReplicatedMergeTreeTableMetadata ReplicatedMergeTreeTableMetadata::parse(const S
return metadata;
}
ReplicatedMergeTreeTableMetadata::Diff
ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const
{
Diff diff;
void ReplicatedMergeTreeTableMetadata::checkImmutableFieldsEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
if (data_format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
{
if (date_column != from_zk.date_column)
@ -163,10 +161,12 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
ErrorCodes::METADATA_MISMATCH);
}
else if (!from_zk.date_column.empty())
{
throw Exception(
"Existing table metadata in ZooKeeper differs in date index column."
" Stored in ZooKeeper: " + from_zk.date_column + ", local is custom-partitioned.",
ErrorCodes::METADATA_MISMATCH);
}
if (sampling_expression != from_zk.sampling_expression)
throw Exception("Existing table metadata in ZooKeeper differs in sample expression."
@ -208,63 +208,46 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
" Stored in ZooKeeper: " + from_zk.partition_key + ", local: " + partition_key,
ErrorCodes::METADATA_MISMATCH);
}
void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
checkImmutableFieldsEquals(from_zk);
if (sorting_key != from_zk.sorting_key)
{
if (allow_alter)
{
diff.sorting_key_changed = true;
diff.new_sorting_key = from_zk.sorting_key;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in sorting key expression."
" Stored in ZooKeeper: " + from_zk.sorting_key + ", local: " + sorting_key,
ErrorCodes::METADATA_MISMATCH);
throw Exception(
"Existing table metadata in ZooKeeper differs in sorting key expression."
" Stored in ZooKeeper: " + from_zk.sorting_key + ", local: " + sorting_key,
ErrorCodes::METADATA_MISMATCH);
}
if (ttl_table != from_zk.ttl_table)
{
if (allow_alter)
{
diff.ttl_table_changed = true;
diff.new_ttl_table = from_zk.ttl_table;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in TTL."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
throw Exception(
"Existing table metadata in ZooKeeper differs in TTL."
" Stored in ZooKeeper: " + from_zk.ttl_table +
", local: " + ttl_table,
ErrorCodes::METADATA_MISMATCH);
}
if (skip_indices != from_zk.skip_indices)
{
if (allow_alter)
{
diff.skip_indices_changed = true;
diff.new_skip_indices = from_zk.skip_indices;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in skip indexes."
" Stored in ZooKeeper: " + from_zk.skip_indices +
", local: " + skip_indices,
ErrorCodes::METADATA_MISMATCH);
throw Exception(
"Existing table metadata in ZooKeeper differs in skip indexes."
" Stored in ZooKeeper: " + from_zk.skip_indices +
", local: " + skip_indices,
ErrorCodes::METADATA_MISMATCH);
}
if (constraints != from_zk.constraints)
{
if (allow_alter)
{
diff.constraints_changed = true;
diff.new_constraints = from_zk.constraints;
}
else
throw Exception(
"Existing table metadata in ZooKeeper differs in constraints."
" Stored in ZooKeeper: " + from_zk.constraints +
", local: " + constraints,
ErrorCodes::METADATA_MISMATCH);
throw Exception(
"Existing table metadata in ZooKeeper differs in constraints."
" Stored in ZooKeeper: " + from_zk.constraints +
", local: " + constraints,
ErrorCodes::METADATA_MISMATCH);
}
if (from_zk.index_granularity_bytes_found_in_zk && index_granularity_bytes != from_zk.index_granularity_bytes)
@ -272,6 +255,39 @@ ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTabl
" Stored in ZooKeeper: " + DB::toString(from_zk.index_granularity_bytes) +
", local: " + DB::toString(index_granularity_bytes),
ErrorCodes::METADATA_MISMATCH);
}
ReplicatedMergeTreeTableMetadata::Diff
ReplicatedMergeTreeTableMetadata::checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk) const
{
checkImmutableFieldsEquals(from_zk);
Diff diff;
if (sorting_key != from_zk.sorting_key)
{
diff.sorting_key_changed = true;
diff.new_sorting_key = from_zk.sorting_key;
}
if (ttl_table != from_zk.ttl_table)
{
diff.ttl_table_changed = true;
diff.new_ttl_table = from_zk.ttl_table;
}
if (skip_indices != from_zk.skip_indices)
{
diff.skip_indices_changed = true;
diff.new_skip_indices = from_zk.skip_indices;
}
if (constraints != from_zk.constraints)
{
diff.constraints_changed = true;
diff.new_constraints = from_zk.constraints;
}
return diff;
}

View File

@ -3,6 +3,7 @@
#include <Parsers/IAST.h>
#include <Storages/MergeTree/MergeTreeDataFormatVersion.h>
#include <Core/Types.h>
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
{
@ -59,10 +60,16 @@ struct ReplicatedMergeTreeTableMetadata
}
};
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk, bool allow_alter) const;
void checkEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const;
Diff checkAndFindDiff(const ReplicatedMergeTreeTableMetadata & from_zk) const;
private:
void checkImmutableFieldsEquals(const ReplicatedMergeTreeTableMetadata & from_zk) const;
bool index_granularity_bytes_found_in_zk = false;
};
}

View File

@ -651,17 +651,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
StorageInMemoryMetadata metadata{
.columns = args.columns,
.indices = indices_description,
.constraints = args.constraints,
.partition_by_ast = partition_by_ast,
.order_by_ast = order_by_ast,
.primary_key_ast = primary_key_ast,
.ttl_for_table_ast = ttl_table_ast,
.sample_by_ast = sample_by_ast,
.settings_ast = settings_ast,
};
StorageInMemoryMetadata metadata(args.columns, indices_description, args.constraints);
metadata.partition_by_ast = partition_by_ast;
metadata.order_by_ast = order_by_ast;
metadata.primary_key_ast = primary_key_ast;
metadata.ttl_for_table_ast = ttl_table_ast;
metadata.sample_by_ast = sample_by_ast;
metadata.settings_ast = settings_ast;
if (replicated)
return StorageReplicatedMergeTree::create(
zookeeper_path, replica_name, args.attach, args.table_id, args.relative_data_path,

View File

@ -2,12 +2,14 @@
#include <IO/Operators.h>
#include <Parsers/formatAST.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
@ -19,7 +21,7 @@ namespace ErrorCodes
extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN;
}
std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command, bool from_zookeeper)
{
if (command->type == ASTAlterCommand::DELETE)
{
@ -55,6 +57,31 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
res.index_name = command->index->as<ASTIdentifier &>().name;
return res;
}
else if (from_zookeeper && command->type == ASTAlterCommand::MODIFY_COLUMN)
{
MutationCommand res;
res.ast = command->ptr();
res.type = MutationCommand::Type::READ_COLUMN;
const auto & ast_col_decl = command->col_decl->as<ASTColumnDeclaration &>();
res.column_name = ast_col_decl.name;
res.data_type = DataTypeFactory::instance().get(ast_col_decl.type);
return res;
}
else if (from_zookeeper && command->type == ASTAlterCommand::DROP_COLUMN)
{
MutationCommand res;
res.ast = command->ptr();
res.type = MutationCommand::Type::DROP_COLUMN;
res.column_name = getIdentifierName(command->column);
return res;
}
else if (from_zookeeper && command->type == ASTAlterCommand::DROP_INDEX)
{
MutationCommand res;
res.ast = command->ptr();
res.type = MutationCommand::Type::DROP_INDEX;
res.column_name = command->index->as<ASTIdentifier &>().name;
}
else if (command->type == ASTAlterCommand::MATERIALIZE_TTL)
{
MutationCommand res;
@ -62,8 +89,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
res.type = MATERIALIZE_TTL;
return res;
}
else
return {};
return {};
}
@ -92,7 +118,7 @@ void MutationCommands::readText(ReadBuffer & in)
p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0);
for (ASTAlterCommand * command_ast : commands_ast->as<ASTAlterCommandList &>().commands)
{
auto command = MutationCommand::parse(command_ast);
auto command = MutationCommand::parse(command_ast, true);
if (!command)
throw Exception("Unknown mutation command type: " + DB::toString<int>(command_ast->type), ErrorCodes::UNKNOWN_MUTATION_COMMAND);
push_back(std::move(*command));

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTAlterQuery.h>
#include <Storages/IStorage_fwd.h>
#include <DataTypes/IDataType.h>
#include <Core/Names.h>
#include <optional>
@ -27,6 +28,9 @@ struct MutationCommand
DELETE,
UPDATE,
MATERIALIZE_INDEX,
READ_COLUMN,
DROP_COLUMN,
DROP_INDEX,
MATERIALIZE_TTL
};
@ -42,7 +46,12 @@ struct MutationCommand
String index_name;
ASTPtr partition;
static std::optional<MutationCommand> parse(ASTAlterCommand * command);
/// For reads, drops and etc.
String column_name;
DataTypePtr data_type; /// Maybe empty if we just want to drop column
/// If from_zookeeper, than consider more Alter commands as mutation commands
static std::optional<MutationCommand> parse(ASTAlterCommand * command, bool from_zookeeper=false);
};
/// Multiple mutation commands, possible from different ALTER queries
@ -53,9 +62,6 @@ public:
void writeText(WriteBuffer & out) const;
void readText(ReadBuffer & in);
/// Extra columns that we need to read except ones needed for expressions.
Names additional_columns;
};
}

View File

@ -0,0 +1,80 @@
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
{
StorageInMemoryMetadata::StorageInMemoryMetadata(
const ColumnsDescription & columns_,
const IndicesDescription & indices_,
const ConstraintsDescription & constraints_)
: columns(columns_)
, indices(indices_)
, constraints(constraints_)
{
}
StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other)
: columns(other.columns)
, indices(other.indices)
, constraints(other.constraints)
{
if (other.partition_by_ast)
partition_by_ast = other.partition_by_ast->clone();
if (other.order_by_ast)
order_by_ast = other.order_by_ast->clone();
if (other.primary_key_ast)
primary_key_ast = other.primary_key_ast->clone();
if (other.ttl_for_table_ast)
ttl_for_table_ast = other.ttl_for_table_ast->clone();
if (other.sample_by_ast)
sample_by_ast = other.sample_by_ast->clone();
if (other.settings_ast)
settings_ast = other.settings_ast->clone();
if (other.select)
select = other.select->clone();
}
StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemoryMetadata & other)
{
columns = other.columns;
indices = other.indices;
constraints = other.constraints;
if (other.partition_by_ast)
partition_by_ast = other.partition_by_ast->clone();
else
partition_by_ast.reset();
if (other.order_by_ast)
order_by_ast = other.order_by_ast->clone();
else
order_by_ast.reset();
if (other.primary_key_ast)
primary_key_ast = other.primary_key_ast->clone();
else
primary_key_ast.reset();
if (other.ttl_for_table_ast)
ttl_for_table_ast = other.ttl_for_table_ast->clone();
else
ttl_for_table_ast.reset();
if (other.sample_by_ast)
sample_by_ast = other.sample_by_ast->clone();
else
sample_by_ast.reset();
if (other.settings_ast)
settings_ast = other.settings_ast->clone();
else
settings_ast.reset();
if (other.select)
select = other.select->clone();
else
select.reset();
return *this;
}
}

View File

@ -35,6 +35,12 @@ struct StorageInMemoryMetadata
ASTPtr settings_ast = nullptr;
/// SELECT QUERY. Supported for MaterializedView only.
ASTPtr select = nullptr;
StorageInMemoryMetadata(const StorageInMemoryMetadata & other);
StorageInMemoryMetadata() = default;
StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_);
StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other);
};
}

View File

@ -166,13 +166,9 @@ bool StorageMaterializedView::hasColumn(const String & column_name) const
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
{
return
{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
.select = getSelectQuery(),
};
StorageInMemoryMetadata result(getColumns(), getIndices(), getConstraints());
result.select = getSelectQuery();
return result;
}
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context) const

View File

@ -109,6 +109,7 @@ namespace ErrorCodes
extern const int KEEPER_EXCEPTION;
extern const int ALL_REPLICAS_LOST;
extern const int REPLICA_STATUS_CHANGED;
extern const int CANNOT_ASSIGN_ALTER;
}
namespace ActionLocks
@ -178,7 +179,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
, queue(*this)
, fetcher(*this)
, cleanup_thread(*this)
, alter_thread(*this)
, part_check_thread(*this)
, restarting_thread(*this)
{
@ -246,21 +246,38 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
createTableIfNotExists();
checkTableStructure(false, false);
checkTableStructure(zookeeper_path);
Coordination::Stat metadata_stat;
current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
metadata_version = metadata_stat.version;
createReplica();
}
else
{
checkTableStructure(skip_sanity_checks, true);
checkTableStructure(replica_path);
checkParts(skip_sanity_checks);
/// Temporary directories contain unfinalized results of Merges or Fetches (after forced restart)
if (current_zookeeper->exists(replica_path + "/metadata_version"))
{
metadata_version = parse<int>(current_zookeeper->get(replica_path + "/metadata_version"));
}
else /// This replica was created on old version, so we have to take version of global node
{
Coordination::Stat metadata_stat;
current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
metadata_version = metadata_stat.version;
}
/// Temporary directories contain untinalized results of Merges or Fetches (after forced restart)
/// and don't allow to reinitialize them, so delete each of them immediately
clearOldTemporaryDirectories(0);
}
createNewZooKeeperNodes();
other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper();
}
@ -362,9 +379,6 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
/// Mutations
zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
/// ALTERs of the metadata node.
zookeeper->createIfNotExists(replica_path + "/metadata", String());
}
@ -414,45 +428,24 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
/** Verify that list of columns and table storage_settings_ptr match those specified in ZK (/ metadata).
* If not, throw an exception.
*/
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_prefix)
{
auto zookeeper = getZooKeeper();
ReplicatedMergeTreeTableMetadata old_metadata(*this);
Coordination::Stat metadata_stat;
String metadata_str = zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
String metadata_str = zookeeper->get(zookeeper_prefix + "/metadata", &metadata_stat);
auto metadata_from_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
auto metadata_diff = old_metadata.checkAndFindDiff(metadata_from_zk, allow_alter);
metadata_version = metadata_stat.version;
old_metadata.checkEquals(metadata_from_zk);
Coordination::Stat columns_stat;
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &columns_stat));
columns_version = columns_stat.version;
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat));
const ColumnsDescription & old_columns = getColumns();
if (columns_from_zk != old_columns || !metadata_diff.empty())
if (columns_from_zk != old_columns)
{
if (allow_alter &&
(skip_sanity_checks ||
old_columns.getOrdinary().sizeOfDifference(columns_from_zk.getOrdinary()) +
old_columns.getMaterialized().sizeOfDifference(columns_from_zk.getMaterialized()) <= 2))
{
LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
/// We delay setting table structure till startup() because otherwise new table metadata file can
/// be overwritten in DatabaseOrdinary::createTable.
set_table_structure_at_startup = [columns_from_zk, metadata_diff, this]()
{
/// Without any locks, because table has not been created yet.
setTableStructure(std::move(columns_from_zk), metadata_diff);
};
}
else
{
throw Exception("Table structure in ZooKeeper is too different from local table structure",
ErrorCodes::INCOMPATIBLE_COLUMNS);
}
throw Exception("Table columns structure in ZooKeeper is different from local table structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
}
}
@ -554,7 +547,9 @@ void StorageReplicatedMergeTree::createReplica()
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this).toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent));
/// Check version of /replicas to see if there are any replicas created at the same moment of time.
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version));
@ -727,9 +722,6 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
if (part_name.empty())
part_name = part->name;
check(part->getColumns());
int expected_columns_version = columns_version;
auto local_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(
part->getColumns(), part->checksums);
@ -800,8 +792,8 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
const auto storage_settings_ptr = getSettings();
String part_path = replica_path + "/parts/" + part_name;
ops.emplace_back(zkutil::makeCheckRequest(
zookeeper_path + "/columns", expected_columns_version));
//ops.emplace_back(zkutil::makeCheckRequest(
// zookeeper_path + "/columns", expected_columns_version));
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
@ -943,12 +935,19 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
}
else if (entry.type == LogEntry::MERGE_PARTS)
{
/// Sometimes it's better to fetch merged part instead of merge
/// For example when we don't have all source parts for merge
do_fetch = !tryExecuteMerge(entry);
}
else if (entry.type == LogEntry::MUTATE_PART)
{
/// Sometimes it's better to fetch mutated part instead of merge
do_fetch = !tryExecutePartMutation(entry);
}
else if (entry.type == LogEntry::ALTER_METADATA)
{
return executeMetadataAlter(entry);
}
else
{
throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)), ErrorCodes::LOGICAL_ERROR);
@ -1126,6 +1125,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
}
bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry)
{
const String & source_part_name = entry.source_parts.at(0);
@ -1249,6 +1249,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
{
/// Looking for covering part. After that entry.actual_new_part_name may be filled.
String replica = findReplicaHavingCoveringPart(entry, true);
const auto storage_settings_ptr = getSettings();
@ -1387,7 +1388,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
try
{
if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name;
if (!fetchPart(part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
return false;
}
catch (Exception & e)
@ -2255,11 +2257,11 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
if (part->bytes_on_disk > max_source_part_size_for_mutation)
continue;
std::optional<Int64> desired_mutation_version = merge_pred.getDesiredMutationVersion(part);
std::optional<std::pair<Int64, int>> desired_mutation_version = merge_pred.getDesiredMutationVersion(part);
if (!desired_mutation_version)
continue;
if (createLogEntryToMutatePart(*part, *desired_mutation_version))
if (createLogEntryToMutatePart(*part, desired_mutation_version->first, desired_mutation_version->second))
{
success = true;
break;
@ -2286,6 +2288,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
void StorageReplicatedMergeTree::mutationsFinalizingTask()
{
LOG_DEBUG(log, "Trying to finalize mutations");
bool needs_reschedule = false;
try
@ -2364,7 +2367,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
}
bool StorageReplicatedMergeTree::createLogEntryToMutatePart(const IMergeTreeDataPart & part, Int64 mutation_version)
bool StorageReplicatedMergeTree::createLogEntryToMutatePart(const IMergeTreeDataPart & part, Int64 mutation_version, int alter_version)
{
auto zookeeper = getZooKeeper();
@ -2394,6 +2397,7 @@ bool StorageReplicatedMergeTree::createLogEntryToMutatePart(const IMergeTreeData
entry.source_parts.push_back(part.name);
entry.new_part_name = new_part_name;
entry.create_time = time(nullptr);
entry.alter_version = alter_version;
zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
return true;
@ -2569,10 +2573,6 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entr
return {};
}
}
else
{
entry.actual_new_part_name = entry.new_part_name;
}
return replica;
}
@ -2890,9 +2890,6 @@ void StorageReplicatedMergeTree::startup()
if (is_readonly)
return;
if (set_table_structure_at_startup)
set_table_structure_at_startup();
queue.initialize(
zookeeper_path, replica_path,
getStorageID().getFullTableName() + " (ReplicatedMergeTreeQueue)",
@ -2901,15 +2898,15 @@ void StorageReplicatedMergeTree::startup()
data_parts_exchange_endpoint = std::make_shared<DataPartsExchange::Service>(*this);
global_context.getInterserverIOHandler().addEndpoint(data_parts_exchange_endpoint->getId(replica_path), data_parts_exchange_endpoint);
queue_task_handle = global_context.getBackgroundPool().addTask([this] { return queueTask(); });
if (areBackgroundMovesNeeded())
move_parts_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); });
/// In this thread replica will be activated.
restarting_thread.start();
/// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attmept to do it
startup_event.wait();
queue_task_handle = global_context.getBackgroundPool().addTask([this] { return queueTask(); });
if (areBackgroundMovesNeeded())
move_parts_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); });
}
@ -3173,323 +3170,232 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p
return true;
}
bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMergeTree::LogEntry & entry)
{
auto zookeeper = getZooKeeper();
auto columns_from_entry = ColumnsDescription::parse(entry.columns_str);
auto metadata_from_entry = ReplicatedMergeTreeTableMetadata::parse(entry.metadata_str);
auto metadata_diff = ReplicatedMergeTreeTableMetadata(*this).checkAndFindDiff(metadata_from_entry);
MergeTreeData::DataParts parts;
/// If metadata nodes have changed, we will update table structure locally.
Coordination::Requests requests;
requests.emplace_back(zkutil::makeSetRequest(replica_path + "/columns", entry.columns_str, -1));
requests.emplace_back(zkutil::makeSetRequest(replica_path + "/metadata", entry.metadata_str, -1));
zookeeper->multi(requests);
{
/// TODO (relax this lock)
auto table_lock = lockExclusively(RWLockImpl::NO_QUERY);
LOG_INFO(log, "Metadata changed in ZooKeeper. Applying changes locally.");
setTableStructure(std::move(columns_from_entry), metadata_diff);
metadata_version = entry.alter_version;
LOG_INFO(log, "Applied changes to the metadata of the table. Current metadata version: " << metadata_version);
}
/// This transaction may not happen, but it's OK, because on the next retry we will eventually create/update this node
zookeeper->createOrUpdate(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent);
recalculateColumnSizes();
return true;
}
void StorageReplicatedMergeTree::alter(
const AlterCommands & params, const Context & query_context, TableStructureWriteLockHolder & table_lock_holder)
{
assertNotReadonly();
LOG_DEBUG(log, "Doing ALTER");
auto table_id = getStorageID();
/// We cannot check this alter commands with method isModifyingData()
/// because ReplicatedMergeTree stores both columns and metadata for
/// each replica. So we have to wait AlterThread even with lightweight
/// metadata alter.
if (params.isSettingsAlter())
{
lockStructureExclusively(table_lock_holder, query_context.getCurrentQueryId());
/// We don't replicate storage_settings_ptr ALTER. It's local operation.
/// Also we don't upgrade alter lock to table structure lock.
LOG_DEBUG(log, "ALTER storage_settings_ptr only");
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
changeSettings(metadata.settings_ast, table_lock_holder);
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata);
return;
}
/// Alter is done by modifying the metadata nodes in ZK that are shared between all replicas
/// (/columns, /metadata). We set contents of the shared nodes to the new values and wait while
/// replicas asynchronously apply changes (see ReplicatedMergeTreeAlterThread.cpp) and modify
/// their respective replica metadata nodes (/replicas/<replica>/columns, /replicas/<replica>/metadata).
struct ChangedNode
{
ChangedNode(const String & table_path_, String name_, String new_value_)
: table_path(table_path_), name(std::move(name_)), shared_path(table_path + "/" + name)
, new_value(std::move(new_value_))
{}
const String & table_path;
String name;
String shared_path;
String getReplicaPath(const String & replica) const
{
return table_path + "/replicas/" + replica + "/" + name;
}
String new_value;
int32_t new_version = -1; /// Initialization is to suppress (useless) false positive warning found by cppcheck.
};
auto ast_to_str = [](ASTPtr query) -> String
{
auto ast_to_str = [](ASTPtr query) -> String {
if (!query)
return "";
return queryToString(query);
};
/// /columns and /metadata nodes
std::vector<ChangedNode> changed_nodes;
auto zookeeper = getZooKeeper();
std::optional<ReplicatedMergeTreeLogEntryData> alter_entry;
std::optional<String> mutation_znode;
while (true)
{
/// Clear nodes from previous iteration
alter_entry.emplace();
mutation_znode.reset();
/// We can safely read structure, because we guarded with alter_intention_lock
if (is_readonly)
throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY);
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
String new_columns_str = metadata.columns.toString();
if (new_columns_str != getColumns().toString())
changed_nodes.emplace_back(zookeeper_path, "columns", new_columns_str);
StorageInMemoryMetadata current_metadata = getInMemoryMetadata();
ReplicatedMergeTreeTableMetadata new_metadata(*this);
if (ast_to_str(metadata.order_by_ast) != ast_to_str(order_by_ast))
new_metadata.sorting_key = serializeAST(*extractKeyExpressionList(metadata.order_by_ast));
StorageInMemoryMetadata future_metadata = current_metadata;
params.apply(future_metadata);
if (ast_to_str(metadata.ttl_for_table_ast) != ast_to_str(ttl_table_ast))
new_metadata.ttl_table = serializeAST(*metadata.ttl_for_table_ast);
ReplicatedMergeTreeTableMetadata future_metadata_in_zk(*this);
if (ast_to_str(future_metadata.order_by_ast) != ast_to_str(current_metadata.order_by_ast))
future_metadata_in_zk.sorting_key = serializeAST(*extractKeyExpressionList(future_metadata.order_by_ast));
String new_indices_str = metadata.indices.toString();
if (new_indices_str != getIndices().toString())
new_metadata.skip_indices = new_indices_str;
if (ast_to_str(future_metadata.ttl_for_table_ast) != ast_to_str(current_metadata.ttl_for_table_ast))
future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.ttl_for_table_ast);
String new_constraints_str = metadata.constraints.toString();
if (new_constraints_str != getConstraints().toString())
new_metadata.constraints = new_constraints_str;
String new_indices_str = future_metadata.indices.toString();
if (new_indices_str != current_metadata.indices.toString())
future_metadata_in_zk.skip_indices = new_indices_str;
String new_metadata_str = new_metadata.toString();
if (new_metadata_str != ReplicatedMergeTreeTableMetadata(*this).toString())
changed_nodes.emplace_back(zookeeper_path, "metadata", new_metadata_str);
String new_constraints_str = future_metadata.constraints.toString();
if (new_constraints_str != current_metadata.constraints.toString())
future_metadata_in_zk.constraints = new_constraints_str;
/// Perform settings update locally
Coordination::Requests ops;
String new_metadata_str = future_metadata_in_zk.toString();
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/metadata", new_metadata_str, metadata_version));
String new_columns_str = future_metadata.columns.toString();
ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/columns", new_columns_str, -1));
if (ast_to_str(current_metadata.settings_ast) != ast_to_str(future_metadata.settings_ast))
{
lockStructureExclusively(table_lock_holder, query_context.getCurrentQueryId());
auto old_metadata = getInMemoryMetadata();
old_metadata.settings_ast = metadata.settings_ast;
changeSettings(metadata.settings_ast, table_lock_holder);
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, old_metadata);
/// Just change settings
current_metadata.settings_ast = future_metadata.settings_ast;
changeSettings(current_metadata.settings_ast, table_lock_holder);
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, current_metadata);
}
/// Modify shared metadata nodes in ZooKeeper.
Coordination::Requests ops;
for (const auto & node : changed_nodes)
ops.emplace_back(zkutil::makeSetRequest(node.shared_path, node.new_value, -1));
/// We can be sure, that in case of successfull commit in zookeeper our
/// version will increments by 1. Because we update with version check.
int new_metadata_version = metadata_version + 1;
Coordination::Responses results = getZooKeeper()->multi(ops);
alter_entry->type = LogEntry::ALTER_METADATA;
alter_entry->source_replica = replica_name;
alter_entry->metadata_str = new_metadata_str;
alter_entry->columns_str = new_columns_str;
alter_entry->alter_version = new_metadata_version;
alter_entry->create_time = time(nullptr);
for (size_t i = 0; i < changed_nodes.size(); ++i)
changed_nodes[i].new_version = dynamic_cast<const Coordination::SetResponse &>(*results[i]).stat.version;
}
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
auto maybe_mutation_commands = params.getMutationCommands(current_metadata);
alter_entry->have_mutation = !maybe_mutation_commands.empty();
table_lock_holder.release();
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", alter_entry->toString(), zkutil::CreateMode::PersistentSequential));
/// Wait until all replicas will apply ALTER.
std::optional<EphemeralLocksInAllPartitions> lock_holder;
for (const auto & node : changed_nodes)
{
Coordination::Stat stat;
/// Subscribe to change of shared ZK metadata nodes, to finish waiting if someone will do another ALTER.
if (!getZooKeeper()->exists(node.shared_path, &stat, alter_query_event))
throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
if (stat.version != node.new_version)
/// No we will prepare mutations record
/// This code pretty same with mutate() function but process results slightly differently
if (alter_entry->have_mutation)
{
LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; " +
"overlapping ALTER-s are fine but use caution with nontransitive changes");
return;
String mutations_path = zookeeper_path + "/mutations";
ReplicatedMergeTreeMutationEntry mutation_entry;
mutation_entry.source_replica = replica_name;
mutation_entry.commands = maybe_mutation_commands;
mutation_entry.alter_version = new_metadata_version;
Coordination::Stat mutations_stat;
zookeeper->get(mutations_path, &mutations_stat);
lock_holder.emplace(
zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper);
for (const auto & lock : lock_holder->getLocks())
mutation_entry.block_numbers[lock.partition_id] = lock.number;
mutation_entry.create_time = time(nullptr);
ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
ops.emplace_back(
zkutil::makeCreateRequest(mutations_path + "/", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential));
}
}
Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
Coordination::Responses results;
int32_t rc = zookeeper->tryMulti(ops, results);
std::set<String> inactive_replicas;
std::set<String> timed_out_replicas;
time_t replication_alter_columns_timeout = query_context.getSettingsRef().replication_alter_columns_timeout;
/// This code is quite similar with waitMutationToFinishOnReplicas
/// but contains more complicated details (versions manipulations, multiple nodes, etc.).
/// It will be removed soon in favor of alter-modify implementation on top of mutations.
/// TODO (alesap)
for (const String & replica : replicas)
{
LOG_DEBUG(log, "Waiting for " << replica << " to apply changes");
while (!partial_shutdown_called)
if (rc == Coordination::ZOK)
{
auto zookeeper = getZooKeeper();
/// Replica could be inactive.
if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
if (alter_entry->have_mutation)
{
LOG_WARNING(log, "Replica " << replica << " is not active during ALTER query."
" ALTER will be done asynchronously when replica becomes active.");
/// ALTER_METADATA record in replication /log
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results[2]).path_created;
alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1);
inactive_replicas.emplace(replica);
break;
}
struct ReplicaNode
{
explicit ReplicaNode(String path_) : path(std::move(path_)) {}
String path;
String value;
int32_t version = -1;
};
std::vector<ReplicaNode> replica_nodes;
for (const auto & node : changed_nodes)
replica_nodes.emplace_back(node.getReplicaPath(replica));
bool replica_was_removed = false;
for (auto & node : replica_nodes)
{
Coordination::Stat stat;
/// Replica could has been removed.
if (!zookeeper->tryGet(node.path, node.value, &stat))
{
LOG_WARNING(log, replica << " was removed");
replica_was_removed = true;
break;
}
node.version = stat.version;
}
if (replica_was_removed)
break;
bool alter_was_applied = true;
for (size_t i = 0; i < replica_nodes.size(); ++i)
{
if (replica_nodes[i].value != changed_nodes[i].new_value)
{
alter_was_applied = false;
break;
}
}
/// The ALTER has been successfully applied.
if (alter_was_applied)
break;
for (const auto & node : changed_nodes)
{
Coordination::Stat stat;
if (!zookeeper->exists(node.shared_path, &stat))
throw Exception(node.shared_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
if (stat.version != node.new_version)
{
LOG_WARNING(log, node.shared_path + " changed before this ALTER finished; "
"overlapping ALTER-s are fine but use caution with nontransitive changes");
return;
}
}
bool replica_nodes_changed_concurrently = false;
for (const auto & replica_node : replica_nodes)
{
Coordination::Stat stat;
if (!zookeeper->exists(replica_node.path, &stat, alter_query_event))
{
LOG_WARNING(log, replica << " was removed");
replica_was_removed = true;
break;
}
if (stat.version != replica_node.version)
{
replica_nodes_changed_concurrently = true;
break;
}
}
if (replica_was_removed)
break;
if (replica_nodes_changed_concurrently)
continue;
/// alter_query_event subscribed with zookeeper watch callback to /repliacs/{replica}/metadata
/// and /replicas/{replica}/columns nodes for current relica + shared nodes /columns and /metadata,
/// which is common for all replicas. If changes happen with this nodes (delete, set and create)
/// than event will be notified and wait will be interrupted.
///
/// ReplicatedMergeTreeAlterThread responsible for local /replicas/{replica}/metadata and
/// /replicas/{replica}/columns changes. Shared /columns and /metadata nodes can be changed by *newer*
/// concurrent alter from other replica. First of all it will update shared nodes and we will have no
/// ability to identify, that our *current* alter finshed. So we cannot do anything better than just
/// return from *current* alter with success result.
if (!replication_alter_columns_timeout)
{
alter_query_event->wait();
/// Everything is fine.
}
else if (alter_query_event->tryWait(replication_alter_columns_timeout * 1000))
{
/// Everything is fine.
/// ReplicatedMergeTreeMutationEntry record in /mutations
String mutation_path = dynamic_cast<const Coordination::CreateResponse &>(*results.back()).path_created;
mutation_znode = mutation_path.substr(mutation_path.find_last_of('/') + 1);
}
else
{
LOG_WARNING(log, "Timeout when waiting for replica " << replica << " to apply ALTER."
" ALTER will be done asynchronously.");
timed_out_replicas.emplace(replica);
break;
/// ALTER_METADATA record in replication /log
String alter_path = dynamic_cast<const Coordination::CreateResponse &>(*results.back()).path_created;
alter_entry->znode_name = alter_path.substr(alter_path.find_last_of('/') + 1);
}
break;
}
if (partial_shutdown_called)
throw Exception("Alter is not finished because table shutdown was called. Alter will be done after table restart.",
ErrorCodes::UNFINISHED);
if (!inactive_replicas.empty() || !timed_out_replicas.empty())
else if (rc == Coordination::ZBADVERSION)
{
std::stringstream exception_message;
exception_message << "Alter is not finished because";
if (dynamic_cast<const Coordination::SetResponse &>(*results[0]).error)
throw Exception("Metadata on replica is not up to date with common metadata in Zookeeper. Cannot alter", ErrorCodes::CANNOT_ASSIGN_ALTER);
if (!inactive_replicas.empty())
{
exception_message << " some replicas are inactive right now";
for (auto it = inactive_replicas.begin(); it != inactive_replicas.end(); ++it)
exception_message << (it == inactive_replicas.begin() ? ": " : ", ") << *it;
}
if (!timed_out_replicas.empty() && !inactive_replicas.empty())
exception_message << " and";
if (!timed_out_replicas.empty())
{
exception_message << " timeout when waiting for some replicas";
for (auto it = timed_out_replicas.begin(); it != timed_out_replicas.end(); ++it)
exception_message << (it == timed_out_replicas.begin() ? ": " : ", ") << *it;
exception_message << " (replication_alter_columns_timeout = " << replication_alter_columns_timeout << ")";
}
exception_message << ". Alter will be done asynchronously.";
throw Exception(exception_message.str(), ErrorCodes::UNFINISHED);
continue;
}
else
{
throw Coordination::Exception("Alter cannot be assigned because of Zookeeper error", rc);
}
}
LOG_DEBUG(log, "ALTER finished");
table_lock_holder.release();
std::vector<String> unwaited;
if (query_context.getSettingsRef().replication_alter_partitions_sync == 2)
{
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
unwaited = waitForAllReplicasToProcessLogEntry(*alter_entry, false);
}
else if (query_context.getSettingsRef().replication_alter_partitions_sync == 1)
{
LOG_DEBUG(log, "Updated shared metadata nodes in ZooKeeper. Waiting for replicas to apply changes.");
waitForReplicaToProcessLogEntry(replica_name, *alter_entry);
}
if (!unwaited.empty())
throw Exception("Some replicas doesn't finish metadata alter", ErrorCodes::UNFINISHED);
if (mutation_znode)
{
LOG_DEBUG(log, "Metadata changes applied. Will wait for data changes.");
waitMutation(*mutation_znode, query_context.getSettingsRef().replication_alter_partitions_sync);
LOG_DEBUG(log, "Data changes applied.");
}
}
void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context)
@ -3886,21 +3792,27 @@ StorageReplicatedMergeTree::allocateBlockNumber(
}
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
Strings StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active)
{
LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);
auto zookeeper = getZooKeeper();
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
Strings unwaited;
for (const String & replica : replicas)
{
if (wait_for_non_active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
{
waitForReplicaToProcessLogEntry(replica, entry);
}
else
{
unwaited.push_back(replica);
}
}
LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
return unwaited;
}
@ -4052,7 +3964,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
res.zookeeper_path = zookeeper_path;
res.replica_name = replica_name;
res.replica_path = replica_path;
res.columns_version = columns_version;
res.columns_version = -1;
res.log_max_index = 0;
res.log_pointer = 0;
@ -4560,18 +4472,23 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const
throw Coordination::Exception("Unable to create a mutation znode", rc);
}
waitMutation(entry.znode_name, query_context.getSettingsRef().mutations_sync);
}
void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) const
{
auto zookeeper = getZooKeeper();
/// we have to wait
if (query_context.getSettingsRef().mutations_sync != 0)
if (mutations_sync != 0)
{
Strings replicas;
if (query_context.getSettingsRef().mutations_sync == 2) /// wait for all replicas
replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
else if (query_context.getSettingsRef().mutations_sync == 1) /// just wait for ourself
replicas.push_back(replica_path);
if (mutations_sync == 2) /// wait for all replicas
replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
else if (mutations_sync == 1) /// just wait for ourself
replicas.push_back(replica_name);
waitMutationToFinishOnReplicas(replicas, entry.znode_name);
waitMutationToFinishOnReplicas(replicas, znode_name);
}
}
std::vector<MergeTreeMutationStatus> StorageReplicatedMergeTree::getMutationsStatus() const
@ -5000,7 +4917,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_
entry_replace.new_part_names.emplace_back(part->name);
for (const String & checksum : part_checksums)
entry_replace.part_names_checksums.emplace_back(checksum);
entry_replace.columns_version = columns_version;
entry_replace.columns_version = -1;
}
/// We are almost ready to commit changes, remove fetches and merges from drop range
@ -5191,7 +5108,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
entry_replace.new_part_names.emplace_back(part->name);
for (const String & checksum : part_checksums)
entry_replace.part_names_checksums.emplace_back(checksum);
entry_replace.columns_version = columns_version;
entry_replace.columns_version = -1;
}
queue.removePartProducingOpsInRange(zookeeper, drop_range, entry);
@ -5296,11 +5213,6 @@ void StorageReplicatedMergeTree::getCommitPartOps(
}
/// Information about the part, in the replica
ops.emplace_back(zkutil::makeCheckRequest(
zookeeper_path + "/columns",
columns_version));
if (storage_settings_ptr->use_minimalistic_part_header_in_zookeeper)
{
ops.emplace_back(zkutil::makeCreateRequest(
@ -5517,4 +5429,32 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const
}
StorageInMemoryMetadata
StorageReplicatedMergeTree::getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const
{
auto replicated_metadata = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
StorageInMemoryMetadata result = getInMemoryMetadata();
result.columns = ColumnsDescription::parse(columns_str);
result.constraints = ConstraintsDescription::parse(replicated_metadata.constraints);
result.indices = IndicesDescription::parse(replicated_metadata.skip_indices);
ParserExpression expression_p;
/// The only thing, that can be changed is ttl expression
if (replicated_metadata.primary_key.empty())
throw Exception("Primary key cannot be empty" , ErrorCodes::LOGICAL_ERROR);
if (!replicated_metadata.sorting_key.empty())
{
result.order_by_ast = parseQuery(expression_p, "(" + replicated_metadata.sorting_key + ")", 0);
result.primary_key_ast = parseQuery(expression_p, "(" + replicated_metadata.primary_key + ")", 0);
}
else
{
result.order_by_ast = parseQuery(expression_p, "(" + replicated_metadata.primary_key + ")", 0);
}
return result;
}
}

View File

@ -14,7 +14,6 @@
#include <Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAlterThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
#include <Storages/MergeTree/EphemeralLockInZooKeeper.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
@ -36,7 +35,7 @@ namespace DB
/** The engine that uses the merge tree (see MergeTreeData) and replicated through ZooKeeper.
*
* ZooKeeper is used for the following things:
* - the structure of the table (/ metadata, /columns)
* - the structure of the table (/metadata, /columns)
* - action log with data (/log/log-...,/replicas/replica_name/queue/queue-...);
* - a replica list (/replicas), and replica activity tag (/replicas/replica_name/is_active), replica addresses (/replicas/replica_name/host);
* - select the leader replica (/leader_election) - this is the replica that assigns the merge;
@ -107,6 +106,7 @@ public:
void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & query_context) override;
void mutate(const MutationCommands & commands, const Context & context) override;
void waitMutation(const String & znode_name, size_t mutation_sync) const;
std::vector<MergeTreeMutationStatus> getMutationsStatus() const override;
CancellationCode killMutation(const String & mutation_id) override;
@ -177,6 +177,8 @@ public:
/// Checks ability to use granularity
bool canUseAdaptiveGranularity() const override;
int getMetadataVersion() const { return metadata_version; }
private:
/// Get a sequential consistent view of current parts.
@ -216,17 +218,6 @@ private:
*/
zkutil::EphemeralNodeHolderPtr replica_is_active_node;
/** Version of the /columns node in ZooKeeper corresponding to the current data.columns.
* Read and modify along with the data.columns - under TableStructureLock.
*/
int columns_version = -1;
/// Version of the /metadata node in ZooKeeper.
int metadata_version = -1;
/// Used to delay setting table structure till startup() in case of an offline ALTER.
std::function<void()> set_table_structure_at_startup;
/** Is this replica "leading". The leader replica selects the parts to merge.
*/
std::atomic<bool> is_leader {false};
@ -260,6 +251,7 @@ private:
/// Limiting parallel fetches per one table
std::atomic_uint current_table_fetches {0};
int metadata_version = 0;
/// Threads.
/// A task that keeps track of the updates in the logs of all replicas and loads them into the queue.
@ -286,9 +278,6 @@ private:
/// A thread that removes old parts, log entries, and blocks.
ReplicatedMergeTreeCleanupThread cleanup_thread;
/// A thread monitoring changes to the column list in ZooKeeper and updating the parts in accordance with these changes.
ReplicatedMergeTreeAlterThread alter_thread;
/// A thread that checks the data of the parts, as well as the queue of the parts to be checked.
ReplicatedMergeTreePartCheckThread part_check_thread;
@ -313,11 +302,7 @@ private:
*/
void createNewZooKeeperNodes();
/** Verify that the list of columns and table settings match those specified in ZK (/metadata).
* If not, throw an exception.
* Must be called before startup().
*/
void checkTableStructure(bool skip_sanity_checks, bool allow_alter);
void checkTableStructure(const String & zookeeper_prefix);
/// A part of ALTER: apply metadata changes only (data parts are altered separately).
/// Must be called under IStorage::lockStructureForAlter() lock.
@ -381,9 +366,20 @@ private:
/// Do the merge or recommend to make the fetch instead of the merge
bool tryExecuteMerge(const LogEntry & entry);
/// Execute alter of table metadata. Set replica/metdata and replica/columns
/// nodes in zookeeper and also changes in memory metadata.
/// New metadata and columns values stored in entry.
bool executeMetadataAlter(const LogEntry & entry);
/// Execute MUTATE_PART entry. Part name and mutation commands
/// stored in entry. This function relies on MergerMutator class.
bool tryExecutePartMutation(const LogEntry & entry);
/// Fetch part from other replica (inserted or merged/mutated)
/// NOTE: Attention! First of all tries to find covering part on other replica
/// and set it into entry.actual_new_part_name. After that tries to fetch this new covering part.
/// If fetch was not successful, clears entry.actual_new_part_name.
bool executeFetch(LogEntry & entry);
void executeClearColumnOrIndexInPartition(const LogEntry & entry);
@ -443,7 +439,7 @@ private:
bool force_ttl,
ReplicatedMergeTreeLogEntryData * out_log_entry = nullptr);
bool createLogEntryToMutatePart(const IMergeTreeDataPart & part, Int64 mutation_version);
bool createLogEntryToMutatePart(const IMergeTreeDataPart & part, Int64 mutation_version, int alter_version);
/// Exchange parts.
@ -486,7 +482,7 @@ private:
* Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock.
* TODO: There are wrong usages of this method that are not fixed yet.
*/
void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
Strings waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true);
/** Wait until the specified replica executes the specified action from the log.
* NOTE: See comment about locks above.
@ -537,6 +533,8 @@ private:
void waitMutationToFinishOnReplicas(
const Strings & replicas, const String & mutation_id) const;
StorageInMemoryMetadata getMetadataFromSharedZookeeper(const String & metadata_str, const String & columns_str) const;
protected:
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
*/

View File

@ -1,5 +1,8 @@
DROP TABLE IF EXISTS test.replicated_alter1;
DROP TABLE IF EXISTS test.replicated_alter2;
SET replication_alter_partitions_sync = 2;
CREATE TABLE test.replicated_alter1 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r1', d, k, 8192);
CREATE TABLE test.replicated_alter2 (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/clickhouse/tables/test/alter', 'r2', d, k, 8192);

View File

@ -1,9 +1,9 @@
SET send_logs_level = 'none';
DROP TABLE IF EXISTS test.old_style;
CREATE TABLE test.old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
DROP TABLE test.old_style;
DROP TABLE IF EXISTS old_style;
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
DROP TABLE old_style;
DROP TABLE IF EXISTS test.summing_r1;
DROP TABLE IF EXISTS test.summing_r2;
@ -40,8 +40,11 @@ SELECT '*** Check SHOW CREATE TABLE ***';
SHOW CREATE TABLE test.summing_r2;
DETACH TABLE test.summing_r2;
ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t); -- { serverError 341 }
ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t) SETTINGS replication_alter_partitions_sync = 2; -- { serverError 341 }
ATTACH TABLE test.summing_r2;
SELECT sleep(1) Format Null;
SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***';
SHOW CREATE TABLE test.summing_r2;

View File

@ -3,6 +3,7 @@ DROP TABLE IF EXISTS test.minmax_idx_r;
DROP TABLE IF EXISTS test.minmax_idx2;
DROP TABLE IF EXISTS test.minmax_idx2_r;
SET replication_alter_partitions_sync = 2;
CREATE TABLE test.minmax_idx
(
@ -109,4 +110,4 @@ SELECT * FROM test.minmax_idx2_r WHERE u64 * i32 >= 2 ORDER BY (u64, i32);
DROP TABLE test.minmax_idx;
DROP TABLE test.minmax_idx_r;
DROP TABLE test.minmax_idx2;
DROP TABLE test.minmax_idx2_r;
DROP TABLE test.minmax_idx2_r;

View File

@ -1,6 +1,8 @@
DROP TABLE IF EXISTS replicated_table_for_alter1;
DROP TABLE IF EXISTS replicated_table_for_alter2;
SET replication_alter_partitions_sync = 2;
CREATE TABLE replicated_table_for_alter1 (
id UInt64,
Data String

View File

@ -0,0 +1,10 @@
14850
14850
59700
59700
59700
0
6000
6000
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500

View File

@ -0,0 +1,71 @@
DROP TABLE IF EXISTS test_alter_on_mutation;
CREATE TABLE test_alter_on_mutation
(
date Date,
key UInt64,
value String
)
ENGINE ReplicatedMergeTree('/clickhouse/tables/test_alter_on_mutation', '1')
ORDER BY key PARTITION BY date;
INSERT INTO test_alter_on_mutation select toDate('2020-01-05'), number, toString(number) from system.numbers limit 100;
INSERT INTO test_alter_on_mutation select toDate('2020-01-06'), number, toString(number) from system.numbers limit 100;
INSERT INTO test_alter_on_mutation select toDate('2020-01-07'), number, toString(number) from system.numbers limit 100;
SELECT sum(cast(value as UInt64)) from test_alter_on_mutation;
ALTER TABLE test_alter_on_mutation MODIFY COLUMN value UInt64;
SELECT sum(value) from test_alter_on_mutation;
INSERT INTO test_alter_on_mutation select toDate('2020-01-05'), number, toString(number) from system.numbers limit 100, 100;
INSERT INTO test_alter_on_mutation select toDate('2020-01-06'), number, toString(number) from system.numbers limit 100, 100;
INSERT INTO test_alter_on_mutation select toDate('2020-01-07'), number, toString(number) from system.numbers limit 100, 100;
OPTIMIZE TABLE test_alter_on_mutation FINAL;
SELECT sum(value) from test_alter_on_mutation;
ALTER TABLE test_alter_on_mutation MODIFY COLUMN value String;
SELECT sum(cast(value as UInt64)) from test_alter_on_mutation;
OPTIMIZE TABLE test_alter_on_mutation FINAL;
SELECT sum(cast(value as UInt64)) from test_alter_on_mutation;
ALTER TABLE test_alter_on_mutation ADD COLUMN value1 Float64;
SELECT sum(value1) from test_alter_on_mutation;
ALTER TABLE test_alter_on_mutation DROP COLUMN value;
SELECT sum(value) from test_alter_on_mutation; -- {serverError 47}
ALTER TABLE test_alter_on_mutation ADD COLUMN value String DEFAULT '10';
SELECT sum(cast(value as UInt64)) from test_alter_on_mutation;
-- TODO(alesap)
OPTIMIZE table test_alter_on_mutation FINAL;
ALTER TABLE test_alter_on_mutation MODIFY COLUMN value UInt64 DEFAULT 10;
SELECT sum(value) from test_alter_on_mutation;
DROP TABLE IF EXISTS test_alter_on_mutation;
DROP TABLE IF EXISTS nested_alter;
CREATE TABLE nested_alter (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT '0') ENGINE = ReplicatedMergeTree('/clickhouse/tables/nested_alter', 'r2', d, k, 8192);
INSERT INTO nested_alter VALUES ('2015-01-01', 6,38,'2014-07-15 13:26:50',[10,20,30],['asd','qwe','qwe'],['2000-01-01','2000-01-01','2000-01-03'],'100500');
SELECT * FROM nested_alter;
ALTER TABLE nested_alter DROP COLUMN `n.d`;
SELECT * FROM nested_alter;
DROP TABLE nested_alter;

View File

@ -0,0 +1,8 @@
Wrong column name.
CREATE TABLE default.table_for_bad_alters (`key` UInt64, `value1` UInt8, `value2` String) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/table_for_bad_alters\', \'1\') ORDER BY key SETTINGS index_granularity = 8192
CREATE TABLE default.table_for_bad_alters (`key` UInt64, `value1` UInt8, `value2` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/table_for_bad_alters\', \'1\') ORDER BY key SETTINGS index_granularity = 8192
syntax error at begin of string.
7
Hello
World
Wrong index name.

View File

@ -0,0 +1,41 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_bad_alters";
$CLICKHOUSE_CLIENT -n --query "CREATE TABLE table_for_bad_alters (
key UInt64,
value1 UInt8,
value2 String
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/table_for_bad_alters', '1')
ORDER BY key;"
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_bad_alters VALUES(1, 1, 'Hello');"
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_bad_alters MODIFY COLUMN value1 UInt32, DROP COLUMN non_existing_column" 2>&1 | grep -o "Wrong column name." | uniq
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_bad_alters;" # nothing changed
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_bad_alters MODIFY COLUMN value2 UInt32 SETTINGS replication_alter_partitions_sync=0;"
sleep 2
while [[ $($CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='0000000000'" 2>&1) ]]; do
sleep 1
done
$CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE table_for_bad_alters;" # Type changed, but we can revert back
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_bad_alters VALUES(2, 2, 7)"
$CLICKHOUSE_CLIENT --query "SELECT distinct(value2) FROM table_for_bad_alters" 2>&1 | grep -o 'syntax error at begin of string.' | uniq
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_bad_alters MODIFY COLUMN value2 String SETTINGS replication_alter_partitions_sync=2"
$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_bad_alters VALUES(3, 3, 'World')"
$CLICKHOUSE_CLIENT --query "SELECT value2 FROM table_for_bad_alters ORDER BY value2"
$CLICKHOUSE_CLIENT --query "ALTER TABLE table_for_bad_alters DROP INDEX idx2" 2>&1 | grep -o 'Wrong index name.' | uniq
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_bad_alters"

View File

@ -0,0 +1,9 @@
Starting alters
Finishing alters
Equal number of columns
0
0
0
0
0
0

View File

@ -0,0 +1,111 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
REPLICAS=3
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_add_drop_$i"
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_add_drop_$i (key UInt64, value0 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_alter_add_drop_column', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_1 SELECT number, number + 10 from numbers(100000)"
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_$i"
done
function alter_thread()
{
TYPES=(Float64 String UInt8 UInt32)
while true; do
REPLICA=$(($RANDOM % 3 + 1))
ADD=$(($RANDOM % 5 + 1))
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency
DROP=$(($RANDOM % 5 + 1))
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency
sleep 0.$RANDOM
done
}
function optimize_thread()
{
while true; do
REPLICA=$(($RANDOM % 3 + 1))
$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_alter_add_drop_$REPLICA FINAL SETTINGS replication_alter_partitions_sync=0";
sleep 0.$RANDOM
done
}
function insert_thread()
{
while true; do
REPLICA=$(($RANDOM % 3 + 1))
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_$REPLICA VALUES($RANDOM, 7)"
sleep 0.$RANDOM
done
}
echo "Starting alters"
export -f alter_thread;
export -f optimize_thread;
export -f insert_thread;
TIMEOUT=30
# Sometimes we detach and attach tables
timeout $TIMEOUT bash -c alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c optimize_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
wait
echo "Finishing alters"
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null)
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null)
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null)
while [ "$columns1" != "$columns2" ] || [ "$columns2" != "$columns3" ]; do
columns1=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_1'" 2> /dev/null)
columns2=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_2'" 2> /dev/null)
columns3=$($CLICKHOUSE_CLIENT --query "select count() from system.columns where table='concurrent_alter_add_drop_3'" 2> /dev/null)
sleep 1
done
echo "Equal number of columns"
# This alter will finish all previous, but replica 1 maybe still not up-to-date
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_1 MODIFY COLUMN value0 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
sleep 1
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_$i"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'"
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i'"
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i'"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_add_drop_$i"
done

View File

@ -0,0 +1,11 @@
1725
1725
1725
Starting alters
Finishing alters
1
0
1
0
1
0

View File

@ -0,0 +1,110 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
REPLICAS=3
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_detach_$i"
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_detach_$i (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_alter_detach', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10)"
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10, 40)"
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alter_detach_$i"
done
INITIAL_SUM=`$CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alter_detach_1"`
# This alters mostly requires not only metadata change
# but also conversion of data. Also they are all compatible
# between each other, so can be executed concurrently.
function correct_alter_thread()
{
TYPES=(Float64 String UInt8 UInt32)
while true; do
REPLICA=$(($RANDOM % 3 + 1))
TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]}
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency
sleep 0.$RANDOM
done
}
# This thread add some data to table. After we finish we can check, that
# all our data have same types.
# insert queries will fail sometime because of wrong types.
function insert_thread()
{
VALUES=(7.0 7 '7')
while true; do
REPLICA=$(($RANDOM % 3 + 1))
VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]}
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)"
sleep 0.$RANDOM
done
}
function detach_attach_thread()
{
while true; do
REPLICA=$(($RANDOM % 3 + 1))
$CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_detach_$REPLICA"
sleep 0.$RANDOM
$CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_detach_$REPLICA"
done
}
echo "Starting alters"
export -f correct_alter_thread;
export -f insert_thread;
export -f detach_attach_thread;
TIMEOUT=15
# Sometimes we detach and attach tables
timeout $TIMEOUT bash -c detach_attach_thread 2> /dev/null &
timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
wait
echo "Finishing alters"
sleep 1
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_detach_$i" 2> /dev/null
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_detach_$i"
done
# This alter will finish all previous, but replica 1 maybe still not up-to-date
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
sleep 1
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_detach_$i"
$CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_detach_$i"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_detach_$i'" # all mutations have to be done
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_detach_$i'" # all mutations have to be done
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_detach_$i"
done

View File

@ -0,0 +1,12 @@
1725
1725
1725
1725
1725
Starting alters
Finishing alters
1
1
1
1
1

View File

@ -0,0 +1,17 @@
1725
1725
1725
1725
1725
Starting alters
Finishing alters
1
0
1
0
1
0
1
0
1
0

View File

@ -0,0 +1,115 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
REPLICAS=5
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_mt_$i"
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_mt_$i (key UInt64, value1 UInt64, value2 Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/concurrent_alter_mt', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000"
done
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_1 SELECT number, number + 10, number from numbers(10)"
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_1 SELECT number, number + 10, number from numbers(10, 40)"
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_mt_$i"
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alter_mt_$i"
done
INITIAL_SUM=`$CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alter_mt_1"`
# This alters mostly requires not only metadata change
# but also conversion of data. Also they are all compatible
# between each other, so can be executed concurrently.
function correct_alter_thread()
{
TYPES=(Float64 String UInt8 UInt32)
while true; do
REPLICA=$(($RANDOM % 5 + 1))
TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]}
$CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency
sleep 0.$RANDOM
done
}
# This thread add some data to table. After we finish we can check, that
# all our data have same types.
# insert queries will fail sometime because of wrong types.
function insert_thread()
{
VALUES=(7.0 7 '7')
while true; do
REPLICA=$(($RANDOM % 5 + 1))
VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]}
$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)"
sleep 0.$RANDOM
done
}
# Some select load, to be sure, that our selects work in concurrent execution with alters
function select_thread()
{
while true; do
REPLICA=$(($RANDOM % 5 + 1))
$CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) FROM concurrent_alter_mt_$REPLICA" 1>/dev/null
sleep 0.$RANDOM
done
}
echo "Starting alters"
export -f correct_alter_thread;
export -f insert_thread;
export -f select_thread;
TIMEOUT=30
# Selects should run successfully
timeout $TIMEOUT bash -c select_thread &
timeout $TIMEOUT bash -c select_thread &
timeout $TIMEOUT bash -c select_thread &
timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
timeout $TIMEOUT bash -c insert_thread 2> /dev/null &
wait
echo "Finishing alters"
# This alter will finish all previous, but replica 1 maybe still not up-to-date
while [[ $($CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_1 MODIFY COLUMN value1 String SETTINGS replication_alter_partitions_sync=2" 2>&1) ]]; do
sleep 1
done
for i in `seq $REPLICAS`; do
$CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_mt_$i"
$CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) > $INITIAL_SUM FROM concurrent_alter_mt_$i"
$CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_mt_$i'" # all mutations have to be done
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done=0 and table = 'concurrent_alter_mt_$i'"
$CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_mt_$i'"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_alter_mt_$i"
done