mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge pull request #50181 from ClickHouse/repro_49887
Make exception about `ALTER TABLE ... DROP COLUMN|INDEX|PROJECTION` more clear
This commit is contained in:
commit
0d68a52c06
@ -573,4 +573,36 @@ void replaceColumns(QueryTreeNodePtr & node,
|
||||
visitor.visit(node);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class CollectIdentifiersFullNamesVisitor : public ConstInDepthQueryTreeVisitor<CollectIdentifiersFullNamesVisitor>
|
||||
{
|
||||
public:
|
||||
explicit CollectIdentifiersFullNamesVisitor(NameSet & used_identifiers_)
|
||||
: used_identifiers(used_identifiers_) { }
|
||||
|
||||
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &) { return true; }
|
||||
|
||||
void visitImpl(const QueryTreeNodePtr & node)
|
||||
{
|
||||
auto * column_node = node->as<IdentifierNode>();
|
||||
if (!column_node)
|
||||
return;
|
||||
|
||||
used_identifiers.insert(column_node->getIdentifier().getFullName());
|
||||
}
|
||||
|
||||
NameSet & used_identifiers;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node)
|
||||
{
|
||||
NameSet out;
|
||||
CollectIdentifiersFullNamesVisitor visitor(out);
|
||||
visitor.visit(node);
|
||||
return out;
|
||||
}
|
||||
}
|
||||
|
@ -83,4 +83,8 @@ void replaceColumns(QueryTreeNodePtr & node,
|
||||
const QueryTreeNodePtr & table_expression_node,
|
||||
const std::unordered_map<std::string, QueryTreeNodePtr> & column_name_to_node);
|
||||
|
||||
|
||||
/// Just collect all identifiers from query tree
|
||||
NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node);
|
||||
|
||||
}
|
||||
|
@ -845,6 +845,12 @@ bool AlterCommand::isRemovingProperty() const
|
||||
return to_remove != RemoveProperty::NO_PROPERTY;
|
||||
}
|
||||
|
||||
bool AlterCommand::isDropSomething() const
|
||||
{
|
||||
return type == Type::DROP_COLUMN || type == Type::DROP_INDEX
|
||||
|| type == Type::DROP_CONSTRAINT || type == Type::DROP_PROJECTION;
|
||||
}
|
||||
|
||||
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
{
|
||||
if (!isRequireMutationStage(metadata))
|
||||
|
@ -167,6 +167,8 @@ struct AlterCommand
|
||||
/// Command removing some property from column or table
|
||||
bool isRemovingProperty() const;
|
||||
|
||||
bool isDropSomething() const;
|
||||
|
||||
/// If possible, convert alter command to mutation command. In other case
|
||||
/// return empty optional. Some storages may execute mutations after
|
||||
/// metadata changes.
|
||||
|
@ -1,25 +1,27 @@
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/Utils.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/BackupEntryWrappedWith.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/Increment.h>
|
||||
#include <Common/noexcept_scope.h>
|
||||
#include <Common/ProfileEventsScope.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/Config/ConfigHelper.h>
|
||||
#include <Common/noexcept_scope.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/MergeTree/RangesInDataPart.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
@ -27,31 +29,29 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/hasNullable.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <Disks/createVolume.h>
|
||||
#include <DataTypes/hasNullable.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
#include <Disks/createVolume.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Aggregator.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/MergeTreeTransaction.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TransactionLog.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
#include <Interpreters/inplaceBlockConversions.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTHelpers.h>
|
||||
@ -65,25 +65,24 @@
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/QueryPlan/QueryIdHolder.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/BlockNumberColumn.h>
|
||||
#include <Storages/Freeze.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWide.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/MergeTree/ActiveDataPartSet.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartBuilder.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/BlockNumberColumn.h>
|
||||
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
@ -2970,9 +2969,11 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
|
||||
NamesAndTypesList columns_to_check_conversion;
|
||||
|
||||
auto unfinished_mutations = getUnfinishedMutationCommands();
|
||||
std::optional<NameDependencies> name_deps{};
|
||||
for (const AlterCommand & command : commands)
|
||||
{
|
||||
checkDropCommandDoesntAffectInProgressMutations(command, unfinished_mutations, local_context);
|
||||
/// Just validate partition expression
|
||||
if (command.partition)
|
||||
{
|
||||
@ -4326,7 +4327,7 @@ void MergeTreeData::delayMutationOrThrowIfNeeded(Poco::Event * until, const Cont
|
||||
if (!num_mutations_to_delay && !num_mutations_to_throw)
|
||||
return;
|
||||
|
||||
size_t num_unfinished_mutations = getNumberOfUnfinishedMutations();
|
||||
size_t num_unfinished_mutations = getUnfinishedMutationCommands().size();
|
||||
if (num_mutations_to_throw && num_unfinished_mutations >= num_mutations_to_throw)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::RejectedMutations);
|
||||
@ -7606,6 +7607,70 @@ bool MergeTreeData::canUsePolymorphicParts() const
|
||||
return canUsePolymorphicParts(*getSettings(), unused);
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::checkDropCommandDoesntAffectInProgressMutations(const AlterCommand & command, const std::map<std::string, MutationCommands> & unfinished_mutations, ContextPtr local_context) const
|
||||
{
|
||||
if (!command.isDropSomething() || unfinished_mutations.empty())
|
||||
return;
|
||||
|
||||
auto throw_exception = [] (
|
||||
const std::string & mutation_name,
|
||||
const std::string & entity_name,
|
||||
const std::string & identifier_name)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Cannot drop {} {} because it's affected by mutation with ID '{}' which is not finished yet. "
|
||||
"Wait this mutation, or KILL it with command "
|
||||
"\"KILL MUTATION WHERE mutation_id = '{}'\"",
|
||||
entity_name,
|
||||
backQuoteIfNeed(identifier_name),
|
||||
mutation_name,
|
||||
mutation_name);
|
||||
};
|
||||
|
||||
for (const auto & [mutation_name, commands] : unfinished_mutations)
|
||||
{
|
||||
for (const MutationCommand & mutation_command : commands)
|
||||
{
|
||||
if (command.type == AlterCommand::DROP_INDEX && mutation_command.index_name == command.index_name)
|
||||
{
|
||||
throw_exception(mutation_name, "index", command.index_name);
|
||||
}
|
||||
else if (command.type == AlterCommand::DROP_PROJECTION
|
||||
&& mutation_command.projection_name == command.projection_name)
|
||||
{
|
||||
throw_exception(mutation_name, "projection", command.projection_name);
|
||||
}
|
||||
else if (command.type == AlterCommand::DROP_COLUMN)
|
||||
{
|
||||
if (mutation_command.column_name == command.column_name)
|
||||
throw_exception(mutation_name, "column", command.column_name);
|
||||
|
||||
if (mutation_command.predicate)
|
||||
{
|
||||
auto query_tree = buildQueryTree(mutation_command.predicate, local_context);
|
||||
auto identifiers = collectIdentifiersFullNames(query_tree);
|
||||
|
||||
if (identifiers.contains(command.column_name))
|
||||
throw_exception(mutation_name, "column", command.column_name);
|
||||
}
|
||||
|
||||
for (const auto & [name, expr] : mutation_command.column_to_update_expression)
|
||||
{
|
||||
if (name == command.column_name)
|
||||
throw_exception(mutation_name, "column", command.column_name);
|
||||
|
||||
auto query_tree = buildQueryTree(expr, local_context);
|
||||
auto identifiers = collectIdentifiersFullNames(query_tree);
|
||||
if (identifiers.contains(command.column_name))
|
||||
throw_exception(mutation_name, "column", command.column_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const
|
||||
{
|
||||
if (!canUseAdaptiveGranularity())
|
||||
|
@ -47,6 +47,7 @@ namespace DB
|
||||
/// Number of streams is not number parts, but number or parts*files, hence 1000.
|
||||
const size_t DEFAULT_DELAYED_STREAMS_FOR_PARALLEL_WRITE = 1000;
|
||||
|
||||
struct AlterCommand;
|
||||
class AlterCommands;
|
||||
class InterpreterSelectQuery;
|
||||
class MergeTreePartsMover;
|
||||
@ -580,9 +581,6 @@ public:
|
||||
/// The decision to delay or throw is made according to settings 'number_of_mutations_to_delay' and 'number_of_mutations_to_throw'.
|
||||
void delayMutationOrThrowIfNeeded(Poco::Event * until, const ContextPtr & query_context) const;
|
||||
|
||||
/// Returns number of unfinished mutations (is_done = 0).
|
||||
virtual size_t getNumberOfUnfinishedMutations() const = 0;
|
||||
|
||||
/// Renames temporary part to a permanent part and adds it to the parts set.
|
||||
/// It is assumed that the part does not intersect with existing parts.
|
||||
/// Adds the part in the PreActive state (the part will be added to the active set later with out_transaction->commit()).
|
||||
@ -719,6 +717,13 @@ public:
|
||||
/// If something is wrong, throws an exception.
|
||||
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override;
|
||||
|
||||
/// Throw exception if command is some kind of DROP command (drop column, drop index, etc)
|
||||
/// and we have unfinished mutation which need this column to finish.
|
||||
void checkDropCommandDoesntAffectInProgressMutations(
|
||||
const AlterCommand & command, const std::map<std::string, MutationCommands> & unfinished_mutations, ContextPtr context) const;
|
||||
/// Return mapping unfinished mutation name -> Mutation command
|
||||
virtual std::map<std::string, MutationCommands> getUnfinishedMutationCommands() const = 0;
|
||||
|
||||
/// Checks if the Mutation can be performed.
|
||||
/// (currently no additional checks: always ok)
|
||||
void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include "Storages/MutationCommands.h"
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <base/sort.h>
|
||||
|
||||
@ -557,7 +558,7 @@ bool ReplicatedMergeTreeQueue::removeFailedQuorumPart(const MergeTreePartInfo &
|
||||
return virtual_parts.remove(part_info);
|
||||
}
|
||||
|
||||
int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason)
|
||||
std::pair<int32_t, int32_t> ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback, PullLogsReason reason)
|
||||
{
|
||||
std::lock_guard lock(pull_logs_to_queue_mutex);
|
||||
|
||||
@ -589,7 +590,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
|
||||
/// in the queue.
|
||||
/// With this we ensure that if you read the log state L1 and then the state of mutations M1,
|
||||
/// then L1 "happened-before" M1.
|
||||
updateMutations(zookeeper);
|
||||
int32_t mutations_version = updateMutations(zookeeper);
|
||||
|
||||
if (index_str.empty())
|
||||
{
|
||||
@ -718,7 +719,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
|
||||
storage.background_operations_assignee.trigger();
|
||||
}
|
||||
|
||||
return stat.version;
|
||||
return std::pair{stat.version, mutations_version};
|
||||
}
|
||||
|
||||
|
||||
@ -857,11 +858,12 @@ ActiveDataPartSet getPartNamesToMutate(
|
||||
|
||||
}
|
||||
|
||||
void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback)
|
||||
int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback)
|
||||
{
|
||||
std::lock_guard lock(update_mutations_mutex);
|
||||
|
||||
Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", nullptr, watch_callback);
|
||||
Coordination::Stat mutations_stat;
|
||||
Strings entries_in_zk = zookeeper->getChildrenWatch(fs::path(zookeeper_path) / "mutations", &mutations_stat, watch_callback);
|
||||
StringSet entries_in_zk_set(entries_in_zk.begin(), entries_in_zk.end());
|
||||
|
||||
/// Compare with the local state, delete obsolete entries and determine which new entries to load.
|
||||
@ -976,6 +978,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C
|
||||
if (some_mutations_are_probably_done)
|
||||
storage.mutations_finalizing_task->schedule();
|
||||
}
|
||||
return mutations_stat.version;
|
||||
}
|
||||
|
||||
|
||||
@ -1761,22 +1764,21 @@ size_t ReplicatedMergeTreeQueue::countFinishedMutations() const
|
||||
return count;
|
||||
}
|
||||
|
||||
size_t ReplicatedMergeTreeQueue::countUnfinishedMutations() const
|
||||
std::map<std::string, MutationCommands> ReplicatedMergeTreeQueue::getUnfinishedMutations() const
|
||||
{
|
||||
std::map<std::string, MutationCommands> result;
|
||||
std::lock_guard lock(state_mutex);
|
||||
|
||||
size_t count = 0;
|
||||
for (const auto & [_, status] : mutations_by_znode | std::views::reverse)
|
||||
for (const auto & [name, status] : mutations_by_znode | std::views::reverse)
|
||||
{
|
||||
if (status.is_done)
|
||||
break;
|
||||
++count;
|
||||
result.emplace(name, status.entry->commands);
|
||||
}
|
||||
|
||||
return count;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zkutil::ZooKeeperPtr & zookeeper,
|
||||
std::optional<PartitionIdsHint> && partition_ids_hint)
|
||||
{
|
||||
@ -2211,7 +2213,7 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate(
|
||||
|
||||
committing_blocks = std::make_shared<CommittingBlocks>(getCommittingBlocks(zookeeper, queue.zookeeper_path, queue.log));
|
||||
|
||||
merges_version = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE);
|
||||
std::tie(merges_version, std::ignore) = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE);
|
||||
|
||||
{
|
||||
/// We avoid returning here a version to be used in a lightweight transaction.
|
||||
|
@ -331,11 +331,11 @@ public:
|
||||
* Additionally loads mutations (so that the set of mutations is always more recent than the queue).
|
||||
* Return the version of "logs" node (that is updated for every merge/mutation/... added to the log)
|
||||
*/
|
||||
int32_t pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}, PullLogsReason reason = OTHER);
|
||||
std::pair<int32_t, int32_t> pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}, PullLogsReason reason = OTHER);
|
||||
|
||||
/// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task.
|
||||
/// If watch_callback is not empty, will call it when new mutations appear in ZK.
|
||||
void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback = {});
|
||||
int32_t updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback = {});
|
||||
|
||||
/// Remove a mutation from ZooKeeper and from the local set. Returns the removed entry or nullptr
|
||||
/// if it could not be found. Called during KILL MUTATION query execution.
|
||||
@ -388,12 +388,12 @@ public:
|
||||
|
||||
/// Count the total number of active mutations that are finished (is_done = true).
|
||||
size_t countFinishedMutations() const;
|
||||
/// Count the total number of active mutations that are not finished (is_done = false).
|
||||
size_t countUnfinishedMutations() const;
|
||||
|
||||
std::map<std::string, MutationCommands> getUnfinishedMutations() const;
|
||||
|
||||
/// Returns functor which used by MergeTreeMergerMutator to select parts for merge
|
||||
ReplicatedMergeTreeMergePredicate getMergePredicate(zkutil::ZooKeeperPtr & zookeeper,
|
||||
std::optional<PartitionIdsHint> && partition_ids_hint);
|
||||
ReplicatedMergeTreeMergePredicate
|
||||
getMergePredicate(zkutil::ZooKeeperPtr & zookeeper, std::optional<PartitionIdsHint> && partition_ids_hint);
|
||||
|
||||
MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version,
|
||||
Strings & mutation_ids) const;
|
||||
|
@ -702,6 +702,33 @@ std::optional<MergeTreeMutationStatus> StorageMergeTree::getIncompleteMutationsS
|
||||
return result;
|
||||
}
|
||||
|
||||
std::map<std::string, MutationCommands> StorageMergeTree::getUnfinishedMutationCommands() const
|
||||
{
|
||||
std::lock_guard lock(currently_processing_in_background_mutex);
|
||||
std::vector<PartVersionWithName> part_versions_with_names;
|
||||
auto data_parts = getDataPartsVectorForInternalUsage();
|
||||
part_versions_with_names.reserve(data_parts.size());
|
||||
for (const auto & part : data_parts)
|
||||
part_versions_with_names.emplace_back(PartVersionWithName{part->info.getDataVersion(), part->name});
|
||||
std::sort(part_versions_with_names.begin(), part_versions_with_names.end(), comparator);
|
||||
|
||||
std::map<std::string, MutationCommands> result;
|
||||
|
||||
for (const auto & kv : current_mutations_by_version)
|
||||
{
|
||||
Int64 mutation_version = kv.first;
|
||||
const MergeTreeMutationEntry & entry = kv.second;
|
||||
const PartVersionWithName needle{mutation_version, ""};
|
||||
auto versions_it = std::lower_bound(
|
||||
part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator);
|
||||
|
||||
size_t parts_to_do = versions_it - part_versions_with_names.begin();
|
||||
if (parts_to_do > 0)
|
||||
result.emplace(entry.file_name, entry.commands);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
std::vector<MergeTreeMutationStatus> StorageMergeTree::getMutationsStatus() const
|
||||
{
|
||||
std::lock_guard lock(currently_processing_in_background_mutex);
|
||||
@ -1381,26 +1408,6 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign
|
||||
return scheduled;
|
||||
}
|
||||
|
||||
size_t StorageMergeTree::getNumberOfUnfinishedMutations() const
|
||||
{
|
||||
std::unique_lock lock(currently_processing_in_background_mutex);
|
||||
|
||||
size_t count = 0;
|
||||
for (const auto & [version, _] : current_mutations_by_version | std::views::reverse)
|
||||
{
|
||||
auto status = getIncompleteMutationsStatusUnlocked(version, lock, nullptr, true);
|
||||
if (!status)
|
||||
continue;
|
||||
|
||||
if (status->is_done)
|
||||
break;
|
||||
|
||||
++count;
|
||||
}
|
||||
|
||||
return count;
|
||||
}
|
||||
|
||||
UInt64 StorageMergeTree::getCurrentMutationVersion(
|
||||
const DataPartPtr & part,
|
||||
std::unique_lock<std::mutex> & /*currently_processing_in_background_mutex_lock*/) const
|
||||
|
@ -112,7 +112,7 @@ public:
|
||||
|
||||
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
||||
|
||||
size_t getNumberOfUnfinishedMutations() const override;
|
||||
std::map<std::string, MutationCommands> getUnfinishedMutationCommands() const override;
|
||||
|
||||
MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); }
|
||||
|
||||
|
@ -5739,6 +5739,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
auto ast_to_str = [](ASTPtr query) -> String
|
||||
{
|
||||
if (!query)
|
||||
@ -5753,6 +5754,31 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (shutdown_called || partial_shutdown_called)
|
||||
throw Exception(ErrorCodes::ABORTED, "Cannot assign alter because shutdown called");
|
||||
|
||||
bool pulled_queue = false;
|
||||
std::optional<int32_t> maybe_mutations_version_after_logs_pull;
|
||||
std::map<std::string, MutationCommands> unfinished_mutations;
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.isDropSomething())
|
||||
{
|
||||
if (shutdown_called || partial_shutdown_called)
|
||||
throw Exception(ErrorCodes::ABORTED, "Cannot assign alter because shutdown called");
|
||||
|
||||
if (!pulled_queue)
|
||||
{
|
||||
auto [_, mutations_version] = queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::SYNC);
|
||||
maybe_mutations_version_after_logs_pull.emplace(mutations_version);
|
||||
unfinished_mutations = getUnfinishedMutationCommands();
|
||||
pulled_queue = true;
|
||||
}
|
||||
|
||||
checkDropCommandDoesntAffectInProgressMutations(command, unfinished_mutations, query_context);
|
||||
}
|
||||
}
|
||||
|
||||
/// Clear nodes from previous iteration
|
||||
alter_entry.emplace();
|
||||
mutation_znode.reset();
|
||||
@ -5866,8 +5892,18 @@ void StorageReplicatedMergeTree::alter(
|
||||
mutation_entry.source_replica = replica_name;
|
||||
mutation_entry.commands = std::move(maybe_mutation_commands);
|
||||
|
||||
Coordination::Stat mutations_stat;
|
||||
zookeeper->get(mutations_path, &mutations_stat);
|
||||
int32_t mutations_version;
|
||||
if (maybe_mutations_version_after_logs_pull.has_value())
|
||||
{
|
||||
mutations_version = *maybe_mutations_version_after_logs_pull;
|
||||
}
|
||||
else
|
||||
{
|
||||
Coordination::Stat mutations_stat;
|
||||
zookeeper->get(mutations_path, &mutations_stat);
|
||||
mutations_version = mutations_stat.version;
|
||||
}
|
||||
|
||||
|
||||
partition_block_numbers_holder =
|
||||
allocateBlockNumbersInAffectedPartitions(mutation_entry.commands, query_context, zookeeper);
|
||||
@ -5875,7 +5911,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
mutation_entry.block_numbers = partition_block_numbers_holder.getBlockNumbers();
|
||||
mutation_entry.create_time = time(nullptr);
|
||||
|
||||
ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
|
||||
ops.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_version));
|
||||
mutation_path_idx = ops.size();
|
||||
ops.emplace_back(
|
||||
zkutil::makeCreateRequest(fs::path(mutations_path) / "", mutation_entry.toString(), zkutil::CreateMode::PersistentSequential));
|
||||
@ -8701,9 +8737,9 @@ String StorageReplicatedMergeTree::getTableSharedID() const
|
||||
return toString(table_shared_id);
|
||||
}
|
||||
|
||||
size_t StorageReplicatedMergeTree::getNumberOfUnfinishedMutations() const
|
||||
std::map<std::string, MutationCommands> StorageReplicatedMergeTree::getUnfinishedMutationCommands() const
|
||||
{
|
||||
return queue.countUnfinishedMutations();
|
||||
return queue.getUnfinishedMutations();
|
||||
}
|
||||
|
||||
void StorageReplicatedMergeTree::createTableSharedID() const
|
||||
|
@ -347,7 +347,7 @@ public:
|
||||
// Return table id, common for different replicas
|
||||
String getTableSharedID() const override;
|
||||
|
||||
size_t getNumberOfUnfinishedMutations() const override;
|
||||
std::map<std::string, MutationCommands> getUnfinishedMutationCommands() const override;
|
||||
|
||||
/// Returns the same as getTableSharedID(), but extracts it from a create query.
|
||||
static std::optional<String> tryGetTableSharedIDFromCreateQuery(const IAST & create_query, const ContextPtr & global_context);
|
||||
|
@ -0,0 +1,27 @@
|
||||
CREATE TABLE test (
|
||||
`c_id` String,
|
||||
`p_id` String,
|
||||
`d` String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1')
|
||||
ORDER BY (c_id, p_id);
|
||||
|
||||
INSERT INTO test SELECT '1', '11', '111' FROM numbers(3);
|
||||
|
||||
INSERT INTO test SELECT '2', '22', '22' FROM numbers(3);
|
||||
|
||||
set mutations_sync=0;
|
||||
|
||||
ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1;
|
||||
|
||||
ALTER TABLE test ADD COLUMN x UInt32 default 0;
|
||||
ALTER TABLE test UPDATE d = d || '1' where x = 42;
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36}
|
||||
|
||||
ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2;
|
||||
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2;
|
||||
|
||||
select * from test format Null;
|
||||
|
||||
DROP TABLE test;
|
@ -0,0 +1,27 @@
|
||||
CREATE TABLE test (
|
||||
`c_id` String,
|
||||
`p_id` String,
|
||||
`d` String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1')
|
||||
ORDER BY (c_id, p_id);
|
||||
|
||||
INSERT INTO test SELECT '1', '11', '111' FROM numbers(3);
|
||||
|
||||
INSERT INTO test SELECT '2', '22', '22' FROM numbers(3);
|
||||
|
||||
set mutations_sync=0;
|
||||
|
||||
ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1;
|
||||
|
||||
ALTER TABLE test ADD COLUMN x UInt32 default 0;
|
||||
ALTER TABLE test UPDATE x = x + 1 where 1;
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36}
|
||||
|
||||
ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2;
|
||||
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2;
|
||||
|
||||
select * from test format Null;
|
||||
|
||||
DROP TABLE test;
|
@ -0,0 +1,28 @@
|
||||
CREATE TABLE test (
|
||||
`c_id` String,
|
||||
`p_id` String,
|
||||
`d` UInt32
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1')
|
||||
ORDER BY (c_id, p_id);
|
||||
|
||||
INSERT INTO test SELECT '1', '11', '111' FROM numbers(3);
|
||||
|
||||
INSERT INTO test SELECT '2', '22', '22' FROM numbers(3);
|
||||
|
||||
set mutations_sync=0;
|
||||
|
||||
ALTER TABLE test UPDATE d = d + sleepEachRow(0.3) where 1;
|
||||
|
||||
ALTER TABLE test ADD COLUMN x UInt32 default 0;
|
||||
ALTER TABLE test UPDATE d = x + 1 where 1;
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2; --{serverError 36}
|
||||
|
||||
ALTER TABLE test UPDATE x = x + 1 where 1 SETTINGS mutations_sync = 2;
|
||||
|
||||
ALTER TABLE test DROP COLUMN x SETTINGS mutations_sync = 2;
|
||||
|
||||
select * from test format Null;
|
||||
|
||||
DROP TABLE test;
|
||||
|
@ -0,0 +1,28 @@
|
||||
CREATE TABLE test (
|
||||
`c_id` String,
|
||||
`p_id` String,
|
||||
`d` String
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/test_table', '1')
|
||||
ORDER BY (c_id, p_id);
|
||||
|
||||
INSERT INTO test SELECT '1', '11', '111' FROM numbers(3);
|
||||
|
||||
INSERT INTO test SELECT '2', '22', '22' FROM numbers(3);
|
||||
|
||||
set mutations_sync=0;
|
||||
|
||||
ALTER TABLE test UPDATE d = d || toString(sleepEachRow(0.3)) where 1;
|
||||
|
||||
ALTER TABLE test ADD PROJECTION d_order ( SELECT min(c_id) GROUP BY `d`);
|
||||
ALTER TABLE test MATERIALIZE PROJECTION d_order;
|
||||
ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2; --{serverError 36}
|
||||
|
||||
-- just to wait prev mutation
|
||||
ALTER TABLE test DELETE where d = 'Hello' SETTINGS mutations_sync = 2;
|
||||
|
||||
ALTER TABLE test DROP PROJECTION d_order SETTINGS mutations_sync = 2;
|
||||
|
||||
select * from test format Null;
|
||||
|
||||
DROP TABLE test;
|
Loading…
Reference in New Issue
Block a user