From cb7db6d232e9ece0c16cf9080c865d9eef7ce1e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 15:40:46 +0300 Subject: [PATCH 001/149] Split select and process merges (cherry picked from commit 1889eb0ff18b8ba75744dd795ee3339df9ad3e48) --- src/Storages/StorageMergeTree.cpp | 366 ++++++++++---------- src/Storages/StorageMergeTree.h | 18 +- src/Storages/StorageReplicatedMergeTree.cpp | 44 ++- src/Storages/StorageReplicatedMergeTree.h | 5 + 4 files changed, 239 insertions(+), 194 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0f1afe1bd62..6a2823031f2 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -622,6 +622,100 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } +std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +{ + std::unique_lock lock(currently_processing_in_background_mutex); + auto data_settings = getSettings(); + + FutureMergedMutatedPart future_part; + + /// You must call destructor with unlocked `currently_processing_in_background_mutex`. + CurrentlyMergingPartsTaggerPtr merging_tagger; + MergeList::EntryPtr merge_entry; + + auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) -> bool + { + /// This predicate is checked for the first part of each partition. + /// (left = nullptr, right = "first part of partition") + if (!left) + return !currently_merging_mutating_parts.count(right); + return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) + && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); + }; + + bool selected = false; + + if (partition_id.empty()) + { + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); + bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool; + + /// TTL requirements is much more strict than for regular merge, so + /// if regular not possible, than merge with ttl is not also not + /// possible. + if (max_source_parts_size > 0) + { + selected = merger_mutator.selectPartsToMerge( + future_part, + aggressive, + max_source_parts_size, + can_merge, + merge_with_ttl_allowed, + out_disable_reason); + } + else if (out_disable_reason) + *out_disable_reason = "Current value of max_source_parts_size is zero"; + } + else + { + while (true) + { + UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); + selected = merger_mutator.selectAllPartsToMergeWithinPartition( + future_part, disk_space, can_merge, partition_id, final, out_disable_reason); + + /// If final - we will wait for currently processing merges to finish and continue. + /// TODO Respect query settings for timeout + if (final + && !selected + && !currently_merging_mutating_parts.empty() + && out_disable_reason + && out_disable_reason->empty()) + { + LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", + currently_merging_mutating_parts.size()); + + if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( + lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) + { + *out_disable_reason = "Timeout while waiting for already running merges before running OPTIMIZE with FINAL"; + break; + } + } + else + break; + } + } + + if (!selected) + { + if (out_disable_reason) + { + if (!out_disable_reason->empty()) + { + *out_disable_reason += ". "; + } + *out_disable_reason += "Cannot select parts for optimization"; + } + + return {}; + } + + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); + auto table_id = getStorageID(); + merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); + return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; +} bool StorageMergeTree::merge( bool aggressive, @@ -632,99 +726,12 @@ bool StorageMergeTree::merge( { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto data_settings = getSettings(); - FutureMergedMutatedPart future_part; - - /// You must call destructor with unlocked `currently_processing_in_background_mutex`. - std::optional merging_tagger; - MergeList::EntryPtr merge_entry; - - { - std::unique_lock lock(currently_processing_in_background_mutex); - - auto can_merge = [this, &lock] (const DataPartPtr & left, const DataPartPtr & right, String *) -> bool - { - /// This predicate is checked for the first part of each partition. - /// (left = nullptr, right = "first part of partition") - if (!left) - return !currently_merging_mutating_parts.count(right); - return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) - && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); - }; - - bool selected = false; - - if (partition_id.empty()) - { - UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSizeForMerge(); - bool merge_with_ttl_allowed = getTotalMergesWithTTLInMergeList() < data_settings->max_number_of_merges_with_ttl_in_pool; - - /// TTL requirements is much more strict than for regular merge, so - /// if regular not possible, than merge with ttl is not also not - /// possible. - if (max_source_parts_size > 0) - { - selected = merger_mutator.selectPartsToMerge( - future_part, - aggressive, - max_source_parts_size, - can_merge, - merge_with_ttl_allowed, - out_disable_reason); - } - else if (out_disable_reason) - *out_disable_reason = "Current value of max_source_parts_size is zero"; - } - else - { - while (true) - { - UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, out_disable_reason); - - /// If final - we will wait for currently processing merges to finish and continue. - /// TODO Respect query settings for timeout - if (final - && !selected - && !currently_merging_mutating_parts.empty() - && out_disable_reason - && out_disable_reason->empty()) - { - LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", - currently_merging_mutating_parts.size()); - - if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( - lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) - { - *out_disable_reason = "Timeout while waiting for already running merges before running OPTIMIZE with FINAL"; - break; - } - } - else - break; - } - } - - if (!selected) - { - if (out_disable_reason) - { - if (!out_disable_reason->empty()) - { - *out_disable_reason += ". "; - } - *out_disable_reason += "Cannot select parts for optimization"; - } - return false; - } - - merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - auto table_id = getStorageID(); - merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - } + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); + if (!merge_mutate_entry) + return false; + auto & future_part = merge_mutate_entry->future_part; /// Logging Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -738,14 +745,14 @@ bool StorageMergeTree::merge( future_part.name, new_part, future_part.parts, - merge_entry.get()); + merge_mutate_entry->merge_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *merge_entry, table_lock_holder, time(nullptr), global_context, - merging_tagger->reserved_space, deduplicate); + future_part, metadata_snapshot, *(merge_mutate_entry->merge_entry), table_lock_holder, time(nullptr), + merge_mutate_entry->tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -783,95 +790,98 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() } } +std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) +{ + std::lock_guard lock(currently_processing_in_background_mutex); + size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; + + FutureMergedMutatedPart future_part; + MutationCommands commands; + + CurrentlyMergingPartsTaggerPtr tagger; + + if (current_mutations_by_version.empty()) + return {}; + + auto mutations_end_it = current_mutations_by_version.end(); + for (const auto & part : getDataPartsVector()) + { + if (currently_merging_mutating_parts.count(part)) + continue; + + auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion()); + if (mutations_begin_it == mutations_end_it) + continue; + + size_t max_source_part_size = merger_mutator.getMaxSourcePartSizeForMutation(); + if (max_source_part_size < part->getBytesOnDisk()) + { + LOG_DEBUG(log, "Current max source part size for mutation is {} but part size {}. Will not mutate part {}. " + "Max size depends not only on available space, but also on settings " + "'number_of_free_entries_in_pool_to_execute_mutation' and 'background_pool_size'", + max_source_part_size, part->getBytesOnDisk(), part->name); + continue; + } + + size_t current_ast_elements = 0; + for (auto it = mutations_begin_it; it != mutations_end_it; ++it) + { + size_t commands_size = 0; + MutationCommands commands_for_size_validation; + for (const auto & command : it->second.commands) + { + if (command.type != MutationCommand::Type::DROP_COLUMN + && command.type != MutationCommand::Type::DROP_INDEX + && command.type != MutationCommand::Type::RENAME_COLUMN) + { + commands_for_size_validation.push_back(command); + } + else + { + commands_size += command.ast->size(); + } + } + + if (!commands_for_size_validation.empty()) + { + MutationsInterpreter interpreter( + shared_from_this(), metadata_snapshot, commands_for_size_validation, global_context, false); + commands_size += interpreter.evaluateCommandsSize(); + } + + if (current_ast_elements + commands_size >= max_ast_elements) + break; + + current_ast_elements += commands_size; + commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end()); + } + + auto new_part_info = part->info; + new_part_info.mutation = current_mutations_by_version.rbegin()->first; + + future_part.parts.push_back(part); + future_part.part_info = new_part_info; + future_part.name = part->getNewName(new_part_info); + future_part.type = part->getType(); + + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, true); + auto table_id = getStorageID(); + MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); + return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; + } + return {}; +} bool StorageMergeTree::tryMutatePart() { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); - size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; - FutureMergedMutatedPart future_part; - MutationCommands commands; - /// You must call destructor with unlocked `currently_processing_in_background_mutex`. - std::optional tagger; - { - std::lock_guard lock(currently_processing_in_background_mutex); - - if (current_mutations_by_version.empty()) - return false; - - auto mutations_end_it = current_mutations_by_version.end(); - for (const auto & part : getDataPartsVector()) - { - if (currently_merging_mutating_parts.count(part)) - continue; - - auto mutations_begin_it = current_mutations_by_version.upper_bound(part->info.getDataVersion()); - if (mutations_begin_it == mutations_end_it) - continue; - - size_t max_source_part_size = merger_mutator.getMaxSourcePartSizeForMutation(); - if (max_source_part_size < part->getBytesOnDisk()) - { - LOG_DEBUG(log, "Current max source part size for mutation is {} but part size {}. Will not mutate part {}. " - "Max size depends not only on available space, but also on settings " - "'number_of_free_entries_in_pool_to_execute_mutation' and 'background_pool_size'", - max_source_part_size, part->getBytesOnDisk(), part->name); - continue; - } - - size_t current_ast_elements = 0; - for (auto it = mutations_begin_it; it != mutations_end_it; ++it) - { - size_t commands_size = 0; - MutationCommands commands_for_size_validation; - for (const auto & command : it->second.commands) - { - if (command.type != MutationCommand::Type::DROP_COLUMN - && command.type != MutationCommand::Type::DROP_INDEX - && command.type != MutationCommand::Type::RENAME_COLUMN) - { - commands_for_size_validation.push_back(command); - } - else - { - commands_size += command.ast->size(); - } - } - - if (!commands_for_size_validation.empty()) - { - MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, global_context, false); - commands_size += interpreter.evaluateCommandsSize(); - } - - if (current_ast_elements + commands_size >= max_ast_elements) - break; - - current_ast_elements += commands_size; - commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end()); - } - - auto new_part_info = part->info; - new_part_info.mutation = current_mutations_by_version.rbegin()->first; - - future_part.parts.push_back(part); - future_part.part_info = new_part_info; - future_part.name = part->getNewName(new_part_info); - future_part.type = part->getType(); - - tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - break; - } - } - - if (!tagger) + auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + if (!merge_mutate_entry) return false; - auto table_id = getStorageID(); - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - + auto & future_part = merge_mutate_entry->future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -884,14 +894,14 @@ bool StorageMergeTree::tryMutatePart() future_part.name, new_part, future_part.parts, - merge_entry.get()); + merge_mutate_entry->merge_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, commands, *merge_entry, - time(nullptr), global_context, tagger->reserved_space, table_lock_holder); + future_part, metadata_snapshot, merge_mutate_entry->commands, *(merge_mutate_entry->merge_entry), + time(nullptr), global_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 5662f9e0088..9b740f575c2 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -21,6 +21,8 @@ namespace DB { +struct CurrentlyMergingPartsTagger; + /** See the description of the data structure in MergeTreeData. */ class StorageMergeTree final : public ext::shared_ptr_helper, public MergeTreeData @@ -140,6 +142,20 @@ private: /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. bool tryMutatePart(); + friend struct CurrentlyMergingPartsTagger; + + using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + + struct MergeMutateSelectedEntry + { + FutureMergedMutatedPart future_part; + CurrentlyMergingPartsTaggerPtr tagger; + MergeList::EntryPtr merge_entry; + MutationCommands commands; + }; + + std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); BackgroundProcessingPoolTaskResult mergeMutateTask(); @@ -173,7 +189,7 @@ private: friend class MergeTreeBlockOutputStream; friend class MergeTreeData; - friend struct CurrentlyMergingPartsTagger; + protected: diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2ac8ddb7846..ae30977a9f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2537,16 +2537,8 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } } - -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() +ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueEntry() { - /// If replication queue is stopped exit immediately as we successfully executed the task - if (queue.actions_blocker.isCancelled()) - { - std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - /// This object will mark the element of the queue as running. ReplicatedMergeTreeQueue::SelectedEntry selected; @@ -2559,14 +2551,14 @@ BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() tryLogCurrentException(log, __PRETTY_FUNCTION__); } - LogEntryPtr & entry = selected.first; + return selected; +} - if (!entry) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; +bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & selected_entry) +{ - time_t prev_attempt_time = entry->last_attempt_time; - - bool res = queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) + LogEntryPtr & entry = selected_entry.first; + return queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) { try { @@ -2605,6 +2597,28 @@ BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() throw; } }); +} + +BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() +{ + /// If replication queue is stopped exit immediately as we successfully executed the task + if (queue.actions_blocker.isCancelled()) + { + std::this_thread::sleep_for(std::chrono::milliseconds(5)); + return BackgroundProcessingPoolTaskResult::SUCCESS; + } + + /// This object will mark the element of the queue as running. + ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + + LogEntryPtr & entry = selected_entry.first; + + if (!entry) + return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; + + time_t prev_attempt_time = entry->last_attempt_time; + + bool res = processQueueEntry(selected_entry); /// We will go to sleep if the processing fails and if we have already processed this record recently. bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 0356cccd302..bb13cd7f230 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -416,6 +416,11 @@ private: /// Clone replica if it is lost. void cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper); + + ReplicatedMergeTreeQueue::SelectedEntry selectQueueEntry(); + + bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); + /** Performs actions from the queue. */ BackgroundProcessingPoolTaskResult queueTask(); From 48aaa0ce160b73b3ba83bdff7bd9cfaf55939191 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 16:49:22 +0300 Subject: [PATCH 002/149] Split merge to separate method (cherry picked from commit 14a7a25c4b5d52134349300091df034ab6e652a4) --- src/Storages/StorageMergeTree.cpp | 34 +++++++++++++++++++------------ src/Storages/StorageMergeTree.h | 3 +++ 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6a2823031f2..cf800dc0d50 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -624,6 +624,7 @@ void StorageMergeTree::loadMutations() std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -724,15 +725,18 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); if (!merge_mutate_entry) return false; - auto & future_part = merge_mutate_entry->future_part; - /// Logging + return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); +} + +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -745,14 +749,14 @@ bool StorageMergeTree::merge( future_part.name, new_part, future_part.parts, - merge_mutate_entry->merge_entry.get()); + merge_mutate_entry.merge_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *(merge_mutate_entry->merge_entry), table_lock_holder, time(nullptr), - merge_mutate_entry->tagger->reserved_space, deduplicate); + future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), + merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -766,7 +770,6 @@ bool StorageMergeTree::merge( return true; } - bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const { std::lock_guard background_processing_lock(currently_processing_in_background_mutex); @@ -792,6 +795,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -874,14 +878,19 @@ std::optional StorageMergeTree::sele bool StorageMergeTree::tryMutatePart() { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); if (!merge_mutate_entry) return false; - auto & future_part = merge_mutate_entry->future_part; + return mutateSelectedPart(metadata_snapshot, *merge_mutate_entry); +} + +bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) +{ + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -894,14 +903,14 @@ bool StorageMergeTree::tryMutatePart() future_part.name, new_part, future_part.parts, - merge_mutate_entry->merge_entry.get()); + merge_mutate_entry.merge_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry->commands, *(merge_mutate_entry->merge_entry), - time(nullptr), global_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); + future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_mutate_entry.merge_entry), + time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); @@ -918,7 +927,6 @@ bool StorageMergeTree::tryMutatePart() return true; } - BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 9b740f575c2..ac6939bd900 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -155,7 +155,10 @@ private: }; std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); + bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); BackgroundProcessingPoolTaskResult mergeMutateTask(); From ef355e71d6026f43df5879fd22438384a25db112 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Oct 2020 15:10:46 +0300 Subject: [PATCH 003/149] Buildable code --- src/Storages/StorageMergeTree.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cf800dc0d50..48563fc0c40 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -622,7 +622,7 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr &, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); @@ -712,7 +712,7 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, false); + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); auto table_id = getStorageID(); merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; @@ -756,7 +756,7 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn { new_part = merger_mutator.mergePartsToTemporaryPart( future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), - merge_mutate_entry.tagger->reserved_space, deduplicate); + global_context, merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); write_part_log({}); @@ -868,7 +868,7 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, true); + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); auto table_id = getStorageID(); MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; From dcf489ac8f8887588cd973a955abe916dabfaca8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 16:56:44 +0300 Subject: [PATCH 004/149] Remove unused methods (cherry picked from commit 77f5841a46016a05d757c2415d3b881390c8ba7a) --- src/Storages/StorageMergeTree.cpp | 18 +++++------------- src/Storages/StorageMergeTree.h | 2 -- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 48563fc0c40..b68ce9be37b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -876,17 +876,6 @@ std::optional StorageMergeTree::sele return {}; } -bool StorageMergeTree::tryMutatePart() -{ - StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr(); - - auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); - if (!merge_mutate_entry) - return false; - - return mutateSelectedPart(metadata_snapshot, *merge_mutate_entry); -} - bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -951,11 +940,14 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() clearOldMutations(); } + auto metadata_snapshot = getInMemoryMetadataPtr(); + auto merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); ///TODO: read deduplicate option from table config - if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) + if (merge_entry && mergeSelectedParts(metadata_snapshot, false, *merge_entry)) return BackgroundProcessingPoolTaskResult::SUCCESS; - if (tryMutatePart()) + auto mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + if (mutate_entry && mutateSelectedPart(metadata_snapshot, *mutate_entry)) return BackgroundProcessingPoolTaskResult::SUCCESS; return BackgroundProcessingPoolTaskResult::ERROR; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ac6939bd900..519352a0a8a 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -140,8 +140,6 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version, const String & file_name); - /// Try and find a single part to mutate and mutate it. If some part was successfully mutated, return true. - bool tryMutatePart(); friend struct CurrentlyMergingPartsTagger; using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; From 163d33fd214013e47a17f68db69a9f86d82c2777 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 9 Oct 2020 16:34:44 +0300 Subject: [PATCH 005/149] Add perftest for merges (cherry picked from commit e655f009d2f171b30feb34da2bc25706a0e3f91e) --- tests/performance/insert_sequential.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 tests/performance/insert_sequential.xml diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential.xml new file mode 100644 index 00000000000..0c98c04ec8b --- /dev/null +++ b/tests/performance/insert_sequential.xml @@ -0,0 +1,11 @@ + + + + 1 + 1 + 1 + + CREATE TABLE t (x UInt64) ENGINE = MergeTree() ORDER BY tuple() + INSERT INTO t SELECT * FROM numbers(20000) + DROP TABLE IF EXISTS t + From fd35368c5911a84c40ac2b12e55d86c84c1d3517 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 13 Oct 2020 17:25:42 +0300 Subject: [PATCH 006/149] First simple implementation --- .../MergeTree/BackgroundJobsExecutor.cpp | 49 ++++++++++ .../MergeTree/BackgroundJobsExecutor.h | 34 +++++++ .../MergeTree/MergeTreeBackgroundJob.h | 46 ++++++++++ .../MergeTree/MergeTreeBlockOutputStream.cpp | 7 +- src/Storages/MergeTree/MergeTreeData.h | 3 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/StorageMergeTree.cpp | 92 ++++++++++++++----- src/Storages/StorageMergeTree.h | 7 +- src/Storages/StorageReplicatedMergeTree.cpp | 25 ++++- src/Storages/StorageReplicatedMergeTree.h | 2 + 12 files changed, 238 insertions(+), 34 deletions(-) create mode 100644 src/Storages/MergeTree/BackgroundJobsExecutor.cpp create mode 100644 src/Storages/MergeTree/BackgroundJobsExecutor.h create mode 100644 src/Storages/MergeTree/MergeTreeBackgroundJob.h diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp new file mode 100644 index 00000000000..662fa71d318 --- /dev/null +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -0,0 +1,49 @@ +#include + +namespace DB +{ + +BackgroundJobsExecutor::BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context) + : data(data_) + , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) + , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) +{ + data_processing_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); +} + +void BackgroundJobsExecutor::dataProcessingTask() +try +{ + auto job = data.getDataProcessingJob(); + if (job) + data_processing_pool.scheduleOrThrowOnError(*job); + + data_processing_task->schedule(); +} +catch (...) +{ + tryLogCurrentException(__PRETTY_FUNCTION__); +} + +void BackgroundJobsExecutor::start() +{ + if (data_processing_task) + data_processing_task->activateAndSchedule(); +} + +void BackgroundJobsExecutor::triggerDataProcessing() +{ + if (data_processing_task) + data_processing_task->schedule(); +} + +void BackgroundJobsExecutor::finish() +{ + data_processing_task->deactivate(); + data_processing_pool.wait(); +} + +} diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h new file mode 100644 index 00000000000..aa166eb4d73 --- /dev/null +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class BackgroundJobsExecutor +{ +private: + MergeTreeData & data; + ThreadPool data_processing_pool; + ThreadPool move_pool; + + BackgroundSchedulePool::TaskHolder data_processing_task; + BackgroundSchedulePool::TaskHolder move_processing_task; + + void dataProcessingTask(); + +public: + BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context_); + + void triggerDataProcessing(); + void triggerMovesProcessing(); + void start(); + void finish(); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h new file mode 100644 index 00000000000..2f432114db1 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeBackgroundJob.h @@ -0,0 +1,46 @@ +# pragma once +#include +#include +#include +#include + +namespace DB +{ + +enum PoolType +{ + MERGE_MUTATE, + MOVE, + FETCH, +}; + +struct MergeTreeBackgroundJob +{ + ThreadPool::Job job; + CurrentMetrics::Metric metric; + PoolType execute_in_pool; + + MergeTreeBackgroundJob(ThreadPool::Job && job_, CurrentMetrics::Metric metric_, PoolType execute_in_pool_) + : job(std::move(job_)), metric(metric_), execute_in_pool(execute_in_pool_) + {} + + void operator()() + try + { + if (metric != 0) + { + CurrentMetrics::Increment metric_increment{metric}; + job(); + } + else + { + job(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 4269abe2655..5ad6a7eebc1 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -27,11 +27,8 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); - if (storage.merging_mutating_task_handle) - { - /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. - storage.merging_mutating_task_handle->signalReadyToRun(); - } + /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. + storage.background_executor.triggerDataProcessing(); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5c18661dad1..8c5333315fc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -710,6 +711,8 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; + virtual std::optional getDataProcessingJob() { return {}; } + protected: friend class IMergeTreeDataPart; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fb0a488700c..942bac0d294 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -154,7 +154,6 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si { } - UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -166,7 +165,9 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { if (pool_used > pool_size) + { throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); + } size_t free_entries = pool_size - pool_used; const auto data_settings = data.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 45e16e81208..48caf59e7ba 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1287,7 +1287,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP } if (entry) - return { entry, std::unique_ptr{ new CurrentlyExecuting(entry, *this) } }; + return { entry, std::shared_ptr{ new CurrentlyExecuting(entry, *this) } }; else return {}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 88a61f50225..c72569a5071 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -319,7 +319,7 @@ public: /** Select the next action to process. * merger_mutator is used only to check if the merges are not suspended. */ - using SelectedEntry = std::pair>; + using SelectedEntry = std::pair>; SelectedEntry selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); /** Execute `func` function to handle the action. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b68ce9be37b..00ddb7a57ce 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,6 +27,10 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} namespace DB { @@ -73,7 +77,8 @@ StorageMergeTree::StorageMergeTree( attach) , reader(*this) , writer(*this) - , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) + , background_executor(*this, global_context) { loadDataParts(has_force_restore_data_flag); @@ -100,11 +105,7 @@ void StorageMergeTree::startup() try { - auto & merge_pool = global_context.getBackgroundPool(); - merging_mutating_task_handle = merge_pool.createTask([this] { return mergeMutateTask(); }); - /// Ensure that thread started only after assignment to 'merging_mutating_task_handle' is done. - merge_pool.startTask(merging_mutating_task_handle); - + background_executor.start(); startBackgroundMovesIfNeeded(); } catch (...) @@ -142,8 +143,7 @@ void StorageMergeTree::shutdown() merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); - if (merging_mutating_task_handle) - global_context.getBackgroundPool().removeTask(merging_mutating_task_handle); + background_executor.finish(); if (moving_task_handle) global_context.getBackgroundMovePool().removeTask(moving_task_handle); @@ -361,7 +361,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String current_mutations_by_version.emplace(version, insertion.first->second); LOG_INFO(log, "Added mutation: {}", mutation_file_name); - merging_mutating_task_handle->signalReadyToRun(); + background_executor.triggerDataProcessing(); return version; } @@ -591,7 +591,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) } /// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately. - merging_mutating_task_handle->signalReadyToRun(); + background_executor.triggerDataProcessing(); return CancellationCode::CancelSent; } @@ -712,10 +712,8 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - auto table_id = getStorageID(); - merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), std::move(merge_entry), {}}; + merging_tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); + return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), {}}; } bool StorageMergeTree::merge( @@ -739,6 +737,9 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; + auto table_id = getStorageID(); + + auto merge_list_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); auto write_part_log = [&] (const ExecutionStatus & execution_status) { @@ -749,13 +750,13 @@ bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_sn future_part.name, new_part, future_part.parts, - merge_mutate_entry.merge_entry.get()); + merge_list_entry.get()); }; try { new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, metadata_snapshot, *(merge_mutate_entry.merge_entry), table_lock_holder, time(nullptr), + future_part, metadata_snapshot, *(merge_list_entry), table_lock_holder, time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, deduplicate); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); @@ -868,10 +869,8 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - auto table_id = getStorageID(); - MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); - return MergeMutateSelectedEntry{future_part, std::move(tagger), std::move(merge_entry), commands}; + tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); + return MergeMutateSelectedEntry{future_part, std::move(tagger), commands}; } return {}; } @@ -880,6 +879,9 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; + auto table_id = getStorageID(); + + auto merge_list_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part); Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -892,13 +894,13 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn future_part.name, new_part, future_part.parts, - merge_mutate_entry.merge_entry.get()); + merge_list_entry.get()); }; try { new_part = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_mutate_entry.merge_entry), + future_part, metadata_snapshot, merge_mutate_entry.commands, *(merge_list_entry), time(nullptr), global_context, merge_mutate_entry.tagger->reserved_space, table_lock_holder); renameTempPartAndReplace(new_part); @@ -916,6 +918,52 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } +std::optional StorageMergeTree::getDataProcessingJob() +{ + if (shutdown_called) + return {}; + + if (merger_mutator.merges_blocker.isCancelled()) + return {}; + + auto metadata_snapshot = getInMemoryMetadataPtr(); + std::optional merge_entry, mutate_entry; + + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); + if (!merge_entry) + mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + + if (merge_entry || mutate_entry) + { + auto job = [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + { + if (merge_entry) + mergeSelectedParts(metadata_snapshot, false, *merge_entry); + else if (mutate_entry) + mutateSelectedPart(metadata_snapshot, *mutate_entry); + }; + return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); + } + else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) + { + auto job = [this] () + { + { + auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + /// All use relative_data_path which changes during rename + /// so execute under share lock. + clearOldPartsFromFilesystem(); + clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); + } + clearOldMutations(); + }; + + return std::make_optional(std::move(job), 0, PoolType::MERGE_MUTATE); + } + return {}; +} + BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 519352a0a8a..c028e15416f 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -87,6 +88,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; + std::optional getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait @@ -119,7 +121,7 @@ private: std::atomic shutdown_called {false}; /// Task handler for merges, mutations and moves. - BackgroundProcessingPool::TaskHandle merging_mutating_task_handle; + BackgroundJobsExecutor background_executor; BackgroundProcessingPool::TaskHandle moving_task_handle; void loadMutations(); @@ -142,13 +144,12 @@ private: friend struct CurrentlyMergingPartsTagger; - using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + using CurrentlyMergingPartsTaggerPtr = std::shared_ptr; struct MergeMutateSelectedEntry { FutureMergedMutatedPart future_part; CurrentlyMergingPartsTaggerPtr tagger; - MergeList::EntryPtr merge_entry; MutationCommands commands; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ae30977a9f2..ae7ad4a3518 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -194,7 +194,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replica_path(zookeeper_path + "/replicas/" + replica_name) , reader(*this) , writer(*this) - , merger_mutator(*this, global_context.getBackgroundPool().getNumberOfThreads()) + , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , queue(*this) , fetcher(*this) , cleanup_thread(*this) @@ -2599,6 +2599,29 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel }); } + +std::optional StorageReplicatedMergeTree::getDataProcessingJob() +{ + /// If replication queue is stopped exit immediately as we successfully executed the task + if (queue.actions_blocker.isCancelled()) + return {}; + + /// This object will mark the element of the queue as running. + ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + + LogEntryPtr & entry = selected_entry.first; + + if (!entry) + return {}; + + auto job = [this, selected_entry{std::move(selected_entry)}] () mutable + { + processQueueEntry(selected_entry); + }; + + return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); +} + BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() { /// If replication queue is stopped exit immediately as we successfully executed the task diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bb13cd7f230..1b65ffdbc25 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -195,6 +195,8 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); + std::optional getDataProcessingJob() override; + private: /// Get a sequential consistent view of current parts. From efd3126b5d7979da7ff79f380fa2ee46c2d54c36 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 10:22:48 +0300 Subject: [PATCH 007/149] Moving pool --- .../MergeTree/BackgroundJobsExecutor.cpp | 34 +++++++++++++++- .../MergeTree/BackgroundJobsExecutor.h | 5 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 40 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 14 ++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 + src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 11 +++-- src/Storages/StorageMergeTree.cpp | 21 +++------- src/Storages/StorageMergeTree.h | 5 +-- src/Storages/StorageReplicatedMergeTree.cpp | 24 +++-------- src/Storages/StorageReplicatedMergeTree.h | 7 ++-- 13 files changed, 99 insertions(+), 70 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 662fa71d318..081ef818dcf 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -1,12 +1,14 @@ #include +#include namespace DB { BackgroundJobsExecutor::BackgroundJobsExecutor( MergeTreeData & data_, - Context & global_context) + Context & global_context_) : data(data_) + , global_context(global_context_) , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) { @@ -14,6 +16,20 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); } +void BackgroundJobsExecutor::dataMovingTask() +try +{ + auto job = data.getDataMovingJob(); + if (job) + move_pool.scheduleOrThrowOnError(*job); + + data_moving_task->schedule(); +} +catch(...) +{ + tryLogCurrentException(__PRETTY_FUNCTION__); +} + void BackgroundJobsExecutor::dataProcessingTask() try { @@ -28,10 +44,21 @@ catch (...) tryLogCurrentException(__PRETTY_FUNCTION__); } +void BackgroundJobsExecutor::startMovingTaskIfNeeded() +{ + if (data.areBackgroundMovesNeeded() && !data_moving_task) + { + data_moving_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); + data_moving_task->activateAndSchedule(); + } +} + void BackgroundJobsExecutor::start() { if (data_processing_task) data_processing_task->activateAndSchedule(); + startMovingTaskIfNeeded(); } void BackgroundJobsExecutor::triggerDataProcessing() @@ -44,6 +71,11 @@ void BackgroundJobsExecutor::finish() { data_processing_task->deactivate(); data_processing_pool.wait(); + if (data_moving_task) + { + data_moving_task->deactivate(); + move_pool.wait(); + } } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index aa166eb4d73..0945c4e0b59 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -12,19 +12,22 @@ class BackgroundJobsExecutor { private: MergeTreeData & data; + Context & global_context; ThreadPool data_processing_pool; ThreadPool move_pool; BackgroundSchedulePool::TaskHolder data_processing_task; - BackgroundSchedulePool::TaskHolder move_processing_task; + BackgroundSchedulePool::TaskHolder data_moving_task; void dataProcessingTask(); + void dataMovingTask(); public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); + void startMovingTaskIfNeeded(); void triggerDataProcessing(); void triggerMovesProcessing(); void start(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 40f12428561..03817c70ac0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -943,7 +943,7 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const Storag volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); } -void IMergeTreeDataPart::makeCloneOnDiskDetached(const ReservationPtr & reservation) const +void IMergeTreeDataPart::makeCloneOnDiskDetached(const std::shared_ptr & reservation) const { assertOnDisk(); auto reserved_disk = reservation->getDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 78daf6c9017..89136eaba4e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -317,7 +317,7 @@ public: virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; /// Makes full clone of part in detached/ on another disk - void makeCloneOnDiskDetached(const ReservationPtr & reservation) const; + void makeCloneOnDiskDetached(const std::shared_ptr & reservation) const; /// Checks that .bin and .mrk files exist. /// diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a0c1de756be..4b53ecba3c4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -71,6 +71,7 @@ namespace ProfileEvents namespace CurrentMetrics { extern const Metric DelayedInserts; + extern const Metric BackgroundMovePoolTask; } @@ -3614,12 +3615,31 @@ bool MergeTreeData::selectPartsAndMove() return false; auto moving_tagger = selectPartsForMove(); - if (moving_tagger.parts_to_move.empty()) + if (moving_tagger->parts_to_move.empty()) return false; return moveParts(std::move(moving_tagger)); } +std::optional MergeTreeData::getDataMovingJob() +{ + if (parts_mover.moves_blocker.isCancelled()) + return {}; + + auto moving_tagger = selectPartsForMove(); + if (moving_tagger->parts_to_move.empty()) + return {}; + + auto job = [this, moving_tagger{std::move(moving_tagger)}] () mutable + { + moveParts(moving_tagger); + }; + + MergeTreeBackgroundJob result_job(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); + + return std::make_optional(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); +} + bool MergeTreeData::areBackgroundMovesNeeded() const { auto policy = getStoragePolicy(); @@ -3636,13 +3656,13 @@ bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr spa return false; auto moving_tagger = checkPartsForMove(parts, space); - if (moving_tagger.parts_to_move.empty()) + if (moving_tagger->parts_to_move.empty()) return false; - return moveParts(std::move(moving_tagger)); + return moveParts(moving_tagger); } -MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::selectPartsForMove() +MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::selectPartsForMove() { MergeTreeMovingParts parts_to_move; @@ -3665,10 +3685,10 @@ MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::selectPartsForMove() std::lock_guard moving_lock(moving_parts_mutex); parts_mover.selectPartsForMove(parts_to_move, can_move, moving_lock); - return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this); + return std::make_shared(std::move(parts_to_move), *this); } -MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::checkPartsForMove(const DataPartsVector & parts, SpacePtr space) +MergeTreeData::CurrentlyMovingPartsTaggerPtr MergeTreeData::checkPartsForMove(const DataPartsVector & parts, SpacePtr space) { std::lock_guard moving_lock(moving_parts_mutex); @@ -3693,14 +3713,14 @@ MergeTreeData::CurrentlyMovingPartsTagger MergeTreeData::checkPartsForMove(const parts_to_move.emplace_back(part, std::move(reservation)); } - return CurrentlyMovingPartsTagger(std::move(parts_to_move), *this); + return std::make_shared(std::move(parts_to_move), *this); } -bool MergeTreeData::moveParts(CurrentlyMovingPartsTagger && moving_tagger) +bool MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger) { - LOG_INFO(log, "Got {} parts to move.", moving_tagger.parts_to_move.size()); + LOG_INFO(log, "Got {} parts to move.", moving_tagger->parts_to_move.size()); - for (const auto & moving_part : moving_tagger.parts_to_move) + for (const auto & moving_part : moving_tagger->parts_to_move) { Stopwatch stopwatch; DataPartPtr cloned_part; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8c5333315fc..1ebe21e98af 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -711,7 +711,9 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual std::optional getDataProcessingJob() { return {}; } + virtual std::optional getDataProcessingJob() = 0; + std::optional getDataMovingJob(); + bool areBackgroundMovesNeeded() const; protected: @@ -889,7 +891,6 @@ protected: /// Selects parts for move and moves them, used in background process bool selectPartsAndMove(); - bool areBackgroundMovesNeeded() const; private: /// RAII Wrapper for atomic work with currently moving parts @@ -901,18 +902,19 @@ private: MergeTreeData & data; CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); - CurrentlyMovingPartsTagger(const CurrentlyMovingPartsTagger & other) = delete; ~CurrentlyMovingPartsTagger(); }; + using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; + /// Move selected parts to corresponding disks - bool moveParts(CurrentlyMovingPartsTagger && moving_tagger); + bool moveParts(const CurrentlyMovingPartsTaggerPtr & moving_tagger); /// Select parts for move and disks for them. Used in background moving processes. - CurrentlyMovingPartsTagger selectPartsForMove(); + CurrentlyMovingPartsTaggerPtr selectPartsForMove(); /// Check selected parts for movements. Used by ALTER ... MOVE queries. - CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, SpacePtr space); + CurrentlyMovingPartsTaggerPtr checkPartsForMove(const DataPartsVector & parts, SpacePtr space); bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 942bac0d294..9a0db253abf 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -154,6 +154,7 @@ MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, si { } + UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); @@ -166,6 +167,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_siz { if (pool_used > pool_size) { + std::cerr << "POOLSIZE:" << pool_size << " POOL USED:" << pool_used << std::endl; throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index a1afadec7fa..332a0988d10 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -16,7 +16,7 @@ namespace DB struct MergeTreeMoveEntry { std::shared_ptr part; - ReservationPtr reserved_space; + std::shared_ptr reserved_space; MergeTreeMoveEntry(const std::shared_ptr & part_, ReservationPtr reservation_) : part(part_), reserved_space(std::move(reservation_)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 48caf59e7ba..b7fd7097546 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -584,8 +584,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper LOG_DEBUG(log, "Pulled {} entries to queue.", copied_entries.size()); } - if (storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + storage.background_executor.triggerDataProcessing(); } return stat.version; @@ -668,8 +667,8 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C } } - if (some_active_mutations_were_killed && storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + if (some_active_mutations_were_killed) + storage.background_executor.triggerDataProcessing(); if (!entries_to_load.empty()) { @@ -792,8 +791,8 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name); } - if (mutation_was_active && storage.queue_task_handle) - storage.queue_task_handle->signalReadyToRun(); + if (mutation_was_active) + storage.background_executor.triggerDataProcessing(); return entry; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 00ddb7a57ce..cfdd8a61e9c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -106,7 +106,6 @@ void StorageMergeTree::startup() try { background_executor.start(); - startBackgroundMovesIfNeeded(); } catch (...) { @@ -145,9 +144,6 @@ void StorageMergeTree::shutdown() background_executor.finish(); - if (moving_task_handle) - global_context.getBackgroundMovePool().removeTask(moving_task_handle); - try { /// We clear all old parts after stopping all background operations. @@ -501,18 +497,6 @@ std::optional StorageMergeTree::getIncompleteMutationsS return result; } - -void StorageMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded() && !moving_task_handle) - { - auto & move_pool = global_context.getBackgroundMovePool(); - moving_task_handle = move_pool.createTask([this] { return movePartsTask(); }); - move_pool.startTask(moving_task_handle); - } -} - - std::vector StorageMergeTree::getMutationsStatus() const { std::lock_guard lock(currently_processing_in_background_mutex); @@ -1530,4 +1514,9 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da return it->second.commands; } +void StorageMergeTree::startBackgroundMovesIfNeeded() +{ + background_executor.startMovingTaskIfNeeded(); +} + } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c028e15416f..957b7ce56a6 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -98,6 +98,7 @@ private: MergeTreeDataSelectExecutor reader; MergeTreeDataWriter writer; MergeTreeDataMergerMutator merger_mutator; + BackgroundJobsExecutor background_executor; /// For block numbers. SimpleIncrement increment{0}; @@ -120,10 +121,6 @@ private: std::atomic shutdown_called {false}; - /// Task handler for merges, mutations and moves. - BackgroundJobsExecutor background_executor; - BackgroundProcessingPool::TaskHandle moving_task_handle; - void loadMutations(); /** Determines what parts should be merged and merges it. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ae7ad4a3518..c61f2425b17 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -197,6 +197,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , queue(*this) , fetcher(*this) + , background_executor(*this, global_context) , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) @@ -3530,12 +3531,9 @@ void StorageReplicatedMergeTree::startup() /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. { auto lock = queue.lockQueue(); - auto & pool = global_context.getBackgroundPool(); - queue_task_handle = pool.createTask([this] { return queueTask(); }); - pool.startTask(queue_task_handle); + background_executor.start(); } - startBackgroundMovesIfNeeded(); } catch (...) { @@ -3566,14 +3564,11 @@ void StorageReplicatedMergeTree::shutdown() restarting_thread.shutdown(); - if (queue_task_handle) - global_context.getBackgroundPool().removeTask(queue_task_handle); - { /// Queue can trigger queue_task_handle itself. So we ensure that all /// queue processes finished and after that reset queue_task_handle. auto lock = queue.lockQueue(); - queue_task_handle.reset(); + background_executor.finish(); /// Cancel logs pulling after background task were cancelled. It's still /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, @@ -5921,12 +5916,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI { auto lock = queue.lockQueue(); - if (!queue_task_handle) - return false; - + background_executor.triggerDataProcessing(); /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. - queue_task_handle->signalReadyToRun(); } Poco::Event target_size_event; @@ -6032,15 +6024,9 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar return queue.getFirstAlterMutationCommandsForPart(part); } - void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - if (areBackgroundMovesNeeded() && !move_parts_task_handle) - { - auto & pool = global_context.getBackgroundMovePool(); - move_parts_task_handle = pool.createTask([this] { return movePartsTask(); }); - pool.startTask(move_parts_task_handle); - } + background_executor.startMovingTaskIfNeeded(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1b65ffdbc25..1a0cabae5fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB @@ -275,15 +276,14 @@ private: int metadata_version = 0; /// Threads. + BackgroundJobsExecutor background_executor; + /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. bool queue_update_in_progress = false; BackgroundSchedulePool::TaskHolder queue_updating_task; BackgroundSchedulePool::TaskHolder mutations_updating_task; - /// A task that performs actions from the queue. - BackgroundProcessingPool::TaskHandle queue_task_handle; - /// A task which move parts to another disks/volumes /// Transparent for replication. BackgroundProcessingPool::TaskHandle move_parts_task_handle; @@ -568,7 +568,6 @@ private: MutationCommands getFirtsAlterMutationCommandsForPart(const DataPartPtr & part) const override; void startBackgroundMovesIfNeeded() override; - protected: /** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table. */ From 4014e0f08d0f2a12a06601264b57d2f5d3af4b84 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 15:32:35 +0300 Subject: [PATCH 008/149] Something very similar to working code --- .../MergeTree/BackgroundJobsExecutor.cpp | 100 +++++++++++++++--- .../MergeTree/BackgroundJobsExecutor.h | 7 +- .../MergeTree/MergeTreeBackgroundJob.h | 15 +-- src/Storages/MergeTree/MergeTreeData.cpp | 8 +- src/Storages/MergeTree/MergeTreeData.h | 5 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 60 +---------- src/Storages/StorageMergeTree.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 39 +------ src/Storages/StorageReplicatedMergeTree.h | 7 +- 10 files changed, 112 insertions(+), 137 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 081ef818dcf..f4ef89e20f9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -1,5 +1,14 @@ #include #include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} namespace DB { @@ -9,11 +18,21 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( Context & global_context_) : data(data_) , global_context(global_context_) - , data_processing_pool(global_context.getSettingsRef().background_pool_size, 0, 10000, false) - , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, 10000, false) + , max_pool_size(global_context.getSettingsRef().background_pool_size) + , data_processing_pool(max_pool_size, 0, max_pool_size, false) + , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) + , rng(randomSeed()) { data_processing_task = global_context.getSchedulePool().createTask( data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); + const auto & config = global_context.getConfigRef(); + settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); + settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); + settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); } void BackgroundJobsExecutor::dataMovingTask() @@ -21,7 +40,7 @@ try { auto job = data.getDataMovingJob(); if (job) - move_pool.scheduleOrThrowOnError(*job); + move_pool.scheduleOrThrowOnError(job); data_moving_task->schedule(); } @@ -30,20 +49,77 @@ catch(...) tryLogCurrentException(__PRETTY_FUNCTION__); } -void BackgroundJobsExecutor::dataProcessingTask() -try +namespace { - auto job = data.getDataProcessingJob(); - if (job) - data_processing_pool.scheduleOrThrowOnError(*job); - data_processing_task->schedule(); -} -catch (...) +bool incrementIfLess(std::atomic & atomic_value, long max_value) { - tryLogCurrentException(__PRETTY_FUNCTION__); + auto value = atomic_value.load(std::memory_order_relaxed); + while (value < max_value) + if(atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) + return true; + return false; } +} + +void BackgroundJobsExecutor::dataProcessingTask() +{ + if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) + { + try + { + auto job = data.getDataProcessingJob(); + if (job) + { + data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () + { + try + { + job(); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + errors_count = 0; + } + catch (...) + { + errors_count++; + tryLogCurrentException(__PRETTY_FUNCTION__); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + } + }); + auto errors = errors_count.load(std::memory_order_relaxed); + if (errors != 0) + { + auto next_time_to_execute = 1000 * (std::min( + settings.task_sleep_seconds_when_no_work_max, + settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) + + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); + data_processing_task->scheduleAfter(next_time_to_execute); + } + else + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); + } + else + { + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + } + } + catch(...) + { + CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + else + { + /// Pool overloaded + data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + } +} + + void BackgroundJobsExecutor::startMovingTaskIfNeeded() { if (data.areBackgroundMovesNeeded() && !data_moving_task) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 0945c4e0b59..bf1fd0c5a03 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include namespace DB { @@ -13,8 +14,12 @@ class BackgroundJobsExecutor private: MergeTreeData & data; Context & global_context; + size_t max_pool_size; ThreadPool data_processing_pool; ThreadPool move_pool; + std::atomic errors_count{0}; + pcg64 rng; + BackgroundProcessingPool::PoolSettings settings; BackgroundSchedulePool::TaskHolder data_processing_task; BackgroundSchedulePool::TaskHolder data_moving_task; diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h index 2f432114db1..0b1a6ff3a8d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundJob.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundJob.h @@ -17,25 +17,16 @@ enum PoolType struct MergeTreeBackgroundJob { ThreadPool::Job job; - CurrentMetrics::Metric metric; PoolType execute_in_pool; - MergeTreeBackgroundJob(ThreadPool::Job && job_, CurrentMetrics::Metric metric_, PoolType execute_in_pool_) - : job(std::move(job_)), metric(metric_), execute_in_pool(execute_in_pool_) + MergeTreeBackgroundJob(ThreadPool::Job && job_, PoolType execute_in_pool_) + : job(std::move(job_)), execute_in_pool(execute_in_pool_) {} void operator()() try { - if (metric != 0) - { - CurrentMetrics::Increment metric_increment{metric}; - job(); - } - else - { - job(); - } + job(); } catch (...) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4b53ecba3c4..82c01acbcf4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3621,7 +3621,7 @@ bool MergeTreeData::selectPartsAndMove() return moveParts(std::move(moving_tagger)); } -std::optional MergeTreeData::getDataMovingJob() +ThreadPool::Job MergeTreeData::getDataMovingJob() { if (parts_mover.moves_blocker.isCancelled()) return {}; @@ -3630,14 +3630,10 @@ std::optional MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - auto job = [this, moving_tagger{std::move(moving_tagger)}] () mutable + return [this, moving_tagger{std::move(moving_tagger)}] () mutable { moveParts(moving_tagger); }; - - MergeTreeBackgroundJob result_job(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); - - return std::make_optional(std::move(job), CurrentMetrics::BackgroundMovePoolTask, PoolType::MOVE); } bool MergeTreeData::areBackgroundMovesNeeded() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1ebe21e98af..ed0e7e33d8b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -711,8 +710,8 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual std::optional getDataProcessingJob() = 0; - std::optional getDataMovingJob(); + virtual ThreadPool::Job getDataProcessingJob() = 0; + ThreadPool::Job getDataMovingJob(); bool areBackgroundMovesNeeded() const; protected: diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9a0db253abf..6aa3d1f3238 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -159,15 +159,15 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const { size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); - return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool == 0 ? 0 : busy_threads_in_pool - 1); /// 1 is current thread + return getMaxSourcePartsSizeForMerge(background_pool_size, busy_threads_in_pool); } UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { + //LOG_DEBUG(&Poco::Logger::get("DEBUG"), "POOL SIZE {}, POOL USED {}", pool_size, pool_used); if (pool_used > pool_size) { - std::cerr << "POOLSIZE:" << pool_size << " POOL USED:" << pool_used << std::endl; throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cfdd8a61e9c..5a1ce42d831 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -210,8 +210,9 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & lock_holder) { + lock_holder.release(); { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. @@ -902,7 +903,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } -std::optional StorageMergeTree::getDataProcessingJob() +ThreadPool::Job StorageMergeTree::getDataProcessingJob() { if (shutdown_called) return {}; @@ -919,18 +920,17 @@ std::optional StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - auto job = [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry); else if (mutate_entry) mutateSelectedPart(metadata_snapshot, *mutate_entry); }; - return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - auto job = [this] () + return [this] () { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -942,60 +942,10 @@ std::optional StorageMergeTree::getDataProcessingJob() } clearOldMutations(); }; - - return std::make_optional(std::move(job), 0, PoolType::MERGE_MUTATE); } return {}; } -BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() -{ - if (shutdown_called) - return BackgroundProcessingPoolTaskResult::ERROR; - - if (merger_mutator.merges_blocker.isCancelled()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - try - { - /// Clear old parts. It is unnecessary to do it more than once a second. - if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) - { - { - auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - /// All use relative_data_path which changes during rename - /// so execute under share lock. - clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(); - clearOldWriteAheadLogs(); - } - clearOldMutations(); - } - - auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); - ///TODO: read deduplicate option from table config - if (merge_entry && mergeSelectedParts(metadata_snapshot, false, *merge_entry)) - return BackgroundProcessingPoolTaskResult::SUCCESS; - - auto mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); - if (mutate_entry && mutateSelectedPart(metadata_snapshot, *mutate_entry)) - return BackgroundProcessingPoolTaskResult::SUCCESS; - - return BackgroundProcessingPoolTaskResult::ERROR; - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::ABORTED) - { - LOG_INFO(log, e.message()); - return BackgroundProcessingPoolTaskResult::ERROR; - } - - throw; - } -} - Int64 StorageMergeTree::getCurrentMutationVersion( const DataPartPtr & part, std::unique_lock & /* currently_processing_in_background_mutex_lock */) const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 957b7ce56a6..0c2b633fa70 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -88,7 +88,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; - std::optional getDataProcessingJob() override; + ThreadPool::Job getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait @@ -156,8 +156,6 @@ private: std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); - BackgroundProcessingPoolTaskResult mergeMutateTask(); - Int64 getCurrentMutationVersion( const DataPartPtr & part, std::unique_lock & /* currently_processing_in_background_mutex_lock */) const; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c61f2425b17..1d1b72018e6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2601,7 +2601,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel } -std::optional StorageReplicatedMergeTree::getDataProcessingJob() +ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) @@ -2615,43 +2615,12 @@ std::optional StorageReplicatedMergeTree::getDataProcess if (!entry) return {}; - auto job = [this, selected_entry{std::move(selected_entry)}] () mutable + return [this, selected_entry{std::move(selected_entry)}] () mutable { processQueueEntry(selected_entry); }; - - return std::make_optional(std::move(job), CurrentMetrics::BackgroundPoolTask, PoolType::MERGE_MUTATE); } -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::queueTask() -{ - /// If replication queue is stopped exit immediately as we successfully executed the task - if (queue.actions_blocker.isCancelled()) - { - std::this_thread::sleep_for(std::chrono::milliseconds(5)); - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - - /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); - - LogEntryPtr & entry = selected_entry.first; - - if (!entry) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - time_t prev_attempt_time = entry->last_attempt_time; - - bool res = processQueueEntry(selected_entry); - - /// We will go to sleep if the processing fails and if we have already processed this record recently. - bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10); - - /// If there was no exception, you do not need to sleep. - return need_sleep ? BackgroundProcessingPoolTaskResult::ERROR : BackgroundProcessingPoolTaskResult::SUCCESS; -} - - bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const { return queue.isVirtualPart(part); @@ -3576,10 +3545,6 @@ void StorageReplicatedMergeTree::shutdown() queue.pull_log_blocker.cancelForever(); } - if (move_parts_task_handle) - global_context.getBackgroundMovePool().removeTask(move_parts_task_handle); - move_parts_task_handle.reset(); - if (data_parts_exchange_endpoint) { global_context.getInterserverIOHandler().removeEndpointIfExists(data_parts_exchange_endpoint->getId(replica_path)); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1a0cabae5fa..6c30d966afd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -196,7 +196,7 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); - std::optional getDataProcessingJob() override; + ThreadPool::Job getDataProcessingJob() override; private: @@ -423,15 +423,10 @@ private: bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); - /** Performs actions from the queue. - */ - BackgroundProcessingPoolTaskResult queueTask(); - /// Perform moves of parts to another disks. /// Local operation, doesn't interact with replicationg queue. BackgroundProcessingPoolTaskResult movePartsTask(); - /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) /// or an exception is thrown and leader_election is destroyed. From 0b14a31ba9068c6bf270ef285b4f2b3b9816bca5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 15:44:10 +0300 Subject: [PATCH 009/149] Get rid of background processing pool --- src/Interpreters/Context.cpp | 44 ------------------- src/Interpreters/Context.h | 3 -- .../MergeTree/BackgroundJobsExecutor.h | 31 ++++++++++++- src/Storages/StorageMergeTree.cpp | 17 ------- src/Storages/StorageMergeTree.h | 3 -- src/Storages/StorageReplicatedMergeTree.cpp | 17 ------- src/Storages/StorageReplicatedMergeTree.h | 9 ---- 7 files changed, 29 insertions(+), 95 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e7614a66761..e98a8d59155 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -331,8 +330,6 @@ struct ContextShared ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. std::optional buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. - std::optional background_pool; /// The thread pool for the background work performed by the tables. - std::optional background_move_pool; /// The thread pool for the background moves performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) std::optional distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends) std::optional message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used in kafka streaming) @@ -433,8 +430,6 @@ struct ContextShared external_dictionaries_loader.reset(); external_models_loader.reset(); buffer_flush_schedule_pool.reset(); - background_pool.reset(); - background_move_pool.reset(); schedule_pool.reset(); distributed_schedule_pool.reset(); ddl_worker.reset(); @@ -1369,45 +1364,6 @@ void Context::dropCaches() const shared->mark_cache->reset(); } -BackgroundProcessingPool & Context::getBackgroundPool() -{ - auto lock = getLock(); - if (!shared->background_pool) - { - BackgroundProcessingPool::PoolSettings pool_settings; - const auto & config = getConfigRef(); - pool_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); - pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); - shared->background_pool.emplace(settings.background_pool_size, pool_settings); - } - return *shared->background_pool; -} - -BackgroundProcessingPool & Context::getBackgroundMovePool() -{ - auto lock = getLock(); - if (!shared->background_move_pool) - { - BackgroundProcessingPool::PoolSettings pool_settings; - const auto & config = getConfigRef(); - pool_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); - pool_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); - pool_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - pool_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - pool_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - pool_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - pool_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); - pool_settings.tasks_metric = CurrentMetrics::BackgroundMovePoolTask; - shared->background_move_pool.emplace(settings.background_move_pool_size, pool_settings, "BackgroundMovePool", "BgMoveProcPool"); - } - return *shared->background_move_pool; -} - BackgroundSchedulePool & Context::getBufferFlushSchedulePool() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index bd5e17fe2e4..155d8fbcd73 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -62,7 +62,6 @@ class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalModelsLoader; class InterserverIOHandler; -class BackgroundProcessingPool; class BackgroundSchedulePool; class MergeList; class Cluster; @@ -508,8 +507,6 @@ public: void dropCaches() const; BackgroundSchedulePool & getBufferFlushSchedulePool(); - BackgroundProcessingPool & getBackgroundPool(); - BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); BackgroundSchedulePool & getDistributedSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index bf1fd0c5a03..b7bd63f7169 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -3,11 +3,38 @@ #include #include #include -#include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} + namespace DB { +enum PoolType +{ + MERGE_MUTATING, + MOVING, + +} + +struct PoolSettings +{ + double thread_sleep_seconds = 10; + double thread_sleep_seconds_random_part = 1.0; + double thread_sleep_seconds_if_nothing_to_do = 0.1; + + /// For exponential backoff. + double task_sleep_seconds_when_no_work_min = 10; + double task_sleep_seconds_when_no_work_max = 600; + double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_random_part = 1.0; + + CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; + + PoolSettings() noexcept {} +}; class BackgroundJobsExecutor { @@ -19,7 +46,7 @@ private: ThreadPool move_pool; std::atomic errors_count{0}; pcg64 rng; - BackgroundProcessingPool::PoolSettings settings; + PoolSettings settings; BackgroundSchedulePool::TaskHolder data_processing_task; BackgroundSchedulePool::TaskHolder data_moving_task; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5a1ce42d831..87dfbd4d879 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -762,23 +762,6 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } - -BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask() -{ - try - { - if (!selectPartsAndMove()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - catch (...) - { - tryLogCurrentException(log); - return BackgroundProcessingPoolTaskResult::ERROR; - } -} - std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 0c2b633fa70..b1946e48d4f 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -131,8 +130,6 @@ private: ActionLock stopMergesAndWait(); - BackgroundProcessingPoolTaskResult movePartsTask(); - /// Allocate block number for new mutation, write mutation to disk /// and into in-memory structures. Wake up merge-mutation task. Int64 startMutation(const MutationCommands & commands, String & mutation_file_name); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1d1b72018e6..453956ffa8b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2626,23 +2626,6 @@ bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataP return queue.isVirtualPart(part); } -BackgroundProcessingPoolTaskResult StorageReplicatedMergeTree::movePartsTask() -{ - try - { - if (!selectPartsAndMove()) - return BackgroundProcessingPoolTaskResult::NOTHING_TO_DO; - - return BackgroundProcessingPoolTaskResult::SUCCESS; - } - catch (...) - { - tryLogCurrentException(log); - return BackgroundProcessingPoolTaskResult::ERROR; - } -} - - void StorageReplicatedMergeTree::mergeSelectingTask() { if (!is_leader) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 6c30d966afd..4ce3c0ad3c1 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include @@ -284,10 +283,6 @@ private: BackgroundSchedulePool::TaskHolder mutations_updating_task; - /// A task which move parts to another disks/volumes - /// Transparent for replication. - BackgroundProcessingPool::TaskHandle move_parts_task_handle; - /// A task that selects parts to merge. BackgroundSchedulePool::TaskHolder merge_selecting_task; /// It is acquired for each iteration of the selection of parts to merge or each OPTIMIZE query. @@ -423,10 +418,6 @@ private: bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); - /// Perform moves of parts to another disks. - /// Local operation, doesn't interact with replicationg queue. - BackgroundProcessingPoolTaskResult movePartsTask(); - /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) /// or an exception is thrown and leader_election is destroyed. From 483893cdd40df6e1a47ace63842cd8d2f1d4491d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 17:56:42 +0300 Subject: [PATCH 010/149] Slightly refactored code --- .../MergeTree/BackgroundJobsExecutor.cpp | 344 +++++++++++++----- .../MergeTree/BackgroundJobsExecutor.h | 99 +++-- src/Storages/StorageMergeTree.cpp | 7 +- src/Storages/StorageMergeTree.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.h | 1 + 6 files changed, 336 insertions(+), 122 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f4ef89e20f9..16bb5b3c4f0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -5,48 +5,31 @@ #include #include -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; -} - namespace DB { -BackgroundJobsExecutor::BackgroundJobsExecutor( - MergeTreeData & data_, - Context & global_context_) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +IBackgroundJobExecutor::IBackgroundJobExecutor( + MergeTreeData & data_, + Context & global_context_, + const String & task_name_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_) : data(data_) , global_context(global_context_) - , max_pool_size(global_context.getSettingsRef().background_pool_size) - , data_processing_pool(max_pool_size, 0, max_pool_size, false) - , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) + , task_name(task_name_) + , sleep_settings(sleep_settings_) , rng(randomSeed()) { - data_processing_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); - const auto & config = global_context.getConfigRef(); - settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); - settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); - settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); - settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); -} - -void BackgroundJobsExecutor::dataMovingTask() -try -{ - auto job = data.getDataMovingJob(); - if (job) - move_pool.scheduleOrThrowOnError(job); - - data_moving_task->schedule(); -} -catch(...) -{ - tryLogCurrentException(__PRETTY_FUNCTION__); + for (const auto & pool_config : pools_configs_) + { + pools.try_emplace(pool_config.pool_type, pool_config.max_pool_size, 0, pool_config.max_pool_size, false); + pools_configs.emplace(pool_config.pool_type, pool_config); + } } namespace @@ -63,95 +46,270 @@ bool incrementIfLess(std::atomic & atomic_value, long max_value) } -void BackgroundJobsExecutor::dataProcessingTask() + +void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) { - if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) + auto errors = errors_count.load(std::memory_order_relaxed); + size_t next_time_to_execute = 0; + if (errors != 0) + next_time_to_execute += 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + else if (nothing_to_do) + next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + else + next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + + scheduling_task->scheduleAfter(next_time_to_execute); +} + +void IBackgroundJobExecutor::jobExecutingTask() +try +{ + auto job_and_pool = getBackgroundJob(); + if (job_and_pool) { - try + auto & pool_config = pools_configs[job_and_pool->pool_type]; + /// If corresponding pool is not full, otherwise try next time + if (incrementIfLess(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) { - auto job = data.getDataProcessingJob(); - if (job) + try /// this try required because we have to manually decrement metric { - data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () + pools[job_and_pool->pool_type].scheduleOrThrowOnError([this, pool_config, job{std::move(job_and_pool->job)}] () { - try + try /// We don't want exceptions in background pool { job(); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + CurrentMetrics::values[pool_config.tasks_metric]--; errors_count = 0; } catch (...) { errors_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + CurrentMetrics::values[pool_config.tasks_metric]--; } }); - auto errors = errors_count.load(std::memory_order_relaxed); - if (errors != 0) - { - auto next_time_to_execute = 1000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - data_processing_task->scheduleAfter(next_time_to_execute); - } - else - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); } - else + catch (...) { - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; + tryLogCurrentException(__PRETTY_FUNCTION__); + CurrentMetrics::values[pool_config.tasks_metric]--; } } - catch(...) - { - CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); - tryLogCurrentException(__PRETTY_FUNCTION__); - } + scheduleTask(false); } - else + else /// Nothing to do, no jobs { - /// Pool overloaded - data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); + scheduleTask(true); } } - - -void BackgroundJobsExecutor::startMovingTaskIfNeeded() +catch (...) /// Exception while we looking for task { - if (data.areBackgroundMovesNeeded() && !data_moving_task) + tryLogCurrentException(__PRETTY_FUNCTION__); + scheduleTask(true); +} + +void IBackgroundJobExecutor::start() +{ + if (!scheduling_task) { - data_moving_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); - data_moving_task->activateAndSchedule(); + scheduling_task = global_context.getSchedulePool().createTask( + data.getStorageID().getFullTableName() + task_name, [this]{ jobExecutingTask(); }); } + + scheduling_task->activateAndSchedule(); } -void BackgroundJobsExecutor::start() +void IBackgroundJobExecutor::finish() { - if (data_processing_task) - data_processing_task->activateAndSchedule(); - startMovingTaskIfNeeded(); -} - -void BackgroundJobsExecutor::triggerDataProcessing() -{ - if (data_processing_task) - data_processing_task->schedule(); -} - -void BackgroundJobsExecutor::finish() -{ - data_processing_task->deactivate(); - data_processing_pool.wait(); - if (data_moving_task) + if (scheduling_task) { - data_moving_task->deactivate(); - move_pool.wait(); + scheduling_task->deactivate(); + for (auto & [pool_type, pool] : pools) + pool.wait(); } } +void IBackgroundJobExecutor::triggerDataProcessing() +{ + if (scheduling_task) + scheduling_task->schedule(); +} + + IBackgroundJobExecutor::~IBackgroundJobExecutor() +{ + finish(); +} + +BackgroundJobsExecutor::BackgroundJobsExecutor( + MergeTreeData & data_, + Context & global_context_) + : IBackgroundJobExecutor( + data_, + global_context_, + "(dataProcessingTask)", + TaskSleepSettings{}, + {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) +{ +} + +std::optional BackgroundJobsExecutor::getBackgroundJob() +{ + auto job = data.getDataProcessingJob(); + if (job) + return JobAndPool{job, PoolType::MERGE_MUTATE}; + return {}; +} + +BackgroundMovesExecutor::BackgroundMovesExecutor( + MergeTreeData & data_, + Context & global_context_) + : IBackgroundJobExecutor( + data_, + global_context_, + "(dataMovingTask)", + TaskSleepSettings{}, + {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) +{ +} + +std::optional BackgroundMovesExecutor::getBackgroundJob() +{ + auto job = data.getDataMovingJob(); + if (job) + return JobAndPool{job, PoolType::MOVE}; + return {}; +} + +//BackgroundJobsExecutor::BackgroundJobsExecutor( +// MergeTreeData & data_, +// Context & global_context_) +// : data(data_) +// , global_context(global_context_) +// , max_pool_size(global_context.getSettingsRef().background_pool_size) +// , data_processing_pool(max_pool_size, 0, max_pool_size, false) +// , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) +// , rng(randomSeed()) +//{ +// data_processing_task = global_context.getSchedulePool().createTask( +// data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); +// const auto & config = global_context.getConfigRef(); +// settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); +// settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); +// settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); +// settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); +// settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); +// settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); +// settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); +//} +// +//void BackgroundJobsExecutor::dataMovingTask() +//try +//{ +// auto job = data.getDataMovingJob(); +// if (job) +// move_pool.scheduleOrThrowOnError(job); +// +// data_moving_task->schedule(); +//} +//catch(...) +//{ +// tryLogCurrentException(__PRETTY_FUNCTION__); +//} +// +// +// +//void BackgroundJobsExecutor::dataProcessingTask() +//{ +// if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) +// { +// try +// { +// auto job = data.getDataProcessingJob(); +// if (job) +// { +// data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () +// { +// try +// { +// job(); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// errors_count = 0; +// } +// catch (...) +// { +// errors_count++; +// tryLogCurrentException(__PRETTY_FUNCTION__); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// } +// }); +// auto errors = errors_count.load(std::memory_order_relaxed); +// if (errors != 0) +// { +// auto next_time_to_execute = 1000 * (std::min( +// settings.task_sleep_seconds_when_no_work_max, +// settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) +// + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); +// data_processing_task->scheduleAfter(next_time_to_execute); +// } +// else +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); +// } +// else +// { +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// } +// } +// catch(...) +// { +// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } +// } +// else +// { +// /// Pool overloaded +// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); +// } +//} +// +// +//void BackgroundJobsExecutor::startMovingTaskIfNeeded() +//{ +// if (data.areBackgroundMovesNeeded() && !data_moving_task) +// { +// data_moving_task = global_context.getSchedulePool().createTask( +// data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); +// data_moving_task->activateAndSchedule(); +// } +//} +// +//void BackgroundJobsExecutor::start() +//{ +// if (data_processing_task) +// data_processing_task->activateAndSchedule(); +// startMovingTaskIfNeeded(); +//} +// +//void BackgroundJobsExecutor::triggerDataProcessing() +//{ +// if (data_processing_task) +// data_processing_task->schedule(); +//} +// +//void BackgroundJobsExecutor::finish() +//{ +// data_processing_task->deactivate(); +// data_processing_pool.wait(); +// if (data_moving_task) +// { +// data_moving_task->deactivate(); +// move_pool.wait(); +// } +//} + } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index b7bd63f7169..4d01198f681 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -8,18 +8,14 @@ namespace CurrentMetrics { extern const Metric BackgroundPoolTask; + extern const Metric BackgroundMovePoolTask; } namespace DB { -enum PoolType -{ - MERGE_MUTATING, - MOVING, -} -struct PoolSettings +struct TaskSleepSettings { double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; @@ -29,41 +25,90 @@ struct PoolSettings double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_random_part = 1.0; - - CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; - - PoolSettings() noexcept {} }; -class BackgroundJobsExecutor +enum PoolType { -private: + MERGE_MUTATE, + FETCH, + MOVE, + LOW_PRIORITY, +}; + +struct PoolConfig +{ + PoolType pool_type; + size_t max_pool_size; + CurrentMetrics::Metric tasks_metric; +}; + +struct JobAndPool +{ + ThreadPool::Job job; + PoolType pool_type; +}; + +class IBackgroundJobExecutor +{ +protected: MergeTreeData & data; Context & global_context; - size_t max_pool_size; - ThreadPool data_processing_pool; - ThreadPool move_pool; - std::atomic errors_count{0}; +private: + String task_name; + TaskSleepSettings sleep_settings; pcg64 rng; - PoolSettings settings; - BackgroundSchedulePool::TaskHolder data_processing_task; - BackgroundSchedulePool::TaskHolder data_moving_task; + std::atomic errors_count{0}; - void dataProcessingTask(); - void dataMovingTask(); + std::unordered_map pools; + std::unordered_map pools_configs; + BackgroundSchedulePool::TaskHolder scheduling_task; + +public: + IBackgroundJobExecutor( + MergeTreeData & data_, + Context & global_context_, + const String & task_name_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_); + + void start(); + void triggerDataProcessing(); + void finish(); + + virtual ~IBackgroundJobExecutor(); + +protected: + virtual std::optional getBackgroundJob() = 0; +private: + void jobExecutingTask(); + void scheduleTask(bool nothing_to_do); +}; + + +class BackgroundJobsExecutor final : public IBackgroundJobExecutor +{ public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); - void startMovingTaskIfNeeded(); - void triggerDataProcessing(); - void triggerMovesProcessing(); - void start(); - void finish(); +protected: + std::optional getBackgroundJob() override; }; - + +class BackgroundMovesExecutor final : public IBackgroundJobExecutor +{ +public: + BackgroundMovesExecutor( + MergeTreeData & data_, + Context & global_context_); + +protected: + std::optional getBackgroundJob() override; +}; + } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 87dfbd4d879..c4f8436678d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -79,6 +79,8 @@ StorageMergeTree::StorageMergeTree( , writer(*this) , merger_mutator(*this, global_context.getSettingsRef().background_pool_size) , background_executor(*this, global_context) + , background_moves_executor(*this, global_context) + { loadDataParts(has_force_restore_data_flag); @@ -106,6 +108,8 @@ void StorageMergeTree::startup() try { background_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } catch (...) { @@ -143,6 +147,7 @@ void StorageMergeTree::shutdown() parts_mover.moves_blocker.cancelForever(); background_executor.finish(); + background_moves_executor.finish(); try { @@ -1449,7 +1454,7 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_executor.startMovingTaskIfNeeded(); + background_executor.start(); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b1946e48d4f..f7473872189 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -98,6 +98,7 @@ private: MergeTreeDataWriter writer; MergeTreeDataMergerMutator merger_mutator; BackgroundJobsExecutor background_executor; + BackgroundMovesExecutor background_moves_executor; /// For block numbers. SimpleIncrement increment{0}; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 453956ffa8b..bc55b87fee7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -198,6 +198,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , queue(*this) , fetcher(*this) , background_executor(*this, global_context) + , background_moves_executor(*this, global_context) , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) @@ -3485,6 +3486,8 @@ void StorageReplicatedMergeTree::startup() auto lock = queue.lockQueue(); background_executor.start(); } + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } catch (...) @@ -3527,6 +3530,7 @@ void StorageReplicatedMergeTree::shutdown() /// MUTATE, etc. query. queue.pull_log_blocker.cancelForever(); } + background_moves_executor.finish(); if (data_parts_exchange_endpoint) { @@ -5974,7 +5978,7 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - background_executor.startMovingTaskIfNeeded(); + background_moves_executor.start(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 4ce3c0ad3c1..ba01ca9d3af 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -276,6 +276,7 @@ private: /// Threads. BackgroundJobsExecutor background_executor; + BackgroundMovesExecutor background_moves_executor; /// A task that keeps track of the updates in the logs of all replicas and loads them into the queue. bool queue_update_in_progress = false; From 6a4911dc576dfffa82043f2c43da1db20c20fe7f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 18:05:14 +0300 Subject: [PATCH 011/149] Remove some garbage --- .../MergeTree/BackgroundJobsExecutor.cpp | 129 ------------------ 1 file changed, 129 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 16bb5b3c4f0..c51028862e6 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -183,133 +183,4 @@ std::optional BackgroundMovesExecutor::getBackgroundJob() return {}; } -//BackgroundJobsExecutor::BackgroundJobsExecutor( -// MergeTreeData & data_, -// Context & global_context_) -// : data(data_) -// , global_context(global_context_) -// , max_pool_size(global_context.getSettingsRef().background_pool_size) -// , data_processing_pool(max_pool_size, 0, max_pool_size, false) -// , move_pool(global_context.getSettingsRef().background_move_pool_size, 0, max_pool_size, false) -// , rng(randomSeed()) -//{ -// data_processing_task = global_context.getSchedulePool().createTask( -// data.getStorageID().getFullTableName() + " (dataProcessingTask)", [this]{ dataProcessingTask(); }); -// const auto & config = global_context.getConfigRef(); -// settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); -// settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); -// settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); -// settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); -// settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); -// settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); -// settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); -//} -// -//void BackgroundJobsExecutor::dataMovingTask() -//try -//{ -// auto job = data.getDataMovingJob(); -// if (job) -// move_pool.scheduleOrThrowOnError(job); -// -// data_moving_task->schedule(); -//} -//catch(...) -//{ -// tryLogCurrentException(__PRETTY_FUNCTION__); -//} -// -// -// -//void BackgroundJobsExecutor::dataProcessingTask() -//{ -// if (incrementIfLess(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask], max_pool_size)) -// { -// try -// { -// auto job = data.getDataProcessingJob(); -// if (job) -// { -// data_processing_pool.scheduleOrThrowOnError([this, job{std::move(job)}] () -// { -// try -// { -// job(); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// errors_count = 0; -// } -// catch (...) -// { -// errors_count++; -// tryLogCurrentException(__PRETTY_FUNCTION__); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// } -// }); -// auto errors = errors_count.load(std::memory_order_relaxed); -// if (errors != 0) -// { -// auto next_time_to_execute = 1000 * (std::min( -// settings.task_sleep_seconds_when_no_work_max, -// settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, errors)) -// + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); -// data_processing_task->scheduleAfter(next_time_to_execute); -// } -// else -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); -// } -// else -// { -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// } -// } -// catch(...) -// { -// CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask]--; -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -// } -// else -// { -// /// Pool overloaded -// data_processing_task->scheduleAfter(1000 * (settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng))); -// } -//} -// -// -//void BackgroundJobsExecutor::startMovingTaskIfNeeded() -//{ -// if (data.areBackgroundMovesNeeded() && !data_moving_task) -// { -// data_moving_task = global_context.getSchedulePool().createTask( -// data.getStorageID().getFullTableName() + " (dataMovingTask)", [this]{ dataMovingTask(); }); -// data_moving_task->activateAndSchedule(); -// } -//} -// -//void BackgroundJobsExecutor::start() -//{ -// if (data_processing_task) -// data_processing_task->activateAndSchedule(); -// startMovingTaskIfNeeded(); -//} -// -//void BackgroundJobsExecutor::triggerDataProcessing() -//{ -// if (data_processing_task) -// data_processing_task->schedule(); -//} -// -//void BackgroundJobsExecutor::finish() -//{ -// data_processing_task->deactivate(); -// data_processing_pool.wait(); -// if (data_moving_task) -// { -// data_moving_task->deactivate(); -// move_pool.wait(); -// } -//} - } From fc25b6cd724f563943e026e314cf2495c196b56e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 18:26:00 +0300 Subject: [PATCH 012/149] Remove background processing pool --- .../MergeTree/BackgroundProcessingPool.cpp | 240 ------------------ .../MergeTree/BackgroundProcessingPool.h | 163 ------------ 2 files changed, 403 deletions(-) delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.cpp delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.h diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp deleted file mode 100644 index 3dd1dda0fbf..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ /dev/null @@ -1,240 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -void BackgroundProcessingPoolTaskInfo::signalReadyToRun() -{ - Poco::Timestamp current_time; - { - std::unique_lock lock(pool.tasks_mutex); - - /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. - if (removed) - return; - - /// If this task did nothing the previous time and still should sleep, then reschedule to cancel the sleep. - const auto & scheduled_time = iterator->first; - if (scheduled_time > current_time) - pool.rescheduleTask(iterator, current_time); - - /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. - pool.wake_event.notify_one(); - } -} - - -BackgroundProcessingPool::BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings, - const char * log_name, - const char * thread_name_) - : size(size_) - , thread_name(thread_name_) - , settings(pool_settings) -{ - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); - - threads.resize(size); - for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); -} - - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const Task & task) -{ - return std::make_shared(*this, task); -} - -void BackgroundProcessingPool::startTask(const TaskHandle & task, bool allow_execute_in_parallel) -{ - Poco::Timestamp current_time; - - task->allow_execute_in_parallel = allow_execute_in_parallel; - - { - std::unique_lock lock(tasks_mutex); - task->iterator = tasks.emplace(current_time, task); - - wake_event.notify_all(); - } - -} - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) -{ - TaskHandle res = createTask(task); - startTask(res); - return res; -} - -void BackgroundProcessingPool::removeTask(const TaskHandle & task) -{ - if (task->removed.exchange(true)) - return; - - /// Wait for all executions of this task. - { - std::unique_lock wlock(task->rwlock); - } - - { - std::unique_lock lock(tasks_mutex); - tasks.erase(task->iterator); - /// Note that the task may be still accessible through TaskHandle (shared_ptr). - } -} - -BackgroundProcessingPool::~BackgroundProcessingPool() -{ - try - { - { - std::lock_guard lock(tasks_mutex); - shutdown = true; - wake_event.notify_all(); - } - - for (auto & thread : threads) - thread.join(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -void BackgroundProcessingPool::workLoopFunc() -{ - setThreadName(thread_name); - - { - std::lock_guard lock(tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - } - - SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - - pcg64 rng(randomSeed()); - std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - - Poco::Timestamp scheduled_task_start_time; - - while (true) - { - TaskResult task_result = TaskResult::ERROR; - TaskHandle task; - - { - std::unique_lock lock(tasks_mutex); - - while (!task && !shutdown) - { - for (const auto & [time, handle] : tasks) - { - if (!handle->removed - && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) - { - task = handle; - scheduled_task_start_time = time; - ++task->concurrent_executors; - break; - } - } - - if (task) - { - Poco::Timestamp current_time; - - if (scheduled_task_start_time <= current_time) - continue; - - wake_event.wait_for(lock, - std::chrono::microseconds(scheduled_task_start_time - current_time - + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); - } - else - { - wake_event.wait_for(lock, - std::chrono::duration(settings.thread_sleep_seconds - + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - } - } - - if (shutdown) - break; - } - - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - try - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->task_function(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - { - std::unique_lock lock(tasks_mutex); - - if (shutdown) - break; - - --task->concurrent_executors; - - if (task->removed) - continue; - - if (task_result == TaskResult::SUCCESS) - task->count_no_work_done = 0; - else - ++task->count_no_work_done; - - /// If task has done work, it could be executed again immediately. - /// If not, add delay before next run. - - Poco::Timestamp next_time_to_execute; /// current time - if (task_result == TaskResult::ERROR) - next_time_to_execute += 1000000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - else if (task_result == TaskResult::NOTHING_TO_DO) - next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - - rescheduleTask(task->iterator, next_time_to_execute); - } - } -} - -} diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.h b/src/Storages/MergeTree/BackgroundProcessingPool.h deleted file mode 100644 index dfc83f9e550..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.h +++ /dev/null @@ -1,163 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; -} - -namespace DB -{ - -class BackgroundProcessingPool; -class BackgroundProcessingPoolTaskInfo; - -enum class BackgroundProcessingPoolTaskResult -{ - SUCCESS, - ERROR, - NOTHING_TO_DO, -}; - - -/** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop. - * In this case, one task can run simultaneously from different threads. - * Designed for tasks that perform continuous background work (for example, merge). - * `Task` is a function that returns a bool - did it do any work. - * If not, then the next time will be done later. - */ -class BackgroundProcessingPool -{ -public: - /// Returns true, if some useful work was done. In that case, thread will not sleep before next run of this task. - using TaskResult = BackgroundProcessingPoolTaskResult; - using Task = std::function; - using TaskInfo = BackgroundProcessingPoolTaskInfo; - using TaskHandle = std::shared_ptr; - - - struct PoolSettings - { - double thread_sleep_seconds = 10; - double thread_sleep_seconds_random_part = 1.0; - double thread_sleep_seconds_if_nothing_to_do = 0.1; - - /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; - double task_sleep_seconds_when_no_work_max = 600; - double task_sleep_seconds_when_no_work_multiplier = 1.1; - double task_sleep_seconds_when_no_work_random_part = 1.0; - - CurrentMetrics::Metric tasks_metric = CurrentMetrics::BackgroundPoolTask; - - PoolSettings() noexcept {} - }; - - BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings = {}, - const char * log_name = "BackgroundProcessingPool", - const char * thread_name_ = "BackgrProcPool"); - - size_t getNumberOfThreads() const - { - return size; - } - - /// Create task and start it. - TaskHandle addTask(const Task & task); - - /// The following two methods are invoked by Storage*MergeTree at startup - /// Create task but not start it. - TaskHandle createTask(const Task & task); - /// Start the task that was created but not started. Precondition: task was not started. - void startTask(const TaskHandle & task, bool allow_execute_in_parallel = true); - - /// Invoked by Storage*MergeTree at shutdown - void removeTask(const TaskHandle & task); - - ~BackgroundProcessingPool(); - -protected: - friend class BackgroundProcessingPoolTaskInfo; - - using Tasks = std::multimap; /// key is desired next time to execute (priority). - using Threads = std::vector; - - const size_t size; - const char * thread_name; - Poco::Logger * logger; - - Tasks tasks; /// Ordered in priority. - std::mutex tasks_mutex; - - Threads threads; - - bool shutdown{false}; - std::condition_variable wake_event; - - /// Thread group used for profiling purposes - ThreadGroupStatusPtr thread_group; - - void workLoopFunc(); - - void rescheduleTask(Tasks::iterator & task_it, const Poco::Timestamp & new_scheduled_ts) - { - auto node_handle = tasks.extract(task_it); - node_handle.key() = new_scheduled_ts; - task_it = tasks.insert(std::move(node_handle)); - } - -private: - PoolSettings settings; -}; - - -class BackgroundProcessingPoolTaskInfo -{ -public: - /// Signals random idle thread from the pool that this task is ready to be executed. - void signalReadyToRun(); - - BackgroundProcessingPoolTaskInfo(BackgroundProcessingPool & pool_, const BackgroundProcessingPool::Task & function_) - : pool(pool_), task_function(function_) {} - -protected: - friend class BackgroundProcessingPool; - - BackgroundProcessingPool & pool; - BackgroundProcessingPool::Task task_function; - - /// Read lock is held while task is being executed. - /// Write lock is used for stopping BGProcPool - std::shared_mutex rwlock; - - bool allow_execute_in_parallel = false; - size_t concurrent_executors = 0; - - /// Signals that this task must no longer be planned for execution and is about to be removed - std::atomic removed {false}; - - BackgroundProcessingPool::Tasks::iterator iterator; - - /// For exponential backoff. - size_t count_no_work_done = 0; -}; - -} From a535a089d1af493067c7ecb183bcb34968921722 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 19:09:18 +0300 Subject: [PATCH 013/149] Some style fixes --- src/Interpreters/Context.cpp | 32 +++++++++++++++++++ src/Interpreters/Context.h | 3 ++ .../MergeTree/BackgroundJobsExecutor.cpp | 20 ++++++++---- .../MergeTree/BackgroundJobsExecutor.h | 2 -- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/ya.make | 2 +- 6 files changed, 52 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e98a8d59155..ac48340c9c6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -61,6 +61,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -1375,6 +1376,37 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } +TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() +{ + TaskSleepSettings task_settings; + + const auto & config = getConfigRef(); + task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); + task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + return task_settings; +} + +TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() +{ + TaskSleepSettings task_settings; + + const auto & config = getConfigRef(); + task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); + task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); + task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); + + return task_settings; +} + BackgroundSchedulePool & Context::getSchedulePool() { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 155d8fbcd73..0f90c26a43c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -110,6 +110,7 @@ using OutputFormatPtr = std::shared_ptr; class IVolume; using VolumePtr = std::shared_ptr; struct NamedSession; +struct TaskSleepSettings; #if USE_EMBEDDED_COMPILER @@ -506,6 +507,8 @@ public: */ void dropCaches() const; + TaskSleepSettings getBackgroundProcessingTaskSleepSettings(); + TaskSleepSettings getBackgroundMoveTaskSleepSettings(); BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index c51028862e6..f952b04974b 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -39,27 +39,35 @@ bool incrementIfLess(std::atomic & atomic_value, long max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) - if(atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) + { + if (atomic_value.compare_exchange_weak(value, value + 1, std::memory_order_release, std::memory_order_relaxed)) return true; + } return false; } } - void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) { auto errors = errors_count.load(std::memory_order_relaxed); size_t next_time_to_execute = 0; if (errors != 0) + { next_time_to_execute += 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + } else if (nothing_to_do) - next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + { + next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + } else + { next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + } scheduling_task->scheduleAfter(next_time_to_execute); } @@ -138,7 +146,7 @@ void IBackgroundJobExecutor::triggerDataProcessing() scheduling_task->schedule(); } - IBackgroundJobExecutor::~IBackgroundJobExecutor() +IBackgroundJobExecutor::~IBackgroundJobExecutor() { finish(); } @@ -150,7 +158,7 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( data_, global_context_, "(dataProcessingTask)", - TaskSleepSettings{}, + global_context_.getBackgroundProcessingTaskSleepSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) { } @@ -170,7 +178,7 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( data_, global_context_, "(dataMovingTask)", - TaskSleepSettings{}, + global_context_.getBackgroundMoveTaskSleepSettings(), {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) { } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 4d01198f681..bc051702bfb 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -14,7 +14,6 @@ namespace CurrentMetrics namespace DB { - struct TaskSleepSettings { double thread_sleep_seconds = 10; @@ -88,7 +87,6 @@ private: void scheduleTask(bool nothing_to_do); }; - class BackgroundJobsExecutor final : public IBackgroundJobExecutor { public: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c4f8436678d..b103095eeb2 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -722,7 +722,8 @@ bool StorageMergeTree::merge( return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); } -bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) { +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) +{ auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 107433b5e73..e306787bb92 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -26,7 +26,7 @@ SRCS( LiveView/TemporaryLiveViewCleaner.cpp MergeTree/ActiveDataPartSet.cpp MergeTree/AllMergeSelector.cpp - MergeTree/BackgroundProcessingPool.cpp + MergeTree/BackgroundJobsExecutor.cpp MergeTree/BoolMask.cpp MergeTree/checkDataPart.cpp MergeTree/DataPartsExchange.cpp From 726579251047bda6ae2cbefdf1cd5d8f0492a344 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 21:33:41 +0300 Subject: [PATCH 014/149] Less files --- .../MergeTree/BackgroundJobsExecutor.cpp | 5 --- .../MergeTree/MergeTreeBackgroundJob.h | 37 ------------------- src/Storages/StorageMergeTree.cpp | 1 - 3 files changed, 43 deletions(-) delete mode 100644 src/Storages/MergeTree/MergeTreeBackgroundJob.h diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f952b04974b..a49712d7d29 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - IBackgroundJobExecutor::IBackgroundJobExecutor( MergeTreeData & data_, Context & global_context_, diff --git a/src/Storages/MergeTree/MergeTreeBackgroundJob.h b/src/Storages/MergeTree/MergeTreeBackgroundJob.h deleted file mode 100644 index 0b1a6ff3a8d..00000000000 --- a/src/Storages/MergeTree/MergeTreeBackgroundJob.h +++ /dev/null @@ -1,37 +0,0 @@ -# pragma once -#include -#include -#include -#include - -namespace DB -{ - -enum PoolType -{ - MERGE_MUTATE, - MOVE, - FETCH, -}; - -struct MergeTreeBackgroundJob -{ - ThreadPool::Job job; - PoolType execute_in_pool; - - MergeTreeBackgroundJob(ThreadPool::Job && job_, PoolType execute_in_pool_) - : job(std::move(job_)), execute_in_pool(execute_in_pool_) - {} - - void operator()() - try - { - job(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -}; - -} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b103095eeb2..7af3e5b7b40 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -40,7 +40,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; extern const int NOT_ENOUGH_SPACE; - extern const int ABORTED; extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; extern const int CANNOT_ASSIGN_OPTIMIZE; From ae8ed99805b8ec01bd384e4ffcf702b9c50762e5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 21:35:03 +0300 Subject: [PATCH 015/149] Missed change --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index a49712d7d29..36a7f8d939e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -57,7 +57,7 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) else if (nothing_to_do) { next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); } else { From 3e3505258f01e0c0909cc54aa86bec8b9974b572 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 14 Oct 2020 22:33:36 +0300 Subject: [PATCH 016/149] No sleep when we have job --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 36a7f8d939e..f3b3b6d3110 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -61,7 +61,8 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) } else { - next_time_to_execute = 1000 * std::uniform_real_distribution(0, sleep_settings.thread_sleep_seconds_random_part)(rng); + scheduling_task->schedule(); + return; } scheduling_task->scheduleAfter(next_time_to_execute); @@ -108,7 +109,7 @@ try scheduleTask(true); } } -catch (...) /// Exception while we looking for task +catch (...) /// Exception while we looking for a task { tryLogCurrentException(__PRETTY_FUNCTION__); scheduleTask(true); From 3394c22c03517d93629f4607b287e4fb1b437838 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 10:39:27 +0300 Subject: [PATCH 017/149] Fix race and clang-tidy --- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/Context.h | 4 ++-- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 3 +++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 1 + 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ac48340c9c6..04f9ce3b753 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1376,7 +1376,7 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } -TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() +TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const { TaskSleepSettings task_settings; @@ -1391,7 +1391,7 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() return task_settings; } -TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() +TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const { TaskSleepSettings task_settings; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0f90c26a43c..3f70b6557d2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -507,8 +507,8 @@ public: */ void dropCaches() const; - TaskSleepSettings getBackgroundProcessingTaskSleepSettings(); - TaskSleepSettings getBackgroundMoveTaskSleepSettings(); + TaskSleepSettings getBackgroundProcessingTaskSleepSettings() const; + TaskSleepSettings getBackgroundMoveTaskSleepSettings() const; BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f3b3b6d3110..ef594948453 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -117,6 +117,7 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { + std::lock_guard lock(task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -128,6 +129,7 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { + std::lock_guard lock(task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -138,6 +140,7 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerDataProcessing() { + std::lock_guard lock(task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index bc051702bfb..c3d2371b9b1 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -65,6 +65,7 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; + std::mutex task_mutex; public: IBackgroundJobExecutor( From c5419083e3f998520104116d26912559c2eaf2ab Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 10:43:50 +0300 Subject: [PATCH 018/149] Better method name --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +++--- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index ef594948453..4959aeb71f7 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -138,7 +138,7 @@ void IBackgroundJobExecutor::finish() } } -void IBackgroundJobExecutor::triggerDataProcessing() +void IBackgroundJobExecutor::triggerTask() { std::lock_guard lock(task_mutex); if (scheduling_task) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index c3d2371b9b1..66a6031bbf9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -76,7 +76,7 @@ public: const std::vector & pools_configs_); void start(); - void triggerDataProcessing(); + void triggerTask(); void finish(); virtual ~IBackgroundJobExecutor(); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 5ad6a7eebc1..40714e5af31 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -28,7 +28,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index b7fd7097546..f0732774c3e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -584,7 +584,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper LOG_DEBUG(log, "Pulled {} entries to queue.", copied_entries.size()); } - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); } return stat.version; @@ -668,7 +668,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C } if (some_active_mutations_were_killed) - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); if (!entries_to_load.empty()) { @@ -792,7 +792,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation( } if (mutation_was_active) - storage.background_executor.triggerDataProcessing(); + storage.background_executor.triggerTask(); return entry; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7af3e5b7b40..ca22e5d3ed1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -362,7 +362,7 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String current_mutations_by_version.emplace(version, insertion.first->second); LOG_INFO(log, "Added mutation: {}", mutation_file_name); - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); return version; } @@ -580,7 +580,7 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id) } /// Maybe there is another mutation that was blocked by the killed one. Try to execute it immediately. - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); return CancellationCode::CancelSent; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc55b87fee7..bf10d931f39 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5868,7 +5868,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI { auto lock = queue.lockQueue(); - background_executor.triggerDataProcessing(); + background_executor.triggerTask(); /// This is significant, because the execution of this task could be delayed at BackgroundPool. /// And we force it to be executed. } From 583d7042e99d822274e0e1fb598b72fb3fda1696 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 11:29:18 +0300 Subject: [PATCH 019/149] Get rid of some deadlocks --- .../MergeTree/BackgroundJobsExecutor.cpp | 3 --- .../MergeTree/BackgroundJobsExecutor.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++-------------- 3 files changed, 5 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 4959aeb71f7..c3b351af520 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -117,7 +117,6 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { - std::lock_guard lock(task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -129,7 +128,6 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { - std::lock_guard lock(task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -140,7 +138,6 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerTask() { - std::lock_guard lock(task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 66a6031bbf9..0abb5bce011 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -65,7 +65,6 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; - std::mutex task_mutex; public: IBackgroundJobExecutor( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bf10d931f39..4cf690c78d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3482,10 +3482,7 @@ void StorageReplicatedMergeTree::startup() /// If we don't separate create/start steps, race condition will happen /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. - { - auto lock = queue.lockQueue(); - background_executor.start(); - } + background_executor.start(); if (areBackgroundMovesNeeded()) background_moves_executor.start(); @@ -3518,13 +3515,10 @@ void StorageReplicatedMergeTree::shutdown() parts_mover.moves_blocker.cancelForever(); restarting_thread.shutdown(); + background_executor.finish(); { - /// Queue can trigger queue_task_handle itself. So we ensure that all - /// queue processes finished and after that reset queue_task_handle. auto lock = queue.lockQueue(); - background_executor.finish(); - /// Cancel logs pulling after background task were cancelled. It's still /// required because we can trigger pullLogsToQueue during manual OPTIMIZE, /// MUTATE, etc. query. @@ -5866,12 +5860,9 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeper()); - { - auto lock = queue.lockQueue(); - background_executor.triggerTask(); - /// This is significant, because the execution of this task could be delayed at BackgroundPool. - /// And we force it to be executed. - } + /// This is significant, because the execution of this task could be delayed at BackgroundPool. + /// And we force it to be executed. + background_executor.triggerTask(); Poco::Event target_size_event; auto callback = [&target_size_event, queue_size] (size_t new_queue_size) From e84eda176a91bcf5c9b5de4d8308d947f248588c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:22:02 +0300 Subject: [PATCH 020/149] Add test and slightly refactored --- .../MergeTree/BackgroundJobsExecutor.cpp | 28 +++++---- .../MergeTree/BackgroundJobsExecutor.h | 22 ++++--- .../tests/gtest_background_executor.cpp | 62 +++++++++++++++++++ 3 files changed, 91 insertions(+), 21 deletions(-) create mode 100644 src/Storages/tests/gtest_background_executor.cpp diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index c3b351af520..12c3ff3e418 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -9,14 +9,10 @@ namespace DB { IBackgroundJobExecutor::IBackgroundJobExecutor( - MergeTreeData & data_, Context & global_context_, - const String & task_name_, const TaskSleepSettings & sleep_settings_, const std::vector & pools_configs_) - : data(data_) - , global_context(global_context_) - , task_name(task_name_) + : global_context(global_context_) , sleep_settings(sleep_settings_) , rng(randomSeed()) { @@ -30,7 +26,7 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( namespace { -bool incrementIfLess(std::atomic & atomic_value, long max_value) +bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) @@ -120,7 +116,7 @@ void IBackgroundJobExecutor::start() if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( - data.getStorageID().getFullTableName() + task_name, [this]{ jobExecutingTask(); }); + getBackgroundJobName(), [this]{ jobExecutingTask(); }); } scheduling_task->activateAndSchedule(); @@ -151,14 +147,18 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_) : IBackgroundJobExecutor( - data_, global_context_, - "(dataProcessingTask)", global_context_.getBackgroundProcessingTaskSleepSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) + , data(data_) { } +String BackgroundJobsExecutor::getBackgroundJobName() const +{ + return data.getStorageID().getFullTableName() + " (dataProcessingTask)"; +} + std::optional BackgroundJobsExecutor::getBackgroundJob() { auto job = data.getDataProcessingJob(); @@ -171,14 +171,18 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( MergeTreeData & data_, Context & global_context_) : IBackgroundJobExecutor( - data_, global_context_, - "(dataMovingTask)", global_context_.getBackgroundMoveTaskSleepSettings(), - {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) + {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) + , data(data_) { } +String BackgroundMovesExecutor::getBackgroundJobName() const +{ + return data.getStorageID().getFullTableName() + " (dataMovingTask)"; +} + std::optional BackgroundMovesExecutor::getBackgroundJob() { auto job = data.getDataMovingJob(); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 0abb5bce011..1710cccc54c 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -51,8 +51,6 @@ struct JobAndPool class IBackgroundJobExecutor { -protected: - MergeTreeData & data; Context & global_context; private: String task_name; @@ -67,13 +65,6 @@ private: BackgroundSchedulePool::TaskHolder scheduling_task; public: - IBackgroundJobExecutor( - MergeTreeData & data_, - Context & global_context_, - const String & task_name_, - const TaskSleepSettings & sleep_settings_, - const std::vector & pools_configs_); - void start(); void triggerTask(); void finish(); @@ -81,7 +72,14 @@ public: virtual ~IBackgroundJobExecutor(); protected: + IBackgroundJobExecutor( + Context & global_context_, + const TaskSleepSettings & sleep_settings_, + const std::vector & pools_configs_); + + virtual String getBackgroundJobName() const = 0; virtual std::optional getBackgroundJob() = 0; + private: void jobExecutingTask(); void scheduleTask(bool nothing_to_do); @@ -89,23 +87,29 @@ private: class BackgroundJobsExecutor final : public IBackgroundJobExecutor { +private: + MergeTreeData & data; public: BackgroundJobsExecutor( MergeTreeData & data_, Context & global_context_); protected: + String getBackgroundJobName() const override; std::optional getBackgroundJob() override; }; class BackgroundMovesExecutor final : public IBackgroundJobExecutor { +private: + MergeTreeData & data; public: BackgroundMovesExecutor( MergeTreeData & data_, Context & global_context_); protected: + String getBackgroundJobName() const override; std::optional getBackgroundJob() override; }; diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp new file mode 100644 index 00000000000..1bc96ed0c42 --- /dev/null +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -0,0 +1,62 @@ +#include +#include +#include +#include +#include +#include +using namespace std::chrono_literals; +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; +} + +using namespace DB; + +static std::atomic counter{0}; + +class TestJobExecutor : public IBackgroundJobExecutor +{ +public: + TestJobExecutor(Context & context) + :IBackgroundJobExecutor( + context, + TaskSleepSettings{}, + {PoolConfig{PoolType::MERGE_MUTATE, 4, CurrentMetrics::BackgroundPoolTask}}) + {} + +protected: + String getBackgroundJobName() const override + { + return "TestJob"; + } + + std::optional getBackgroundJob() override + { + return JobAndPool{[] { std::this_thread::sleep_for(1s); counter++; }, PoolType::MERGE_MUTATE}; + } +}; + +using TestExecutorPtr = std::unique_ptr; + +TEST(BackgroundExecutor, TestMetric) +{ + auto & context_holder = getContext(); + std::vector executors; + for (size_t i = 0; i < 100; ++i) + executors.emplace_back(std::make_unique(const_cast(context_holder.context))); + + for (size_t i = 0; i < 100; ++i) + executors[i]->start(); + + for (size_t i = 0; i < 100; ++i) + { + EXPECT_TRUE(CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load() <= 4); + std::this_thread::sleep_for(200ms); + } + + for (size_t i = 0; i < 100; ++i) + executors[i]->finish(); + + /// Sanity check + EXPECT_TRUE(counter > 50); +} From 374949051c9da45b1835440045d6313d61f11d10 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:41:36 +0300 Subject: [PATCH 021/149] Add thread safety --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 3 +++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 1 + 2 files changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 12c3ff3e418..873d7fd8aa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -113,6 +113,7 @@ catch (...) /// Exception while we looking for a task void IBackgroundJobExecutor::start() { + std::lock_guard lock(scheduling_task_mutex); if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( @@ -124,6 +125,7 @@ void IBackgroundJobExecutor::start() void IBackgroundJobExecutor::finish() { + std::lock_guard lock(scheduling_task_mutex); if (scheduling_task) { scheduling_task->deactivate(); @@ -134,6 +136,7 @@ void IBackgroundJobExecutor::finish() void IBackgroundJobExecutor::triggerTask() { + std::lock_guard lock(scheduling_task_mutex); if (scheduling_task) scheduling_task->schedule(); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 1710cccc54c..03d3823bcab 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -63,6 +63,7 @@ private: std::unordered_map pools_configs; BackgroundSchedulePool::TaskHolder scheduling_task; + std::mutex scheduling_task_mutex; public: void start(); From 84b0c2204491874190d43696d47cd18a731dbe22 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 13:54:50 +0300 Subject: [PATCH 022/149] fix possible deadlock --- src/Storages/StorageMergeTree.cpp | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ca22e5d3ed1..b9a675ffedc 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -352,16 +352,18 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String /// where storage can be placed. See loadMutations(). auto disk = getStoragePolicy()->getAnyDisk(); Int64 version; - std::lock_guard lock(currently_processing_in_background_mutex); + { + std::lock_guard lock(currently_processing_in_background_mutex); - MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); - version = increment.get(); - entry.commit(version); - mutation_file_name = entry.file_name; - auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); - current_mutations_by_version.emplace(version, insertion.first->second); + MergeTreeMutationEntry entry(commands, disk, relative_data_path, insert_increment.get()); + version = increment.get(); + entry.commit(version); + mutation_file_name = entry.file_name; + auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); + current_mutations_by_version.emplace(version, insertion.first->second); - LOG_INFO(log, "Added mutation: {}", mutation_file_name); + LOG_INFO(log, "Added mutation: {}", mutation_file_name); + } background_executor.triggerTask(); return version; } From dccf51b3169480907b15b7049b8c38152fe94c21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 14:08:39 +0300 Subject: [PATCH 023/149] Remove redundant logging --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 6aa3d1f3238..90631762548 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -165,11 +165,8 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge() const UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t pool_size, size_t pool_used) const { - //LOG_DEBUG(&Poco::Logger::get("DEBUG"), "POOL SIZE {}, POOL USED {}", pool_size, pool_used); if (pool_used > pool_size) - { throw Exception("Logical error: invalid arguments passed to getMaxSourcePartsSize: pool_used > pool_size", ErrorCodes::LOGICAL_ERROR); - } size_t free_entries = pool_size - pool_used; const auto data_settings = data.getSettings(); From 629506445dfd1f155a4c09b9c460270ef7081119 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 16:57:50 +0300 Subject: [PATCH 024/149] Better backoff policy --- .../MergeTree/BackgroundJobsExecutor.cpp | 32 ++- .../MergeTree/BackgroundJobsExecutor.h | 4 +- .../MergeTree/BackgroundProcessingPool.cpp | 243 ------------------ .../tests/gtest_background_executor.cpp | 4 +- 4 files changed, 19 insertions(+), 264 deletions(-) delete mode 100644 src/Storages/MergeTree/BackgroundProcessingPool.cpp diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 873d7fd8aa0..57003bbfaa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -39,21 +39,17 @@ bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) } -void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) +void IBackgroundJobExecutor::scheduleTask() { - auto errors = errors_count.load(std::memory_order_relaxed); - size_t next_time_to_execute = 0; - if (errors != 0) + auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); + if (no_work_done_times != 0) { - next_time_to_execute += 1000 * (std::min( + auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, errors)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); - } - else if (nothing_to_do) - { - next_time_to_execute += 1000 * (sleep_settings.thread_sleep_seconds_if_nothing_to_do + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + scheduling_task->scheduleAfter(next_time_to_execute); } else { @@ -61,7 +57,6 @@ void IBackgroundJobExecutor::scheduleTask(bool nothing_to_do) return; } - scheduling_task->scheduleAfter(next_time_to_execute); } void IBackgroundJobExecutor::jobExecutingTask() @@ -82,11 +77,11 @@ try { job(); CurrentMetrics::values[pool_config.tasks_metric]--; - errors_count = 0; + no_work_done_count = 0; } catch (...) { - errors_count++; + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } @@ -94,21 +89,24 @@ try } catch (...) { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } } - scheduleTask(false); } else /// Nothing to do, no jobs { - scheduleTask(true); + no_work_done_count++; } + + scheduleTask(); } catch (...) /// Exception while we looking for a task { + no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - scheduleTask(true); + scheduleTask(); } void IBackgroundJobExecutor::start() diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 03d3823bcab..d12124492f9 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -57,7 +57,7 @@ private: TaskSleepSettings sleep_settings; pcg64 rng; - std::atomic errors_count{0}; + std::atomic no_work_done_count{0}; std::unordered_map pools; std::unordered_map pools_configs; @@ -83,7 +83,7 @@ protected: private: void jobExecutingTask(); - void scheduleTask(bool nothing_to_do); + void scheduleTask(); }; class BackgroundJobsExecutor final : public IBackgroundJobExecutor diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp deleted file mode 100644 index d779bd16814..00000000000 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ /dev/null @@ -1,243 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ - -void BackgroundProcessingPoolTaskInfo::signalReadyToRun() -{ - Poco::Timestamp current_time; - { - std::unique_lock lock(pool.tasks_mutex); - - /// This check ensures that the iterator is valid. Must be performed under the same mutex as invalidation. - if (removed) - return; - - /// If this task did nothing the previous time and still should sleep, then reschedule to cancel the sleep. - const auto & scheduled_time = iterator->first; - if (scheduled_time > current_time) - pool.rescheduleTask(iterator, current_time); - - /// Note that if all threads are currently busy doing their work, this call will not wakeup any thread. - pool.wake_event.notify_one(); - } -} - - -BackgroundProcessingPool::BackgroundProcessingPool(int size_, - const PoolSettings & pool_settings, - const char * log_name, - const char * thread_name_) - : size(size_) - , thread_name(thread_name_) - , settings(pool_settings) -{ - logger = &Poco::Logger::get(log_name); - LOG_INFO(logger, "Create {} with {} threads", log_name, size); - - threads.resize(size); - for (auto & thread : threads) - thread = ThreadFromGlobalPool([this] { workLoopFunc(); }); -} - - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::createTask(const Task & task) -{ - return std::make_shared(*this, task); -} - -void BackgroundProcessingPool::startTask(const TaskHandle & task, bool allow_execute_in_parallel) -{ - Poco::Timestamp current_time; - - task->allow_execute_in_parallel = allow_execute_in_parallel; - - { - std::unique_lock lock(tasks_mutex); - task->iterator = tasks.emplace(current_time, task); - - wake_event.notify_all(); - } - -} - -BackgroundProcessingPool::TaskHandle BackgroundProcessingPool::addTask(const Task & task) -{ - TaskHandle res = createTask(task); - startTask(res); - return res; -} - -void BackgroundProcessingPool::removeTask(const TaskHandle & task) -{ - if (task->removed.exchange(true)) - return; - - /// Wait for all executions of this task. - { - std::unique_lock wlock(task->rwlock); - } - - { - std::unique_lock lock(tasks_mutex); - tasks.erase(task->iterator); - /// Note that the task may be still accessible through TaskHandle (shared_ptr). - } -} - -BackgroundProcessingPool::~BackgroundProcessingPool() -{ - try - { - { - std::lock_guard lock(tasks_mutex); - shutdown = true; - wake_event.notify_all(); - } - - for (auto & thread : threads) - thread.join(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - - -void BackgroundProcessingPool::workLoopFunc() -{ - setThreadName(thread_name); - - { - std::lock_guard lock(tasks_mutex); - - if (thread_group) - { - /// Put all threads to one thread pool - CurrentThread::attachTo(thread_group); - } - else - { - CurrentThread::initializeQuery(); - thread_group = CurrentThread::getGroup(); - } - } - - SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); }); - - pcg64 rng(randomSeed()); - std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - - Poco::Timestamp scheduled_task_start_time; - - while (true) - { - TaskResult task_result = TaskResult::ERROR; - TaskHandle task; - - { - std::unique_lock lock(tasks_mutex); - - while (!task && !shutdown) - { - for (const auto & [time, handle] : tasks) - { - if (!handle->removed - && (handle->allow_execute_in_parallel || handle->concurrent_executors == 0)) - { - task = handle; - scheduled_task_start_time = time; - ++task->concurrent_executors; - break; - } - } - - if (task) - { - Poco::Timestamp current_time; - - if (scheduled_task_start_time <= current_time) - continue; - - wake_event.wait_for(lock, - std::chrono::microseconds(scheduled_task_start_time - current_time - + std::uniform_int_distribution(0, settings.thread_sleep_seconds_random_part * 1000000)(rng))); - } - else - { - wake_event.wait_for(lock, - std::chrono::duration(settings.thread_sleep_seconds - + std::uniform_real_distribution(0, settings.thread_sleep_seconds_random_part)(rng))); - } - } - - if (shutdown) - break; - } - - std::shared_lock rlock(task->rwlock); - - if (task->removed) - continue; - - try - { - CurrentMetrics::Increment metric_increment{settings.tasks_metric}; - task_result = task->task_function(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - { - std::unique_lock lock(tasks_mutex); - - if (shutdown) - break; - - --task->concurrent_executors; - - if (task->removed) - continue; - - if (task_result == TaskResult::SUCCESS) - task->count_no_work_done = 0; - else if (task_result == TaskResult::ERROR) - ++task->count_no_work_done; - /// NOTHING_TO_DO should not increment count_no_work_done - /// otherwise error after period of inactivity (lot of NOTHING_TO_DO) - /// leads to 5-10 min replication hang - - /// If task has done work, it could be executed again immediately. - /// If not, add delay before next run. - - Poco::Timestamp next_time_to_execute; /// current time - if (task_result == TaskResult::ERROR) - next_time_to_execute += 1000000 * (std::min( - settings.task_sleep_seconds_when_no_work_max, - settings.task_sleep_seconds_when_no_work_min * std::pow(settings.task_sleep_seconds_when_no_work_multiplier, task->count_no_work_done)) - + std::uniform_real_distribution(0, settings.task_sleep_seconds_when_no_work_random_part)(rng)); - else if (task_result == TaskResult::NOTHING_TO_DO) - next_time_to_execute += 1000000 * settings.thread_sleep_seconds_if_nothing_to_do; - - rescheduleTask(task->iterator, next_time_to_execute); - } - } -} - -} diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp index 1bc96ed0c42..77bc539b1b1 100644 --- a/src/Storages/tests/gtest_background_executor.cpp +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -17,7 +17,7 @@ static std::atomic counter{0}; class TestJobExecutor : public IBackgroundJobExecutor { public: - TestJobExecutor(Context & context) + explicit TestJobExecutor(Context & context) :IBackgroundJobExecutor( context, TaskSleepSettings{}, @@ -40,7 +40,7 @@ using TestExecutorPtr = std::unique_ptr; TEST(BackgroundExecutor, TestMetric) { - auto & context_holder = getContext(); + const auto & context_holder = getContext(); std::vector executors; for (size_t i = 0; i < 100; ++i) executors.emplace_back(std::make_unique(const_cast(context_holder.context))); From 4c1f5cab137e05533d1b8505b0c40a19a86dd8e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 19:10:22 +0300 Subject: [PATCH 025/149] Sleep less in background task --- src/Interpreters/Context.cpp | 12 ++++++------ src/Interpreters/InterpreterSystemQuery.cpp | 11 +++++++++-- src/Storages/IStorage.h | 2 ++ src/Storages/MergeTree/BackgroundJobsExecutor.h | 6 +++--- src/Storages/StorageMergeTree.cpp | 8 ++++++++ src/Storages/StorageMergeTree.h | 2 ++ src/Storages/StorageReplicatedMergeTree.cpp | 9 +++++++++ src/Storages/StorageReplicatedMergeTree.h | 2 ++ 8 files changed, 41 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 47068002b85..5b383968435 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1384,9 +1384,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 5); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 300); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -1399,9 +1399,9 @@ TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 5); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 300); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 4bfa84090c2..4d03e1871ef 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -135,11 +135,15 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, if (table_id) { + auto table = DatabaseCatalog::instance().tryGetTable(table_id, context); context.checkAccess(getRequiredAccessType(action_type), table_id); if (start) - manager->remove(table_id, action_type); + { + manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } else - manager->add(table_id, action_type); + manager->add(table, action_type); } else { @@ -164,7 +168,10 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, } if (start) + { manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } else manager->add(table, action_type); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4a2e70aa84b..cced739863a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -428,6 +428,8 @@ public: return {}; } + virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {} + std::atomic is_dropped{false}; /// Does table support index for IN sections diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index d12124492f9..aa5eeaf2ded 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -21,9 +21,9 @@ struct TaskSleepSettings double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; - double task_sleep_seconds_when_no_work_max = 600; - double task_sleep_seconds_when_no_work_multiplier = 1.1; + double task_sleep_seconds_when_no_work_min = 5; + double task_sleep_seconds_when_no_work_max = 300; + double task_sleep_seconds_when_no_work_multiplier = 1.01; double task_sleep_seconds_when_no_work_random_part = 1.0; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b9a675ffedc..19c8357f2b5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1388,6 +1388,14 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) return {}; } +void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) +{ + if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge) + background_executor.triggerTask(); + else if (action_type == ActionLocks::PartsMove) + background_moves_executor.triggerTask(); +} + CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & context) { CheckResults results; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f7473872189..e6892075da8 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -85,6 +85,8 @@ public: ActionLock getActionLock(StorageActionBlockType action_type) override; + void onActionLockRemove(StorageActionBlockType action_type) override; + CheckResults checkData(const ASTPtr & query, const Context & context) override; ThreadPool::Job getDataProcessingJob() override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4cf690c78d1..dedaa3c0cf0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5852,6 +5852,15 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti return {}; } +void StorageReplicatedMergeTree::onActionLockRemove(StorageActionBlockType action_type) +{ + if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge + || action_type == ActionLocks::PartsFetch || action_type == ActionLocks::PartsSend + || action_type == ActionLocks::ReplicationQueue) + background_executor.triggerTask(); + else if (action_type == ActionLocks::PartsMove) + background_moves_executor.triggerTask(); +} bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ba01ca9d3af..e9c54524892 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -138,6 +138,8 @@ public: ActionLock getActionLock(StorageActionBlockType action_type) override; + void onActionLockRemove(StorageActionBlockType action_type) override; + /// Wait when replication queue size becomes less or equal than queue_size /// If timeout is exceeded returns false bool waitForShrinkingQueueSize(size_t queue_size = 0, UInt64 max_wait_milliseconds = 0); From 2b2a55a0818406739e2cd923553f4f6ddecb51f7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 15 Oct 2020 22:23:46 +0300 Subject: [PATCH 026/149] Add action lock only if table exists --- src/Interpreters/InterpreterSystemQuery.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 4d03e1871ef..2774da7201a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -136,14 +136,16 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, if (table_id) { auto table = DatabaseCatalog::instance().tryGetTable(table_id, context); - context.checkAccess(getRequiredAccessType(action_type), table_id); - if (start) + if (table) { - manager->remove(table, action_type); - table->onActionLockRemove(action_type); + if (start) + { + manager->remove(table, action_type); + table->onActionLockRemove(action_type); + } + else + manager->add(table, action_type); } - else - manager->add(table, action_type); } else { From 0fb1f74b2ed340c2ea68e89fde5f5620d39b8c3f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 11:00:15 +0300 Subject: [PATCH 027/149] Trying to schedule better --- src/Interpreters/Context.cpp | 12 ++++++------ src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 5 +++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5b383968435..47068002b85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1384,9 +1384,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 5); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 300); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; } @@ -1399,9 +1399,9 @@ TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0); task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1); - task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 5); - task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 300); - task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.01); + task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10); + task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600); + task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1); task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0); return task_settings; diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 57003bbfaa0..5825fbec939 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -54,7 +54,6 @@ void IBackgroundJobExecutor::scheduleTask() else { scheduling_task->schedule(); - return; } } @@ -85,6 +84,7 @@ try tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } + scheduleTask(); }); } catch (...) @@ -92,15 +92,16 @@ try no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; + scheduleTask(); } } } else /// Nothing to do, no jobs { no_work_done_count++; + scheduleTask(); } - scheduleTask(); } catch (...) /// Exception while we looking for a task { From 7edbe4d276567db6a0ad653a4020723899c06cd5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 13:12:31 +0300 Subject: [PATCH 028/149] Add some comments --- src/Interpreters/Context.cpp | 8 +- src/Interpreters/Context.h | 8 +- src/Storages/IStorage.h | 1 + .../MergeTree/BackgroundJobsExecutor.cpp | 89 +++++++++++-------- .../MergeTree/BackgroundJobsExecutor.h | 81 ++++++++++++----- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- src/Storages/MergeTree/MergeTreeData.h | 7 +- src/Storages/StorageMergeTree.cpp | 10 +-- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- 11 files changed, 136 insertions(+), 85 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 47068002b85..e47560319ab 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1376,9 +1376,9 @@ BackgroundSchedulePool & Context::getBufferFlushSchedulePool() return *shared->buffer_flush_schedule_pool; } -TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const +BackgroundTaskSchedulingSettings Context::getBackgroundProcessingTaskSchedulingSettings() const { - TaskSleepSettings task_settings; + BackgroundTaskSchedulingSettings task_settings; const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10); @@ -1391,9 +1391,9 @@ TaskSleepSettings Context::getBackgroundProcessingTaskSleepSettings() const return task_settings; } -TaskSleepSettings Context::getBackgroundMoveTaskSleepSettings() const +BackgroundTaskSchedulingSettings Context::getBackgroundMoveTaskSchedulingSettings() const { - TaskSleepSettings task_settings; + BackgroundTaskSchedulingSettings task_settings; const auto & config = getConfigRef(); task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3f70b6557d2..66569d48c08 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -110,7 +110,7 @@ using OutputFormatPtr = std::shared_ptr; class IVolume; using VolumePtr = std::shared_ptr; struct NamedSession; -struct TaskSleepSettings; +struct BackgroundTaskSchedulingSettings; #if USE_EMBEDDED_COMPILER @@ -507,8 +507,10 @@ public: */ void dropCaches() const; - TaskSleepSettings getBackgroundProcessingTaskSleepSettings() const; - TaskSleepSettings getBackgroundMoveTaskSleepSettings() const; + /// Settings for MergeTree background tasks stored in config.xml + BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; + BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; + BackgroundSchedulePool & getBufferFlushSchedulePool(); BackgroundSchedulePool & getSchedulePool(); BackgroundSchedulePool & getMessageBrokerSchedulePool(); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index cced739863a..6a5d9b3ba9a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -428,6 +428,7 @@ public: return {}; } + /// Call when lock from previous method removed virtual void onActionLockRemove(StorageActionBlockType /* action_type */) {} std::atomic is_dropped{false}; diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 5825fbec939..b888eec3dfe 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -5,12 +5,18 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric BackgroundPoolTask; + extern const Metric BackgroundMovePoolTask; +} + namespace DB { IBackgroundJobExecutor::IBackgroundJobExecutor( Context & global_context_, - const TaskSleepSettings & sleep_settings_, + const BackgroundTaskSchedulingSettings & sleep_settings_, const std::vector & pools_configs_) : global_context(global_context_) , sleep_settings(sleep_settings_) @@ -23,10 +29,35 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( } } + +void IBackgroundJobExecutor::scheduleTask() +{ + auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); + /// If we have no jobs or some errors than sleep with backoff + if (no_work_done_times != 0) + { + auto next_time_to_execute = 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + scheduling_task->scheduleAfter(next_time_to_execute); + } + else + { + /// We have background jobs, schedule task as soon as possible + scheduling_task->schedule(); + } + +} + namespace { -bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) +/// Tricky function: we have separate thread pool with max_threads in each background executor for each table +/// But we want total background threads to be less than max_threads value. So we use global atomic counter (BackgroundMetric) +/// to limit total number of background threads. +bool incrementMetricIfLessThanMax(std::atomic & atomic_value, Int64 max_value) { auto value = atomic_value.load(std::memory_order_relaxed); while (value < max_value) @@ -39,34 +70,15 @@ bool incrementIfLess(std::atomic & atomic_value, Int64 max_value) } -void IBackgroundJobExecutor::scheduleTask() -{ - auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); - if (no_work_done_times != 0) - { - auto next_time_to_execute = 1000 * (std::min( - sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); - - scheduling_task->scheduleAfter(next_time_to_execute); - } - else - { - scheduling_task->schedule(); - } - -} - void IBackgroundJobExecutor::jobExecutingTask() try { auto job_and_pool = getBackgroundJob(); - if (job_and_pool) + if (job_and_pool) /// If we have job, than try to assign into background pool { auto & pool_config = pools_configs[job_and_pool->pool_type]; - /// If corresponding pool is not full, otherwise try next time - if (incrementIfLess(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) + /// If corresponding pool is not full increment metric and assign new job + if (incrementMetricIfLessThanMax(CurrentMetrics::values[pool_config.tasks_metric], pool_config.max_pool_size)) { try /// this try required because we have to manually decrement metric { @@ -75,6 +87,7 @@ try try /// We don't want exceptions in background pool { job(); + /// Job done, decrement metric and reset no_work counter CurrentMetrics::values[pool_config.tasks_metric]--; no_work_done_count = 0; } @@ -84,17 +97,23 @@ try tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; } + /// Job done, new empty space in pool, schedule background task scheduleTask(); }); } catch (...) { + /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety add catch here no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; scheduleTask(); } } + else /// Pool is full and we have some work to do, let's try to schedule without backoff + { + scheduleTask(); + } } else /// Nothing to do, no jobs { @@ -103,7 +122,7 @@ try } } -catch (...) /// Exception while we looking for a task +catch (...) /// Exception while we looking for a task, reschedule { no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); @@ -116,7 +135,7 @@ void IBackgroundJobExecutor::start() if (!scheduling_task) { scheduling_task = global_context.getSchedulePool().createTask( - getBackgroundJobName(), [this]{ jobExecutingTask(); }); + getBackgroundTaskName(), [this]{ jobExecutingTask(); }); } scheduling_task->activateAndSchedule(); @@ -150,23 +169,20 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( Context & global_context_) : IBackgroundJobExecutor( global_context_, - global_context_.getBackgroundProcessingTaskSleepSettings(), + global_context_.getBackgroundProcessingTaskSchedulingSettings(), {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) , data(data_) { } -String BackgroundJobsExecutor::getBackgroundJobName() const +String BackgroundJobsExecutor::getBackgroundTaskName() const { return data.getStorageID().getFullTableName() + " (dataProcessingTask)"; } std::optional BackgroundJobsExecutor::getBackgroundJob() { - auto job = data.getDataProcessingJob(); - if (job) - return JobAndPool{job, PoolType::MERGE_MUTATE}; - return {}; + return data.getDataProcessingJob(); } BackgroundMovesExecutor::BackgroundMovesExecutor( @@ -174,23 +190,20 @@ BackgroundMovesExecutor::BackgroundMovesExecutor( Context & global_context_) : IBackgroundJobExecutor( global_context_, - global_context_.getBackgroundMoveTaskSleepSettings(), + global_context_.getBackgroundMoveTaskSchedulingSettings(), {PoolConfig{PoolType::MOVE, global_context_.getSettingsRef().background_move_pool_size, CurrentMetrics::BackgroundMovePoolTask}}) , data(data_) { } -String BackgroundMovesExecutor::getBackgroundJobName() const +String BackgroundMovesExecutor::getBackgroundTaskName() const { return data.getStorageID().getFullTableName() + " (dataMovingTask)"; } std::optional BackgroundMovesExecutor::getBackgroundJob() { - auto job = data.getDataMovingJob(); - if (job) - return JobAndPool{job, PoolType::MOVE}; - return {}; + return data.getDataMovingJob(); } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index aa5eeaf2ded..e2502f534e5 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -5,87 +5,118 @@ #include #include -namespace CurrentMetrics -{ - extern const Metric BackgroundPoolTask; - extern const Metric BackgroundMovePoolTask; -} namespace DB { -struct TaskSleepSettings +/// Settings for background tasks scheduling. Each background executor has one +/// BackgroundSchedulingPoolTask and depending on execution result may put this +/// task to sleep according to settings. Look at scheduleTask function for details. +struct BackgroundTaskSchedulingSettings { double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 5; - double task_sleep_seconds_when_no_work_max = 300; - double task_sleep_seconds_when_no_work_multiplier = 1.01; + double task_sleep_seconds_when_no_work_min = 6; + double task_sleep_seconds_when_no_work_max = 600; + double task_sleep_seconds_when_no_work_multiplier = 1.1; double task_sleep_seconds_when_no_work_random_part = 1.0; }; -enum PoolType +/// Pool type where we must execute new job. Each background executor can have several +/// background pools. When it receives new job it will execute new task in corresponding pool. +enum class PoolType { MERGE_MUTATE, - FETCH, MOVE, - LOW_PRIORITY, -}; - -struct PoolConfig -{ - PoolType pool_type; - size_t max_pool_size; - CurrentMetrics::Metric tasks_metric; }; +/// Result from background job providers. Function which will be executed in pool and pool type. struct JobAndPool { ThreadPool::Job job; PoolType pool_type; }; +/// Background jobs executor which execute heavy-weight background tasks for MergTree tables, like +/// background merges, moves, mutations, fetches and so on. +/// Consists of two important parts: +/// 1) Task in background scheduling pool which receives new jobs from storages and put them into required pool. +/// 2) One or more ThreadPool objects, which execute background jobs. class IBackgroundJobExecutor { +protected: Context & global_context; + + /// Configuration for single background ThreadPool + struct PoolConfig + { + /// This pool type + PoolType pool_type; + /// Max pool size in threads + size_t max_pool_size; + /// Metric that we have to increment when we execute task in this pool + CurrentMetrics::Metric tasks_metric; + }; + private: + /// Name for task in background scheduling pool String task_name; - TaskSleepSettings sleep_settings; + /// Settings for execution control of background scheduling task + BackgroundTaskSchedulingSettings sleep_settings; + /// Useful for random backoff timeouts generation pcg64 rng; + /// How many times execution of background job failed or we have + /// no new jobs. std::atomic no_work_done_count{0}; + /// Pools where we execute background jobs std::unordered_map pools; + /// Configs for background pools std::unordered_map pools_configs; + /// Scheduling task which assign jobs in background pool BackgroundSchedulePool::TaskHolder scheduling_task; + /// Mutex for thread safety std::mutex scheduling_task_mutex; public: + /// These three functions are thread safe + + /// Start background task and start to assign jobs void start(); + /// Schedule background task as soon as possible, even if it sleep at this + /// moment for some reason. void triggerTask(); + /// Finish execution: deactivate background task and wait already scheduled jobs void finish(); + /// Just call finish virtual ~IBackgroundJobExecutor(); protected: IBackgroundJobExecutor( Context & global_context_, - const TaskSleepSettings & sleep_settings_, + const BackgroundTaskSchedulingSettings & sleep_settings_, const std::vector & pools_configs_); - virtual String getBackgroundJobName() const = 0; + /// Name for task in background schedule pool + virtual String getBackgroundTaskName() const = 0; + /// Get job for background execution virtual std::optional getBackgroundJob() = 0; private: + /// Function that executes in background scheduling pool void jobExecutingTask(); + /// Recalculate timeouts when we have to check for a new job void scheduleTask(); }; +/// Main jobs executor: merges, mutations, fetches and so on class BackgroundJobsExecutor final : public IBackgroundJobExecutor { private: @@ -96,10 +127,12 @@ public: Context & global_context_); protected: - String getBackgroundJobName() const override; + String getBackgroundTaskName() const override; std::optional getBackgroundJob() override; }; +/// Move jobs executor, move parts between disks in the background +/// Does nothing in case of default configuration class BackgroundMovesExecutor final : public IBackgroundJobExecutor { private: @@ -110,7 +143,7 @@ public: Context & global_context_); protected: - String getBackgroundJobName() const override; + String getBackgroundTaskName() const override; std::optional getBackgroundJob() override; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 82c01acbcf4..b7aeb29f4cd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3621,7 +3621,7 @@ bool MergeTreeData::selectPartsAndMove() return moveParts(std::move(moving_tagger)); } -ThreadPool::Job MergeTreeData::getDataMovingJob() +std::optional MergeTreeData::getDataMovingJob() { if (parts_mover.moves_blocker.isCancelled()) return {}; @@ -3630,10 +3630,10 @@ ThreadPool::Job MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - return [this, moving_tagger{std::move(moving_tagger)}] () mutable + return JobAndPool{[this, moving_tagger{std::move(moving_tagger)}] () mutable { moveParts(moving_tagger); - }; + }, PoolType::MOVE}; } bool MergeTreeData::areBackgroundMovesNeeded() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ed0e7e33d8b..d3bfe06da99 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -40,6 +40,7 @@ class AlterCommands; class MergeTreePartsMover; class MutationCommands; class Context; +struct JobAndPool; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -710,8 +711,10 @@ public: /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; - virtual ThreadPool::Job getDataProcessingJob() = 0; - ThreadPool::Job getDataMovingJob(); + /// Return main processing background job, like merge/mutate/fetch and so on + virtual std::optional getDataProcessingJob() = 0; + /// Return job to move parts between disks/volumes and so on. + std::optional getDataMovingJob(); bool areBackgroundMovesNeeded() const; protected: diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 19c8357f2b5..286ab93e6b8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -893,7 +893,7 @@ bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_sn return true; } -ThreadPool::Job StorageMergeTree::getDataProcessingJob() +std::optional StorageMergeTree::getDataProcessingJob() { if (shutdown_called) return {}; @@ -910,17 +910,17 @@ ThreadPool::Job StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - return [this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry); else if (mutate_entry) mutateSelectedPart(metadata_snapshot, *mutate_entry); - }; + }, PoolType::MERGE_MUTATE}; } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - return [this] () + return JobAndPool{[this] () { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -931,7 +931,7 @@ ThreadPool::Job StorageMergeTree::getDataProcessingJob() clearOldWriteAheadLogs(); } clearOldMutations(); - }; + }, PoolType::MERGE_MUTATE}; } return {}; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index e6892075da8..b3d281e77b7 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -89,7 +89,7 @@ public: CheckResults checkData(const ASTPtr & query, const Context & context) override; - ThreadPool::Job getDataProcessingJob() override; + std::optional getDataProcessingJob() override; private: /// Mutex and condvar for synchronous mutations wait diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dedaa3c0cf0..7df82620bf1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2601,8 +2601,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel }); } - -ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() +std::optional StorageReplicatedMergeTree::getDataProcessingJob() { /// If replication queue is stopped exit immediately as we successfully executed the task if (queue.actions_blocker.isCancelled()) @@ -2616,10 +2615,10 @@ ThreadPool::Job StorageReplicatedMergeTree::getDataProcessingJob() if (!entry) return {}; - return [this, selected_entry{std::move(selected_entry)}] () mutable + return JobAndPool{[this, selected_entry{std::move(selected_entry)}] () mutable { processQueueEntry(selected_entry); - }; + }, PoolType::MERGE_MUTATE}; } bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e9c54524892..79520109406 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -197,7 +197,7 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); - ThreadPool::Job getDataProcessingJob() override; + std::optional getDataProcessingJob() override; private: From c22c760ac1d1fa94e8fa269f8f0ae82caf0dc133 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 13:18:56 +0300 Subject: [PATCH 029/149] Fix unit test build --- src/Storages/tests/gtest_background_executor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/tests/gtest_background_executor.cpp b/src/Storages/tests/gtest_background_executor.cpp index 77bc539b1b1..bf9a305ccc9 100644 --- a/src/Storages/tests/gtest_background_executor.cpp +++ b/src/Storages/tests/gtest_background_executor.cpp @@ -20,14 +20,14 @@ public: explicit TestJobExecutor(Context & context) :IBackgroundJobExecutor( context, - TaskSleepSettings{}, + BackgroundTaskSchedulingSettings{}, {PoolConfig{PoolType::MERGE_MUTATE, 4, CurrentMetrics::BackgroundPoolTask}}) {} protected: - String getBackgroundJobName() const override + String getBackgroundTaskName() const override { - return "TestJob"; + return "TestTask"; } std::optional getBackgroundJob() override From 518ba2ceb2acabfb28e5691b1525b4f96f1c448f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 16 Oct 2020 16:48:12 +0300 Subject: [PATCH 030/149] Safe random generator --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 7 ++++++- src/Storages/MergeTree/BackgroundJobsExecutor.h | 4 ++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index b888eec3dfe..4efc154faa0 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -29,6 +29,11 @@ IBackgroundJobExecutor::IBackgroundJobExecutor( } } +double IBackgroundJobExecutor::getSleepRandomAdd() +{ + std::lock_guard random_lock(random_mutex); + return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); +} void IBackgroundJobExecutor::scheduleTask() { @@ -39,7 +44,7 @@ void IBackgroundJobExecutor::scheduleTask() auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng)); + + getSleepRandomAdd()); scheduling_task->scheduleAfter(next_time_to_execute); } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index e2502f534e5..07155233ce6 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -83,6 +83,8 @@ private: BackgroundSchedulePool::TaskHolder scheduling_task; /// Mutex for thread safety std::mutex scheduling_task_mutex; + /// Mutex for pcg random generator thread safety + std::mutex random_mutex; public: /// These three functions are thread safe @@ -114,6 +116,8 @@ private: void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job void scheduleTask(); + /// Return random add for sleep in case of error + double getSleepRandomAdd(); }; /// Main jobs executor: merges, mutations, fetches and so on From 5254a5ded7bdc38446eb356570b097e771d67d89 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 18 Oct 2020 20:18:31 +0800 Subject: [PATCH 031/149] add setting aggregate_functions_null_for_empty --- src/Core/Settings.h | 3 +- src/Interpreters/TreeRewriter.cpp | 27 ++++++++++++++++++ ...gregate_functions_null_for_empty.reference | 8 ++++++ ...ing_aggregate_functions_null_for_empty.sql | 28 +++++++++++++++++++ 4 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference create mode 100644 tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d73098ca6e0..ac743768ff8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -400,7 +400,8 @@ class IColumn; M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \ M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ - M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) + M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ + M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0b2f8ac3eb7..2726594f51c 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -31,6 +31,7 @@ #include #include +#include namespace DB { @@ -110,6 +111,25 @@ struct CustomizeFunctionsSuffixData char ifDistinct[] = "ifdistinct"; using CustomizeIfDistinctVisitor = InDepthNodeVisitor>, true>; +/// Used to rewrite all aggregate functions to add -OrNull suffix to them if setting `aggregate_functions_null_for_empty` is set. +struct CustomizeAggregateFunctionsSuffixData +{ + using TypeToVisit = ASTFunction; + + const String & customized_func_suffix; + + void visit(ASTFunction & func, ASTPtr &) + { + if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name) + && !endsWith(func.name, customized_func_suffix)) + { + func.name = func.name + customized_func_suffix; + } + } +}; + +using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor, true>; + /// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form. /// Expand asterisks and qualified asterisks with column names. /// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer. @@ -692,6 +712,13 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings & CustomizeGlobalNotInVisitor(data_global_not_null_in).visit(query); } + // Rewrite all aggregate functions to add -OrNull suffix to them + if (settings.aggregate_functions_null_for_empty) + { + CustomizeAggregateFunctionsOrNullVisitor::Data data_or_null{"OrNull"}; + CustomizeAggregateFunctionsOrNullVisitor(data_or_null).visit(query); + } + /// Creates a dictionary `aliases`: alias -> ASTPtr QueryAliasesVisitor(aliases).visit(query); diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference new file mode 100644 index 00000000000..570ea20ffad --- /dev/null +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference @@ -0,0 +1,8 @@ +0 +\N +\N +\N +45 +45 +45 +45 diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql new file mode 100644 index 00000000000..c59b592e701 --- /dev/null +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql @@ -0,0 +1,28 @@ +DROP TABLE IF EXISTS defaults; + +CREATE TABLE defaults +( + n Int8 +)ENGINE = Memory(); + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +SET aggregate_functions_null_for_empty=1; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +INSERT INTO defaults SELECT * FROM numbers(10); + +SET aggregate_functions_null_for_empty=0; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +SET aggregate_functions_null_for_empty=1; + +SELECT sum(n) FROM defaults; +SELECT sumOrNull(n) FROM defaults; + +DROP TABLE defaults; From cb139abe2a0b393effa5338c959a7fb5c21d62c2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Oct 2020 11:47:15 +0300 Subject: [PATCH 032/149] Better schedule after job already running --- .../MergeTree/BackgroundJobsExecutor.cpp | 46 +++++++++---------- .../MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 4efc154faa0..59633e32279 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -35,12 +35,17 @@ double IBackgroundJobExecutor::getSleepRandomAdd() return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); } -void IBackgroundJobExecutor::scheduleTask() +void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) { - auto no_work_done_times = no_work_done_count.load(std::memory_order_relaxed); - /// If we have no jobs or some errors than sleep with backoff - if (no_work_done_times != 0) + if (job_done_or_has_job_to_do) { + no_work_done_count = 0; + /// We have background jobs, schedule task as soon as possible + scheduling_task->schedule(); + } + else + { + auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) @@ -48,12 +53,6 @@ void IBackgroundJobExecutor::scheduleTask() scheduling_task->scheduleAfter(next_time_to_execute); } - else - { - /// We have background jobs, schedule task as soon as possible - scheduling_task->schedule(); - } - } namespace @@ -79,7 +78,7 @@ void IBackgroundJobExecutor::jobExecutingTask() try { auto job_and_pool = getBackgroundJob(); - if (job_and_pool) /// If we have job, than try to assign into background pool + if (job_and_pool) /// If we have job, then try to assign into background pool { auto & pool_config = pools_configs[job_and_pool->pool_type]; /// If corresponding pool is not full increment metric and assign new job @@ -94,44 +93,43 @@ try job(); /// Job done, decrement metric and reset no_work counter CurrentMetrics::values[pool_config.tasks_metric]--; - no_work_done_count = 0; + /// Job done, new empty space in pool, schedule background task + scheduleTask(true); } catch (...) { - no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; + scheduleTask(false); } - /// Job done, new empty space in pool, schedule background task - scheduleTask(); }); + /// We've scheduled task in then background pool and when it will finish we will be triggered again. But this task can be + /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. + scheduleTask(true); } catch (...) { - /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety add catch here - no_work_done_count++; + /// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here tryLogCurrentException(__PRETTY_FUNCTION__); CurrentMetrics::values[pool_config.tasks_metric]--; - scheduleTask(); + scheduleTask(false); } } - else /// Pool is full and we have some work to do, let's try to schedule without backoff + else /// Pool is full and we have some work to do, let's schedule our task as fast as possible { - scheduleTask(); + scheduleTask(true); } } else /// Nothing to do, no jobs { - no_work_done_count++; - scheduleTask(); + scheduleTask(false); } } catch (...) /// Exception while we looking for a task, reschedule { - no_work_done_count++; tryLogCurrentException(__PRETTY_FUNCTION__); - scheduleTask(); + scheduleTask(false); } void IBackgroundJobExecutor::start() diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 07155233ce6..c9f6e3649e4 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -115,7 +115,7 @@ private: /// Function that executes in background scheduling pool void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job - void scheduleTask(); + void scheduleTask(bool job_done_or_has_job_to_do); /// Return random add for sleep in case of error double getSleepRandomAdd(); }; From 514a95c292019a60188b64e72b283b8f4e16e473 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 19 Oct 2020 12:06:51 +0300 Subject: [PATCH 033/149] Fixup --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 286ab93e6b8..0261f17bc55 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1464,7 +1464,7 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_executor.start(); + background_moves_executor.start(); } } From ce8ee01d754782c93e2e7104f22dfbcdeeaaefbf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 14:27:50 +0300 Subject: [PATCH 034/149] Fix background moves start --- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 7 +++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 574fcfe933e..c0cabee42ae 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -108,8 +108,7 @@ void StorageMergeTree::startup() try { background_executor.start(); - if (areBackgroundMovesNeeded()) - background_moves_executor.start(); + startBackgroundMovesIfNeeded(); } catch (...) { @@ -1465,7 +1464,8 @@ MutationCommands StorageMergeTree::getFirtsAlterMutationCommandsForPart(const Da void StorageMergeTree::startBackgroundMovesIfNeeded() { - background_moves_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 567bfbdb914..9ad573a4fbc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3483,9 +3483,7 @@ void StorageReplicatedMergeTree::startup() /// If we don't separate create/start steps, race condition will happen /// between the assignment of queue_task_handle and queueTask that use the queue_task_handle. background_executor.start(); - if (areBackgroundMovesNeeded()) - background_moves_executor.start(); - + startBackgroundMovesIfNeeded(); } catch (...) { @@ -5984,7 +5982,8 @@ MutationCommands StorageReplicatedMergeTree::getFirtsAlterMutationCommandsForPar void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() { - background_moves_executor.start(); + if (areBackgroundMovesNeeded()) + background_moves_executor.start(); } } From 7256d0157940c95559bdd47f7089addeb2c59b97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 16:11:01 +0300 Subject: [PATCH 035/149] Normal backoff value instead of 10 seconds --- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 6 ++++-- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 59633e32279..f2061c73b9e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -42,16 +42,18 @@ void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) no_work_done_count = 0; /// We have background jobs, schedule task as soon as possible scheduling_task->schedule(); + } else { auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); + auto next_time_to_execute = 1000 * (std::min( sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.task_sleep_seconds_when_no_work_min * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + getSleepRandomAdd()); - scheduling_task->scheduleAfter(next_time_to_execute); + scheduling_task->scheduleAfter(next_time_to_execute, false); } } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index c9f6e3649e4..8660583ec02 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -19,7 +19,7 @@ struct BackgroundTaskSchedulingSettings double thread_sleep_seconds_if_nothing_to_do = 0.1; /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 6; + double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; double task_sleep_seconds_when_no_work_multiplier = 1.1; From 30ca17b09b541fac0b53f34796c3a7729f2c6caf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Oct 2020 16:12:25 +0300 Subject: [PATCH 036/149] Comment to settings --- src/Storages/MergeTree/BackgroundJobsExecutor.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 8660583ec02..e7e1c982c70 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -14,16 +14,17 @@ namespace DB /// task to sleep according to settings. Look at scheduleTask function for details. struct BackgroundTaskSchedulingSettings { - double thread_sleep_seconds = 10; double thread_sleep_seconds_random_part = 1.0; double thread_sleep_seconds_if_nothing_to_do = 0.1; - - /// For exponential backoff. - double task_sleep_seconds_when_no_work_min = 10; double task_sleep_seconds_when_no_work_max = 600; + /// For exponential backoff. double task_sleep_seconds_when_no_work_multiplier = 1.1; double task_sleep_seconds_when_no_work_random_part = 1.0; + + /// deprected settings, don't affect background execution + double thread_sleep_seconds = 10; + double task_sleep_seconds_when_no_work_min = 10; }; /// Pool type where we must execute new job. Each background executor can have several From c77b154398dc2becbc15c87c6fff06eca6e4cbda Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 21 Oct 2020 00:10:55 +0300 Subject: [PATCH 037/149] Better locks in merge tree --- src/Storages/StorageMergeTree.cpp | 49 +++++++++++++------------------ src/Storages/StorageMergeTree.h | 8 ++--- 2 files changed, 25 insertions(+), 32 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c0cabee42ae..2c96c96af07 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -214,16 +214,13 @@ void StorageMergeTree::drop() dropAllData(); } -void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder & lock_holder) +void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { - lock_holder.release(); { /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); - /// NOTE: It's assumed that this method is called under lockForAlter. - auto parts_to_remove = getDataPartsVector(); removePartsFromWorkingSet(parts_to_remove, true); @@ -613,9 +610,9 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason) +std::optional StorageMergeTree::selectPartsToMerge( + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -714,18 +711,18 @@ bool StorageMergeTree::merge( bool deduplicate, String * out_disable_reason) { + auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason); + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder); if (!merge_mutate_entry) return false; - return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry); + return mergeSelectedParts(metadata_snapshot, deduplicate, *merge_mutate_entry, table_lock_holder); } -bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry) +bool StorageMergeTree::mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; Stopwatch stopwatch; MutableDataPartPtr new_part; @@ -769,9 +766,8 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } -std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */) +std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -850,9 +846,8 @@ std::optional StorageMergeTree::sele return {}; } -bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry) +bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) { - auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto & future_part = merge_mutate_entry.future_part; auto table_id = getStorageID(); @@ -904,32 +899,30 @@ std::optional StorageMergeTree::getDataProcessingJob() auto metadata_snapshot = getInMemoryMetadataPtr(); std::optional merge_entry, mutate_entry; - merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr); + auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock); if (!merge_entry) - mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr); + mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr, share_lock); if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}, share_lock] () mutable { if (merge_entry) - mergeSelectedParts(metadata_snapshot, false, *merge_entry); + mergeSelectedParts(metadata_snapshot, false, *merge_entry, share_lock); else if (mutate_entry) - mutateSelectedPart(metadata_snapshot, *mutate_entry); + mutateSelectedPart(metadata_snapshot, *mutate_entry, share_lock); }, PoolType::MERGE_MUTATE}; } else if (auto lock = time_after_previous_cleanup.compareAndRestartDeferred(1)) { - return JobAndPool{[this] () + return JobAndPool{[this, share_lock] () { - { - auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); - /// All use relative_data_path which changes during rename - /// so execute under share lock. - clearOldPartsFromFilesystem(); - clearOldTemporaryDirectories(); - clearOldWriteAheadLogs(); - } + /// All use relative_data_path which changes during rename + /// so execute under share lock. + clearOldPartsFromFilesystem(); + clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); clearOldMutations(); }, PoolType::MERGE_MUTATE}; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b3d281e77b7..12de2d186ab 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -150,11 +150,11 @@ private: MutationCommands commands; }; - std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason); - bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); - std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason); - bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry); + std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); + bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( const DataPartPtr & part, From 9f47c82bbdbcd7406d6df535f3e4a2d29cc07826 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 21 Oct 2020 11:31:57 +0300 Subject: [PATCH 038/149] Sleep a little when pool is full --- .../MergeTree/BackgroundJobsExecutor.cpp | 30 ++++++++++++------- .../MergeTree/BackgroundJobsExecutor.h | 2 +- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index f2061c73b9e..5aba208a86e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -35,9 +35,9 @@ double IBackgroundJobExecutor::getSleepRandomAdd() return std::uniform_real_distribution(0, sleep_settings.task_sleep_seconds_when_no_work_random_part)(rng); } -void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) +void IBackgroundJobExecutor::scheduleTask(bool job_done, bool with_backoff) { - if (job_done_or_has_job_to_do) + if (job_done) { no_work_done_count = 0; /// We have background jobs, schedule task as soon as possible @@ -46,14 +46,22 @@ void IBackgroundJobExecutor::scheduleTask(bool job_done_or_has_job_to_do) } else { - auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); + size_t next_time_to_execute; + if (with_backoff) + { + auto no_work_done_times = no_work_done_count.fetch_add(1, std::memory_order_relaxed); - auto next_time_to_execute = 1000 * (std::min( - sleep_settings.task_sleep_seconds_when_no_work_max, - sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) - + getSleepRandomAdd()); + next_time_to_execute = 1000 * (std::min( + sleep_settings.task_sleep_seconds_when_no_work_max, + sleep_settings.thread_sleep_seconds_if_nothing_to_do * std::pow(sleep_settings.task_sleep_seconds_when_no_work_multiplier, no_work_done_times)) + + getSleepRandomAdd()); + } + else + { + next_time_to_execute = 1000 * sleep_settings.thread_sleep_seconds_if_nothing_to_do; + } - scheduling_task->scheduleAfter(next_time_to_execute, false); + scheduling_task->scheduleAfter(next_time_to_execute, false); } } @@ -105,7 +113,7 @@ try scheduleTask(false); } }); - /// We've scheduled task in then background pool and when it will finish we will be triggered again. But this task can be + /// We've scheduled task in the background pool and when it will finish we will be triggered again. But this task can be /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. scheduleTask(true); } @@ -117,9 +125,9 @@ try scheduleTask(false); } } - else /// Pool is full and we have some work to do, let's schedule our task as fast as possible + else /// Pool is full and we have some work to do { - scheduleTask(true); + scheduleTask(false, /* with_backoff = */ false); } } else /// Nothing to do, no jobs diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index e7e1c982c70..537a6064c86 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -116,7 +116,7 @@ private: /// Function that executes in background scheduling pool void jobExecutingTask(); /// Recalculate timeouts when we have to check for a new job - void scheduleTask(bool job_done_or_has_job_to_do); + void scheduleTask(bool job_done, bool with_backoff=false); /// Return random add for sleep in case of error double getSleepRandomAdd(); }; From eb0e3a83d0b0ecb8f20fc9260ad4d8da02f22863 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:02:26 +0300 Subject: [PATCH 039/149] Support SNI in https connections to remote resources --- src/IO/HTTPCommon.cpp | 19 ++++++++++--------- src/IO/HTTPCommon.h | 7 ++++++- src/IO/ReadWriteBufferFromHTTP.cpp | 1 - 3 files changed, 16 insertions(+), 11 deletions(-) delete mode 100644 src/IO/ReadWriteBufferFromHTTP.cpp diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 6b7f30cd9b6..bda615edcd5 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -20,6 +20,7 @@ # include # include # include +# include #endif #include @@ -68,27 +69,27 @@ namespace throw Exception("Unsupported scheme in URI '" + uri.toString() + "'", ErrorCodes::UNSUPPORTED_URI_SCHEME); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host=true) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host = true) { HTTPSessionPtr session; if (https) + { #if USE_SSL - session = std::make_shared(); + /// Cannot resolve host in advance, otherwise SNI won't work in Poco. + session = std::make_shared(host, port); #else throw Exception("ClickHouse was built without HTTPS support", ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME); #endif + } else - session = std::make_shared(); + { + String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; + session = std::make_shared(resolved_host, port); + } ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - if (resolve_host) - session->setHost(DNSResolver::instance().resolveHost(host).toString()); - else - session->setHost(host); - session->setPort(port); - /// doesn't work properly without patch #if defined(POCO_CLICKHOUSE_PATCH) session->setKeepAlive(keep_alive); diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 66764b1c805..db0abe8fc6e 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -13,6 +13,7 @@ #include + namespace Poco { namespace Net @@ -24,6 +25,7 @@ namespace Net namespace DB { + constexpr int HTTP_TOO_MANY_REQUESTS = 429; class SingleEndpointHTTPSessionPool : public PoolBase @@ -39,6 +41,7 @@ private: public: SingleEndpointHTTPSessionPool(const std::string & host_, UInt16 port_, bool https_, size_t max_pool_size_); }; + using PooledHTTPSessionPtr = SingleEndpointHTTPSessionPool::Entry; using HTTPSessionPtr = std::shared_ptr; @@ -59,5 +62,7 @@ bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status); */ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, bool allow_redirects); -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects = false); + +void assertResponseIsOk( + const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr, const bool allow_redirects = false); } diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp deleted file mode 100644 index 4d046bfe2c6..00000000000 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ /dev/null @@ -1 +0,0 @@ -#include From ec3f19b79e0b967436272fd2c753a9f5ecbe0cd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:25:54 +0300 Subject: [PATCH 040/149] Add a test --- .../0_stateless/01532_client_sni.reference | 1 + tests/queries/0_stateless/01532_client_sni.sh | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01532_client_sni.reference create mode 100755 tests/queries/0_stateless/01532_client_sni.sh diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference new file mode 100644 index 00000000000..7fb5e6c6321 --- /dev/null +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -0,0 +1 @@ +\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh new file mode 100755 index 00000000000..9bfaf2d5427 --- /dev/null +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +# Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. + +strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | + grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' +# ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +# | | | | | +# server name data | hostname | y a n d e x . r u +# extension id len: 14 | type | +# | | +# hostnames list hostname +# len, 14 len, 9 From 237384bad99d4c27fd604398860e15af03ea74ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Oct 2020 05:26:48 +0300 Subject: [PATCH 041/149] Fix "Arcadia" --- src/IO/ya.make | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/ya.make b/src/IO/ya.make index 28099818b46..ad6833bc2ef 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -44,7 +44,6 @@ SRCS( ReadBufferFromPocoSocket.cpp readFloatText.cpp ReadHelpers.cpp - ReadWriteBufferFromHTTP.cpp SeekAvoidingReadBuffer.cpp UseSSL.cpp WriteBufferAIO.cpp From 608c7383e92836bded9e97db5be98fc16bf27974 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Oct 2020 05:27:23 +0300 Subject: [PATCH 042/149] Update 01532_client_sni.sh --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 9bfaf2d5427..51693639a37 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -3,7 +3,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -# Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. +# Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' From 86de5962d2d7ff5743bc938f21c3b365405aabb3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Oct 2020 05:27:47 +0300 Subject: [PATCH 043/149] Update 01532_client_sni.sh --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 51693639a37..9284661dbc0 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -13,4 +13,4 @@ strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * # extension id len: 14 | type | # | | # hostnames list hostname -# len, 14 len, 9 +# len, 12 len, 9 From 39e47c5338587db1dea86f4929f8caf08494ac3b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 11:54:00 +0300 Subject: [PATCH 044/149] Less strange lambdas --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 17 ++- src/Storages/StorageMergeTree.cpp | 116 ++++++++---------- src/Storages/StorageMergeTree.h | 31 ++++- src/Storages/StorageReplicatedMergeTree.cpp | 16 ++- src/Storages/StorageReplicatedMergeTree.h | 4 +- 8 files changed, 106 insertions(+), 86 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b20b5e19e1a..1a7062766b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3630,7 +3630,7 @@ std::optional MergeTreeData::getDataMovingJob() if (moving_tagger->parts_to_move.empty()) return {}; - return JobAndPool{[this, moving_tagger{std::move(moving_tagger)}] () mutable + return JobAndPool{[this, moving_tagger] () mutable { moveParts(moving_tagger); }, PoolType::MOVE}; diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 332a0988d10..a1afadec7fa 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -16,7 +16,7 @@ namespace DB struct MergeTreeMoveEntry { std::shared_ptr part; - std::shared_ptr reserved_space; + ReservationPtr reserved_space; MergeTreeMoveEntry(const std::shared_ptr & part_, ReservationPtr reservation_) : part(part_), reserved_space(std::move(reservation_)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f0732774c3e..95358ecee97 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1259,7 +1259,7 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() } -ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data) +ReplicatedMergeTreeQueue::SelectedEntryPtr ReplicatedMergeTreeQueue::selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data) { LogEntryPtr entry; @@ -1286,7 +1286,7 @@ ReplicatedMergeTreeQueue::SelectedEntry ReplicatedMergeTreeQueue::selectEntryToP } if (entry) - return { entry, std::shared_ptr{ new CurrentlyExecuting(entry, *this) } }; + return std::make_shared(entry, std::unique_ptr{ new CurrentlyExecuting(entry, *this) }); else return {}; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index c72569a5071..ead97579a4f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -259,6 +259,8 @@ private: ~CurrentlyExecuting(); }; + using CurrentlyExecutingPtr = std::unique_ptr; + public: ReplicatedMergeTreeQueue(StorageReplicatedMergeTree & storage_); ~ReplicatedMergeTreeQueue(); @@ -319,8 +321,19 @@ public: /** Select the next action to process. * merger_mutator is used only to check if the merges are not suspended. */ - using SelectedEntry = std::pair>; - SelectedEntry selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); + struct SelectedEntry + { + ReplicatedMergeTreeQueue::LogEntryPtr log_entry; + CurrentlyExecutingPtr currently_executing_holder; + + SelectedEntry(const ReplicatedMergeTreeQueue::LogEntryPtr & log_entry_, CurrentlyExecutingPtr && currently_executing_holder_) + : log_entry(log_entry_) + , currently_executing_holder(std::move(currently_executing_holder_)) + {} + }; + + using SelectedEntryPtr = std::shared_ptr; + SelectedEntryPtr selectEntryToProcess(MergeTreeDataMergerMutator & merger_mutator, MergeTreeData & data); /** Execute `func` function to handle the action. * In this case, at runtime, mark the queue element as running diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0e0cc13b0c7..11e8859e76c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -309,72 +309,62 @@ void StorageMergeTree::alter( /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. -struct CurrentlyMergingPartsTagger +StorageMergeTree::CurrentlyMergingPartsTagger::CurrentlyMergingPartsTagger( + FutureMergedMutatedPart & future_part_, + size_t total_size, + StorageMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot, + bool is_mutation) + : future_part(future_part_), storage(storage_) { - FutureMergedMutatedPart future_part; - ReservationPtr reserved_space; + /// Assume mutex is already locked, because this method is called from mergeTask. - StorageMergeTree & storage; - -public: - CurrentlyMergingPartsTagger( - FutureMergedMutatedPart & future_part_, - size_t total_size, - StorageMergeTree & storage_, - const StorageMetadataPtr & metadata_snapshot, - bool is_mutation) - : future_part(future_part_), storage(storage_) + /// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks + if (is_mutation) + reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->volume); + else { - /// Assume mutex is already locked, because this method is called from mergeTask. + IMergeTreeDataPart::TTLInfos ttl_infos; + size_t max_volume_index = 0; + for (auto & part_ptr : future_part_.parts) + { + ttl_infos.update(part_ptr->ttl_infos); + max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); + } - /// if we mutate part, than we should reserve space on the same disk, because mutations possible can create hardlinks + reserved_space = storage.tryReserveSpacePreferringTTLRules(metadata_snapshot, total_size, ttl_infos, time(nullptr), max_volume_index); + } + if (!reserved_space) + { if (is_mutation) - reserved_space = storage.tryReserveSpace(total_size, future_part_.parts[0]->volume); + throw Exception("Not enough space for mutating part '" + future_part_.parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE); else - { - IMergeTreeDataPart::TTLInfos ttl_infos; - size_t max_volume_index = 0; - for (auto & part_ptr : future_part_.parts) - { - ttl_infos.update(part_ptr->ttl_infos); - max_volume_index = std::max(max_volume_index, storage.getStoragePolicy()->getVolumeIndexByDisk(part_ptr->volume->getDisk())); - } - - reserved_space = storage.tryReserveSpacePreferringTTLRules(metadata_snapshot, total_size, ttl_infos, time(nullptr), max_volume_index); - } - if (!reserved_space) - { - if (is_mutation) - throw Exception("Not enough space for mutating part '" + future_part_.parts[0]->name + "'", ErrorCodes::NOT_ENOUGH_SPACE); - else - throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE); - } - - future_part_.updatePath(storage, reserved_space); - - for (const auto & part : future_part.parts) - { - if (storage.currently_merging_mutating_parts.count(part)) - throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - } - storage.currently_merging_mutating_parts.insert(future_part.parts.begin(), future_part.parts.end()); + throw Exception("Not enough space for merging parts", ErrorCodes::NOT_ENOUGH_SPACE); } - ~CurrentlyMergingPartsTagger() + future_part_.updatePath(storage, reserved_space); + + for (const auto & part : future_part.parts) { - std::lock_guard lock(storage.currently_processing_in_background_mutex); - - for (const auto & part : future_part.parts) - { - if (!storage.currently_merging_mutating_parts.count(part)) - std::terminate(); - storage.currently_merging_mutating_parts.erase(part); - } - - storage.currently_processing_in_background_condition.notify_all(); + if (storage.currently_merging_mutating_parts.count(part)) + throw Exception("Tagging already tagged part " + part->name + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); } -}; + storage.currently_merging_mutating_parts.insert(future_part.parts.begin(), future_part.parts.end()); +} +StorageMergeTree::CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() +{ + std::lock_guard lock(storage.currently_processing_in_background_mutex); + + for (const auto & part : future_part.parts) + { + if (!storage.currently_merging_mutating_parts.count(part)) + std::terminate(); + storage.currently_merging_mutating_parts.erase(part); + } + + storage.currently_processing_in_background_condition.notify_all(); +} Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String & mutation_file_name) { @@ -643,7 +633,7 @@ void StorageMergeTree::loadMutations() increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); } -std::optional StorageMergeTree::selectPartsToMerge( +std::shared_ptr StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) { std::unique_lock lock(currently_processing_in_background_mutex); @@ -733,8 +723,8 @@ std::optional StorageMergeTree::sele return {}; } - merging_tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); - return MergeMutateSelectedEntry{future_part, std::move(merging_tagger), {}}; + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false); + return std::make_shared(future_part, std::move(merging_tagger), MutationCommands{}); } bool StorageMergeTree::merge( @@ -799,7 +789,7 @@ bool StorageMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & p return currently_merging_mutating_parts.count(part); } -std::optional StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) +std::shared_ptr StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */, TableLockHolder & /* table_lock_holder */) { std::lock_guard lock(currently_processing_in_background_mutex); size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements; @@ -873,8 +863,8 @@ std::optional StorageMergeTree::sele future_part.name = part->getNewName(new_part_info); future_part.type = part->getType(); - tagger = std::make_shared(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); - return MergeMutateSelectedEntry{future_part, std::move(tagger), commands}; + tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace({part}), *this, metadata_snapshot, true); + return std::make_shared(future_part, std::move(tagger), commands); } return {}; } @@ -930,7 +920,7 @@ std::optional StorageMergeTree::getDataProcessingJob() return {}; auto metadata_snapshot = getInMemoryMetadataPtr(); - std::optional merge_entry, mutate_entry; + std::shared_ptr merge_entry, mutate_entry; auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); merge_entry = selectPartsToMerge(metadata_snapshot, false, {}, false, nullptr, share_lock); @@ -939,7 +929,7 @@ std::optional StorageMergeTree::getDataProcessingJob() if (merge_entry || mutate_entry) { - return JobAndPool{[this, metadata_snapshot, merge_entry{std::move(merge_entry)}, mutate_entry{std::move(mutate_entry)}, share_lock] () mutable + return JobAndPool{[this, metadata_snapshot, merge_entry, mutate_entry, share_lock] () mutable { if (merge_entry) mergeSelectedParts(metadata_snapshot, false, *merge_entry, share_lock); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 43982ddbc78..71a32fbd203 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -21,8 +21,6 @@ namespace DB { -struct CurrentlyMergingPartsTagger; - /** See the description of the data structure in MergeTreeData. */ class StorageMergeTree final : public ext::shared_ptr_helper, public MergeTreeData @@ -140,21 +138,42 @@ private: /// Wait until mutation with version will finish mutation for all parts void waitForMutation(Int64 version, const String & file_name); - friend struct CurrentlyMergingPartsTagger; + struct CurrentlyMergingPartsTagger + { + FutureMergedMutatedPart future_part; + ReservationPtr reserved_space; - using CurrentlyMergingPartsTaggerPtr = std::shared_ptr; + StorageMergeTree & storage; + + CurrentlyMergingPartsTagger( + FutureMergedMutatedPart & future_part_, + size_t total_size, + StorageMergeTree & storage_, + const StorageMetadataPtr & metadata_snapshot, + bool is_mutation); + + ~CurrentlyMergingPartsTagger(); + }; + + using CurrentlyMergingPartsTaggerPtr = std::unique_ptr; + friend struct CurrentlyMergingPartsTagger; struct MergeMutateSelectedEntry { FutureMergedMutatedPart future_part; CurrentlyMergingPartsTaggerPtr tagger; MutationCommands commands; + MergeMutateSelectedEntry(const FutureMergedMutatedPart & future_part_, CurrentlyMergingPartsTaggerPtr && tagger_, const MutationCommands & commands_) + : future_part(future_part_) + , tagger(std::move(tagger_)) + , commands(commands_) + {} }; - std::optional selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); - std::optional selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5d670ab2edf..53f2ff14b3b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2540,10 +2540,10 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() } } -ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueEntry() +ReplicatedMergeTreeQueue::SelectedEntryPtr StorageReplicatedMergeTree::selectQueueEntry() { /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected; + ReplicatedMergeTreeQueue::SelectedEntryPtr selected; try { @@ -2557,10 +2557,10 @@ ReplicatedMergeTreeQueue::SelectedEntry StorageReplicatedMergeTree::selectQueueE return selected; } -bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & selected_entry) +bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry) { - LogEntryPtr & entry = selected_entry.first; + LogEntryPtr & entry = selected_entry->log_entry; return queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry_to_process) { try @@ -2609,14 +2609,12 @@ std::optional StorageReplicatedMergeTree::getDataProcessingJob() return {}; /// This object will mark the element of the queue as running. - ReplicatedMergeTreeQueue::SelectedEntry selected_entry = selectQueueEntry(); + ReplicatedMergeTreeQueue::SelectedEntryPtr selected_entry = selectQueueEntry(); - LogEntryPtr & entry = selected_entry.first; - - if (!entry) + if (!selected_entry) return {}; - return JobAndPool{[this, selected_entry{std::move(selected_entry)}] () mutable + return JobAndPool{[this, selected_entry] () mutable { processQueueEntry(selected_entry); }, PoolType::MERGE_MUTATE}; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 277c1302540..92e17412ecc 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -418,9 +418,9 @@ private: void cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper); - ReplicatedMergeTreeQueue::SelectedEntry selectQueueEntry(); + ReplicatedMergeTreeQueue::SelectedEntryPtr selectQueueEntry(); - bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntry & entry); + bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry); /// Postcondition: /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) From 166b3de466a832e56863094df9f391f2a1767c05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 13:31:33 +0300 Subject: [PATCH 045/149] Make insert test faster --- tests/performance/insert_sequential.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential.xml index 0c98c04ec8b..950a6c0c29c 100644 --- a/tests/performance/insert_sequential.xml +++ b/tests/performance/insert_sequential.xml @@ -6,6 +6,6 @@ 1 CREATE TABLE t (x UInt64) ENGINE = MergeTree() ORDER BY tuple() - INSERT INTO t SELECT * FROM numbers(20000) + INSERT INTO t SELECT * FROM numbers(3000) DROP TABLE IF EXISTS t From 84dbbdb312e5bf70aaa063799a0ebb5019b6919f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Oct 2020 13:42:15 +0300 Subject: [PATCH 046/149] Rename test --- ...sequential.xml => insert_sequential_and_background_merges.xml} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/performance/{insert_sequential.xml => insert_sequential_and_background_merges.xml} (100%) diff --git a/tests/performance/insert_sequential.xml b/tests/performance/insert_sequential_and_background_merges.xml similarity index 100% rename from tests/performance/insert_sequential.xml rename to tests/performance/insert_sequential_and_background_merges.xml From d908cddb494498f4fcf27929892d999f2402792c Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 23 Oct 2020 22:08:38 +0300 Subject: [PATCH 047/149] done --- src/Interpreters/ActionsVisitor.cpp | 10 +++++++--- src/Interpreters/ActionsVisitor.h | 4 +++- src/Interpreters/ExpressionAnalyzer.cpp | 16 +++++++++++++--- src/Interpreters/ExpressionAnalyzer.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 5 files changed, 27 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e0e921b003b..2f183d7dd93 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -385,7 +385,7 @@ ActionsMatcher::Data::Data( const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_) + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having_) : context(context_) , set_size_limit(set_size_limit_) , subquery_depth(subquery_depth_) @@ -396,6 +396,7 @@ ActionsMatcher::Data::Data( , no_makeset(no_makeset_) , only_consts(only_consts_) , no_storage_or_local(no_storage_or_local_) + , has_having(has_having_) , visit_depth(0) , actions_stack(std::move(actions), context) , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) @@ -944,12 +945,15 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su SetPtr set = std::make_shared(data.set_size_limit, false, data.context.getSettingsRef().transform_null_in); - /** The following happens for GLOBAL INs: + /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. + * + * In case that we have HAVING with IN subquery, we have to force creating set for it. + * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ - if (!subquery_for_set.source && data.no_storage_or_local) + if ((!subquery_for_set.source && data.no_storage_or_local) || data.has_having) { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_unique(); diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index f4da9932163..0179e1fd09e 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -118,6 +118,7 @@ public: bool no_makeset; bool only_consts; bool no_storage_or_local; + bool has_having; size_t visit_depth; ScopeStack actions_stack; @@ -131,7 +132,8 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_); + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, + bool has_having); /// Does result of the calculation already exists in the block. bool hasColumn(const String & column_name) const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f79bb36ec46..b852ab75e1f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -396,7 +396,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage()); + no_subqueries, false, only_consts, !isRemoteStorage(), false); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -407,7 +407,17 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, true, only_consts, !isRemoteStorage()); + no_subqueries, true, only_consts, !isRemoteStorage(), false); + ActionsVisitor(visitor_data, log.stream()).visit(ast); + actions = visitor_data.getActions(); +} + +void ExpressionAnalyzer::getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) +{ + LogAST log; + ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, + sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, + no_subqueries, false, only_consts, !isRemoteStorage(), true); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -825,7 +835,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); step.required_output.push_back(select_query->having()->getColumnName()); - getRootActions(select_query->having(), only_types, step.actions()); + getRootActionsHasHaving(select_query->having(), only_types, step.actions()); return true; } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 6389d8a142c..622c5204257 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -151,6 +151,8 @@ protected: */ void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, * or after all the actions that are normally performed before aggregation. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d9821be4e4e..07c1942c08d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -958,7 +958,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu preliminary_sort(); // If there is no global subqueries, we can run subqueries only when receive them on server. - if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) + if (expressions.hasHaving() || (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } @@ -1071,7 +1071,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } } - if (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) + if (expressions.hasHaving() || (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } From 7822dafcae3b19f3a4500736776c82b5a9e05dba Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 23 Oct 2020 22:37:54 +0300 Subject: [PATCH 048/149] test added --- .../01532_having_with_totals.reference | 12 +++++ .../0_stateless/01532_having_with_totals.sql | 45 +++++++++++++++++++ 2 files changed, 57 insertions(+) create mode 100644 tests/queries/0_stateless/01532_having_with_totals.reference create mode 100644 tests/queries/0_stateless/01532_having_with_totals.sql diff --git a/tests/queries/0_stateless/01532_having_with_totals.reference b/tests/queries/0_stateless/01532_having_with_totals.reference new file mode 100644 index 00000000000..2087369cae7 --- /dev/null +++ b/tests/queries/0_stateless/01532_having_with_totals.reference @@ -0,0 +1,12 @@ +127.0.0.{1,2} +1 + +0 +127.0.0.1 +1 + +0 +with explicit having +1 2 + +0 2 diff --git a/tests/queries/0_stateless/01532_having_with_totals.sql b/tests/queries/0_stateless/01532_having_with_totals.sql new file mode 100644 index 00000000000..00b8987fd83 --- /dev/null +++ b/tests/queries/0_stateless/01532_having_with_totals.sql @@ -0,0 +1,45 @@ +drop table if exists local_t; +create table local_t engine Log as select 1 a; + +SELECT '127.0.0.{1,2}'; +SELECT * +FROM +( + SELECT a + FROM remote('127.0.0.{1,2}', default, local_t) + GROUP BY a + WITH TOTALS +) +WHERE a IN +( + SELECT 1 +); + +SELECT '127.0.0.1'; +SELECT * +FROM +( + SELECT a + FROM remote('127.0.0.1', default, local_t) + GROUP BY a + WITH TOTALS +) +WHERE a IN +( + SELECT 1 +); + +SELECT 'with explicit having'; +SELECT + a, + count() +FROM remote('127.0.0.{1,2}', default, t) +GROUP BY a + WITH TOTALS +HAVING a IN +( + SELECT 1 +); + + +drop table if exists local_t; \ No newline at end of file From fe9440689a8bf9d2ef173e8e6aac1e787867ca7d Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Sat, 24 Oct 2020 00:05:24 +0300 Subject: [PATCH 049/149] better --- src/Interpreters/ActionsVisitor.h | 2 +- tests/queries/0_stateless/01532_having_with_totals.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 0179e1fd09e..8c0b56f0c3c 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -132,7 +132,7 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having); /// Does result of the calculation already exists in the block. diff --git a/tests/queries/0_stateless/01532_having_with_totals.sql b/tests/queries/0_stateless/01532_having_with_totals.sql index 00b8987fd83..10f55c8c135 100644 --- a/tests/queries/0_stateless/01532_having_with_totals.sql +++ b/tests/queries/0_stateless/01532_having_with_totals.sql @@ -6,7 +6,7 @@ SELECT * FROM ( SELECT a - FROM remote('127.0.0.{1,2}', default, local_t) + FROM remote('127.0.0.{1,2}', currentDatabase(), local_t) GROUP BY a WITH TOTALS ) @@ -20,7 +20,7 @@ SELECT * FROM ( SELECT a - FROM remote('127.0.0.1', default, local_t) + FROM remote('127.0.0.1', currentDatabase(), local_t) GROUP BY a WITH TOTALS ) @@ -33,7 +33,7 @@ SELECT 'with explicit having'; SELECT a, count() -FROM remote('127.0.0.{1,2}', default, t) +FROM remote('127.0.0.{1,2}', currentDatabase(), local_t) GROUP BY a WITH TOTALS HAVING a IN From c6d450f3b5c9dab5f83f4e4c39976d89106d01ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 09:43:29 +0300 Subject: [PATCH 050/149] Weird test --- tests/queries/0_stateless/01532_client_sni.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 9bfaf2d5427..a3bc7be884e 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly use SNI extension in Client Hello packet in HTTPS connection. -strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru/', RawBLOB, 'data String')" 2>&1 | +sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' + +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:8443/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | @@ -14,3 +16,5 @@ strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * # | | # hostnames list hostname # len, 14 len, 9 + +sudo bash -c 'sed -i.bak "/yandex\.ru/d" /etc/hosts' From 7187490f98a37eeb9481f5d65de88d7f5897ed40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 09:45:51 +0300 Subject: [PATCH 051/149] Weird test --- tests/queries/0_stateless/01532_client_sni.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index f9a8a131780..8a62cccf381 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:8443/', RawBLOB, 'data String')" 2>&1 | +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | From 81f492f8b82ef73acfc1637c2edffa6d28db7998 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Oct 2020 21:50:01 +0300 Subject: [PATCH 052/149] Improve test --- tests/queries/0_stateless/01532_client_sni.reference | 2 +- tests/queries/0_stateless/01532_client_sni.sh | 10 +++------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference index 7fb5e6c6321..4bf62e12856 100644 --- a/tests/queries/0_stateless/01532_client_sni.reference +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -1 +1 @@ -\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75 +\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 8a62cccf381..92c2f6be233 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,16 +5,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. -sudo bash -c 'echo "127.0.0.1 yandex.ru" >> /etc/hosts' - -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://yandex.ru:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | - grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x79\x61\x6e\x64\x65\x78\x2e\x72\x75' +echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | + grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | -# server name data | hostname | y a n d e x . r u +# server name data | hostname | l o c a l h o s t # extension id len: 14 | type | # | | # hostnames list hostname # len, 12 len, 9 - -sudo bash -c 'sed -i.bak "/yandex\.ru/d" /etc/hosts' From fb5d281c69d3e546545e2fcaaca431f98f0cb2a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Oct 2020 05:00:30 +0300 Subject: [PATCH 053/149] Better test --- tests/queries/0_stateless/01532_client_sni.reference | 2 +- tests/queries/0_stateless/01532_client_sni.sh | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference index 4bf62e12856..879b5791c7b 100644 --- a/tests/queries/0_stateless/01532_client_sni.reference +++ b/tests/queries/0_stateless/01532_client_sni.reference @@ -1 +1 @@ -\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74 +0000000e000c0000096c6f63616c686f7374 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index 92c2f6be233..b3f2a12bf22 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -5,8 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. -echo -ne 'y\r\n' | strace -f -x -s10000 -e trace=write,sendto ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:${CLICKHOUSE_PORT_HTTPS}/', RawBLOB, 'data String')" 2>&1 | - grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' +nc -q0 -l 5678 | xxd -p | grep -oF $'0000000e000c0000096c6f63616c686f7374' & + +${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:5678/', RawBLOB, 'data String')" 2>&1 | grep -v -F 'Timeout' + +# grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' # ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ # | | | | | # server name data | hostname | l o c a l h o s t From 7a65ef8c6704f5862ecfa9efa453bd5338572a1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Oct 2020 05:01:04 +0300 Subject: [PATCH 054/149] Better test --- tests/queries/0_stateless/01532_client_sni.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh index b3f2a12bf22..0b122713fde 100755 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ b/tests/queries/0_stateless/01532_client_sni.sh @@ -17,3 +17,5 @@ ${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:5678/', RawBLO # | | # hostnames list hostname # len, 12 len, 9 + +wait From 2aa03627655ea4cd28a29e7690d9d6a2302cdeb0 Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Sat, 31 Oct 2020 08:45:53 -0400 Subject: [PATCH 055/149] Add farmFingerprint64 function --- src/Functions/FunctionsHashing.cpp | 1 + src/Functions/FunctionsHashing.h | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/src/Functions/FunctionsHashing.cpp b/src/Functions/FunctionsHashing.cpp index 900455a1015..5983e97a093 100644 --- a/src/Functions/FunctionsHashing.cpp +++ b/src/Functions/FunctionsHashing.cpp @@ -18,6 +18,7 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 678214cdbad..17165e12e37 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -439,6 +439,18 @@ struct ImplCityHash64 static constexpr bool use_int_hash_for_pods = true; }; +// see farmhash.h for definition of NAMESPACE_FOR_HASH_FUNCTIONS +struct ImplFarmFingerprint64 +{ + static constexpr auto name = "farmFingerprint64"; + using ReturnType = UInt64; + using uint128_t = NAMESPACE_FOR_HASH_FUNCTIONS::uint128_t; + + static auto combineHashes(UInt64 h1, UInt64 h2) { return NAMESPACE_FOR_HASH_FUNCTIONS::Fingerprint(uint128_t(h1, h2)); } + static auto apply(const char * s, const size_t len) { return NAMESPACE_FOR_HASH_FUNCTIONS::Fingerprint64(s, len); } + static constexpr bool use_int_hash_for_pods = true; +}; + // see farmhash.h for definition of NAMESPACE_FOR_HASH_FUNCTIONS struct ImplFarmHash64 { @@ -1316,6 +1328,7 @@ using FunctionSHA256 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionStringHashFixedString; using FunctionCityHash64 = FunctionAnyHash; +using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; using FunctionMetroHash64 = FunctionAnyHash; From c9ad4e8d96bce84382653756997b7324f50c281e Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Sat, 31 Oct 2020 09:04:18 -0400 Subject: [PATCH 056/149] Add farmFingerprint64 docs --- docs/en/sql-reference/functions/hash-functions.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 73e9ef6254e..3594737c18a 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -153,15 +153,18 @@ A fast, decent-quality non-cryptographic hash function for a string obtained fro `URLHash(s, N)` – Calculates a hash from a string up to the N level in the URL hierarchy, without one of the trailing symbols `/`,`?` or `#` at the end, if present. Levels are the same as in URLHierarchy. This function is specific to Yandex.Metrica. +## farmFingerprint64 {#farmfingerprint64} + ## farmHash64 {#farmhash64} -Produces a 64-bit [FarmHash](https://github.com/google/farmhash) hash value. +Produces a 64-bit [FarmHash](https://github.com/google/farmhash) or Fingerprint value. Prefer `farmFingerprint64` for a stable and portable value. ``` sql +farmFingerprint64(par1, ...) farmHash64(par1, ...) ``` -The function uses the `Hash64` method from all [available methods](https://github.com/google/farmhash/blob/master/src/farmhash.h). +These functions use the `Fingerprint64` and `Hash64` method respectively from all [available methods](https://github.com/google/farmhash/blob/master/src/farmhash.h). **Parameters** From ceb83602b8fdf28fae45feac73e61cb0de53c383 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 2 Nov 2020 15:07:01 +0300 Subject: [PATCH 057/149] Refactor --- src/Interpreters/ActionsVisitor.cpp | 7 +++---- src/Interpreters/ActionsVisitor.h | 6 ++---- src/Interpreters/ExpressionAnalyzer.cpp | 10 +++++----- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 5 files changed, 13 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 202cda2d467..3054f4781d0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -385,7 +385,7 @@ ActionsMatcher::Data::Data( const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, bool has_having_) + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_) : context(context_) , set_size_limit(set_size_limit_) , subquery_depth(subquery_depth_) @@ -395,8 +395,7 @@ ActionsMatcher::Data::Data( , no_subqueries(no_subqueries_) , no_makeset(no_makeset_) , only_consts(only_consts_) - , no_storage_or_local(no_storage_or_local_) - , has_having(has_having_) + , create_source_for_in(create_source_for_in_) , visit_depth(0) , actions_stack(std::move(actions), context) , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) @@ -1054,7 +1053,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ - if ((!subquery_for_set.source && data.no_storage_or_local) || data.has_having) + if (!subquery_for_set.source && data.create_source_for_in) { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_unique(); diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 10583efa5b0..c2dd9c9b033 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -117,8 +117,7 @@ public: bool no_subqueries; bool no_makeset; bool only_consts; - bool no_storage_or_local; - bool has_having; + bool create_source_for_in; size_t visit_depth; ScopeStack actions_stack; @@ -132,8 +131,7 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_, - bool has_having); + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_); /// Does result of the calculation already exists in the block. bool hasColumn(const String & column_name) const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 6f062548653..516c12b2be7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -396,7 +396,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage(), false); + no_subqueries, false, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -407,17 +407,17 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, true, only_consts, !isRemoteStorage(), false); + no_subqueries, true, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } -void ExpressionAnalyzer::getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, - no_subqueries, false, only_consts, !isRemoteStorage(), true); + no_subqueries, false, only_consts, true); ActionsVisitor(visitor_data, log.stream()).visit(ast); actions = visitor_data.getActions(); } @@ -836,7 +836,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); step.required_output.push_back(select_query->having()->getColumnName()); - getRootActions(select_query->having(), only_types, step.actions()); + getRootActionsForHaving(select_query->having(), only_types, step.actions()); return true; } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 622c5204257..bd027e5a613 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -151,7 +151,7 @@ protected: */ void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); - void getRootActionsHasHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsForHaving(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 07c1942c08d..814350d2267 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -958,7 +958,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu preliminary_sort(); // If there is no global subqueries, we can run subqueries only when receive them on server. - if (expressions.hasHaving() || (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) + if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty()) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } @@ -1071,7 +1071,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } } - if (expressions.hasHaving() || (query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())) + if (!subqueries_for_sets.empty() && (expressions.hasHaving() || query_analyzer->hasGlobalSubqueries())) executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets); } From 08b63fde77df7f858a667caccdca99dda9dffa10 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 2 Nov 2020 21:39:54 +0300 Subject: [PATCH 058/149] done --- src/Interpreters/ExpressionActions.cpp | 5 ++++- .../0_stateless/01548_with_totals_having.reference | 12 ++++++++++++ .../queries/0_stateless/01548_with_totals_having.sql | 2 ++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01548_with_totals_having.reference create mode 100644 tests/queries/0_stateless/01548_with_totals_having.sql diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 762ad6ae575..f5cd0ff409d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -50,6 +50,7 @@ namespace ErrorCodes extern const int TOO_MANY_TEMPORARY_COLUMNS; extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS; extern const int TYPE_MISMATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } /// Read comment near usage @@ -375,7 +376,6 @@ void ExpressionAction::execute(Block & block, bool dry_run) const auto source = block.getByName(source_name); block.erase(source_name); source.column = source.column->convertToFullColumnIfConst(); - const ColumnArray * array = typeid_cast(source.column.get()); if (!array) throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); @@ -387,6 +387,9 @@ void ExpressionAction::execute(Block & block, bool dry_run) const source.type = assert_cast(*source.type).getNestedType(); source.name = result_name; + if (source.type->getTypeId() == TypeIndex::Nothing) + throw Exception("ARRAY JOIN of empty array is not allowed", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + block.insert(std::move(source)); break; diff --git a/tests/queries/0_stateless/01548_with_totals_having.reference b/tests/queries/0_stateless/01548_with_totals_having.reference new file mode 100644 index 00000000000..c728434f5dd --- /dev/null +++ b/tests/queries/0_stateless/01548_with_totals_having.reference @@ -0,0 +1,12 @@ +0 +0 +0 +0 +1 +1 +1 +2 +2 +3 + +0 diff --git a/tests/queries/0_stateless/01548_with_totals_having.sql b/tests/queries/0_stateless/01548_with_totals_having.sql new file mode 100644 index 00000000000..167d879bbeb --- /dev/null +++ b/tests/queries/0_stateless/01548_with_totals_having.sql @@ -0,0 +1,2 @@ +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 43 } +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([3, 2, 1, 0]) ORDER BY number; From a25996ed8d44743608932725f5a87421202ed9cf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 3 Nov 2020 02:10:19 +0300 Subject: [PATCH 059/149] Upload perf test results to the CI database --- docker/test/performance-comparison/compare.sh | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index d0cf12baa9e..20720e0b976 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1074,6 +1074,56 @@ wait unset IFS } +function upload_results +{ + if ! [ -v CHPC_DATABASE_URL ] + then + echo Database for test results is not specified, will not upload them. + return 0 + fi + + # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 + # so I have to do this instead. I tried to use Poco URI parser for this, + # but it's also broken and can't parse host:port. + IFS=':' read host port <<<"${CHPC_DATABASE_URL}" + + upload_client=(clickhouse-client + --host "${host}" + --port "${port}" + --secure + --user "${CHPC_DATABASE_USER}" + --password "${CHPC_DATABASE_PASSWORD}" + --config "ch/tests/config/client_config.xml" + --database perftest + -m + --date_time_input_format=best_effort) + + set +x # Don't show password in the log + cat "report/all-query-metrics.tsv" | "${upload_client[@]}" --query " + insert into query_metrics_tmp + select + toDate(event_time) event_date, + toDateTime('$(cd ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + $PR_TO_TEST pr_number, + '$REF_SHA' old_sha, + '$SHA_TO_TEST' new_sha, + test, + query_index, + query_display_name, + metric_name, + old_value, + new_value, + diff, + stat_threshold + from input('metric_name text, old_value float, new_value float, diff float, + ratio_display_text text, stat_threshold float, + test text, query_index int, query_display_name text') + settings date_time_input_format='best_effort' + format TSV + settings date_time_input_format='best_effort'" + set -x +} + # Check that local and client are in PATH clickhouse-local --version > /dev/null clickhouse-client --version > /dev/null @@ -1145,6 +1195,9 @@ case "$stage" in time "$script_dir/report.py" --report=all-queries > all-queries.html 2> >(tee -a report/errors.log 1>&2) ||: time "$script_dir/report.py" > report.html ;& +"upload_results") + time upload_results ||: + ;& esac # Print some final debug info to help debug Weirdness, of which there is plenty. From 2534ea342689f686d64ad64291eb85583c9cb31e Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 3 Nov 2020 10:22:51 +0000 Subject: [PATCH 060/149] fix bug in minMap and maxMap with zero values --- .../AggregateFunctionSumMap.h | 55 ++++++++++--------- .../01280_min_map_max_map.reference | 5 ++ .../0_stateless/01280_min_map_max_map.sql | 7 +++ 3 files changed, 42 insertions(+), 25 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 456334ee9c3..33fd87c0a87 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -56,7 +56,7 @@ struct AggregateFunctionMapData * minMap and maxMap share the same idea, but calculate min and max correspondingly. */ -template +template class AggregateFunctionMapBase : public IAggregateFunctionDataHelper< AggregateFunctionMapData>, Derived> { @@ -255,23 +255,27 @@ public: { // Final step does compaction of keys that have zero values, this mutates the state auto & merged_maps = this->data(place).merged_maps; - for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) - { - // Key is not compacted if it has at least one non-zero value - bool erase = true; - for (size_t col = 0; col < values_types.size(); ++col) - { - if (it->second[col] != values_types[col]->getDefault()) - { - erase = false; - break; - } - } - if (erase) - it = merged_maps.erase(it); - else - ++it; + if constexpr (compact) + { + for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) + { + // Key is not compacted if it has at least one non-zero value + bool erase = true; + for (size_t col = 0; col < values_types.size(); ++col) + { + if (it->second[col] != values_types[col]->getDefault()) + { + erase = false; + break; + } + } + + if (erase) + it = merged_maps.erase(it); + else + ++it; + } } size_t size = merged_maps.size(); @@ -314,11 +318,11 @@ public: template class AggregateFunctionSumMap final : - public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorSum, overflow, tuple_argument, true> { private: using Self = AggregateFunctionSumMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionSumMap(const DataTypePtr & keys_type_, @@ -342,11 +346,12 @@ class AggregateFunctionSumMapFiltered final : AggregateFunctionSumMapFiltered, FieldVisitorSum, overflow, - tuple_argument> + tuple_argument, + true> { private: using Self = AggregateFunctionSumMapFiltered; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; /// ARCADIA_BUILD disallow unordered_set for big ints for some reason static constexpr const bool allow_hash = !OverBigInt; @@ -474,11 +479,11 @@ public: template class AggregateFunctionMinMap final : - public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorMin, true, tuple_argument, false> { private: using Self = AggregateFunctionMinMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionMinMap(const DataTypePtr & keys_type_, @@ -498,11 +503,11 @@ public: template class AggregateFunctionMaxMap final : - public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument> + public AggregateFunctionMapBase, FieldVisitorMax, true, tuple_argument, false> { private: using Self = AggregateFunctionMaxMap; - using Base = AggregateFunctionMapBase; + using Base = AggregateFunctionMapBase; public: AggregateFunctionMaxMap(const DataTypePtr & keys_type_, diff --git a/tests/queries/0_stateless/01280_min_map_max_map.reference b/tests/queries/0_stateless/01280_min_map_max_map.reference index dd707d602c7..abb500553fd 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.reference +++ b/tests/queries/0_stateless/01280_min_map_max_map.reference @@ -22,3 +22,8 @@ (['1970-01-01 03:00:01'],[2]) ([1.01],[2]) (['a'],[2]) +([1,2],[0,0]) +([1,2],[0,0]) +(['A','B'],['','']) +(['A','B'],['','']) +([],[]) diff --git a/tests/queries/0_stateless/01280_min_map_max_map.sql b/tests/queries/0_stateless/01280_min_map_max_map.sql index 02731eee601..61d4a8ee4c9 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.sql +++ b/tests/queries/0_stateless/01280_min_map_max_map.sql @@ -31,3 +31,10 @@ select maxMap(val, cnt) from values ('val Array(Date), cnt Array(Int16)', ([1], select maxMap(val, cnt) from values ('val Array(DateTime(\'Europe/Moscow\')), cnt Array(Int32)', ([1], [1]), ([1], [2])); select maxMap(val, cnt) from values ('val Array(Decimal(10, 2)), cnt Array(Int16)', (['1.01'], [1]), (['1.01'], [2])); select maxMap(val, cnt) from values ('val Array(Enum16(\'a\'=1)), cnt Array(Int16)', (['a'], [1]), (['a'], [2])); + +-- bugfix, minMap and maxMap should not remove values with zero and empty strings but this behavior should not affect sumMap +select minMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); +select maxMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); +select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select sumMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); From cc8932e7a10fedabca306fe0b0a32abd0f00474d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 3 Nov 2020 15:51:26 +0300 Subject: [PATCH 061/149] Update DDLWorker.h --- src/Interpreters/DDLWorker.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 39cdcab709e..75d0ce3f5a4 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -105,7 +105,7 @@ private: private: std::atomic is_circular_replicated = false; - Context & context; + Context context; Poco::Logger * log; std::string host_fqdn; /// current host domain name From 2892252b3695080e73e9326ff505e170c21b2386 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 22 Oct 2020 23:23:44 +0300 Subject: [PATCH 062/149] Support collate in LowCardinality(String) and Nullable(String) and refactor ColumnString --- src/Columns/ColumnLowCardinality.cpp | 78 ++++-- src/Columns/ColumnLowCardinality.h | 9 + src/Columns/ColumnNullable.cpp | 51 +++- src/Columns/ColumnNullable.h | 9 + src/Columns/ColumnString.cpp | 230 +++++------------- src/Columns/ColumnString.h | 16 +- src/Interpreters/sortBlock.cpp | 57 ++++- ...01532_collate_in_low_cardinality.reference | 28 +++ .../01532_collate_in_low_cardinality.sql | 18 ++ .../01533_collate_in_nullable.reference | 36 +++ .../0_stateless/01533_collate_in_nullable.sql | 18 ++ 11 files changed, 342 insertions(+), 208 deletions(-) create mode 100644 tests/queries/0_stateless/01532_collate_in_low_cardinality.reference create mode 100644 tests/queries/0_stateless/01532_collate_in_low_cardinality.sql create mode 100644 tests/queries/0_stateless/01533_collate_in_nullable.reference create mode 100644 tests/queries/0_stateless/01533_collate_in_nullable.sql diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 64b503ed325..2e941a3ef8a 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -14,6 +15,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; + extern const int BAD_COLLATION; } namespace @@ -295,14 +297,24 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num compare_results, direction, nan_direction_hint); } -void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const { if (limit == 0) limit = size(); size_t unique_limit = getDictionary().size(); Permutation unique_perm; - getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->getPermutationWithCollation(*collator, reverse, unique_limit, unique_perm); + } + else + getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); /// TODO: optimize with sse. @@ -330,7 +342,8 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di } } -void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +template +void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const { if (equal_ranges.empty()) return; @@ -345,20 +358,17 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan EqualRanges new_ranges; SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + for (size_t i = 0; i < number_of_ranges; ++i) { const auto& [first, last] = equal_ranges[i]; - if (reverse) - std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); - else - std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; }); + std::sort(res.begin() + first, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (comparator(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -379,17 +389,12 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; }); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b) - {return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; }); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { - if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) != 0) + if (comparator(res[new_first],res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -401,7 +406,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan auto new_last = limit; for (auto j = limit; j < last; ++j) { - if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) == 0) + if (comparator(res[new_first], res[j]) == 0) { std::swap(res[new_last], res[j]); ++new_last; @@ -412,6 +417,43 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan } } +void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, nan_direction_hint, res); +} + +void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + auto comparator = [this, nan_direction_hint, reverse](size_t lhs, size_t rhs) + { + int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint); + return reverse ? -ret : ret; + }; + + updatePermutationImpl(limit, res, equal_ranges, comparator); +} + +void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, nan_direction_hint, res, &collator); +} + +void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +{ + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + auto comparator = [this, &column_string, &collator, reverse](size_t lhs, size_t rhs) + { + int ret = column_string->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *column_string, collator); + return reverse ? -ret : ret; + }; + + updatePermutationImpl(limit, res, equal_ranges, comparator); +} + std::vector ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const { auto columns = getIndexes().scatter(num_columns, selector); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 0aeda4567fd..e45449873fc 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -31,6 +31,11 @@ class ColumnLowCardinality final : public COWHelper + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -129,6 +134,10 @@ public: void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const; + + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + ColumnPtr replicate(const Offsets & offsets) const override { return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets)); diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 51248a598af..63b86f38342 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -17,6 +18,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; + extern const int BAD_COLLATION; } @@ -256,10 +258,21 @@ void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const { /// Cannot pass limit because of unknown amount of NULLs. - getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); + + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(&getNestedColumn()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->getPermutationWithCollation(*collator, reverse, 0, res); + } + else + getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); if ((null_direction_hint > 0) != reverse) { @@ -329,7 +342,7 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi } } -void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const { if (equal_ranges.empty()) return; @@ -432,12 +445,42 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire } } - getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); + if (collator) + { + /// Collations are supported only for ColumnString + const ColumnString * column_string = checkAndGetColumn(getNestedColumn()); + if (!column_string) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + + column_string->updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); + } + else + getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); equal_ranges = std::move(new_ranges); std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges)); } +void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, null_direction_hint, res); +} + +void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_ranges); +} + +void ColumnNullable::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + getPermutationImpl(reverse, limit, null_direction_hint, res, &collator); +} + +void ColumnNullable::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_range, &collator); +} + void ColumnNullable::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index e4033e22737..3d7a7970bd3 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -6,6 +6,7 @@ #include #include +class Collator; namespace DB { @@ -30,6 +31,11 @@ private: ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_); ColumnNullable(const ColumnNullable &) = default; + void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; + public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -94,6 +100,9 @@ public: int direction, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const; + void updatePermutationWithCollation( + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index cd06ea20f83..27dd9e54685 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,22 +284,23 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } -template -struct ColumnString::less +struct ColumnString::cmp { const ColumnString & parent; - explicit less(const ColumnString & parent_) : parent(parent_) {} - bool operator()(size_t lhs, size_t rhs) const + bool reverse; + cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1, parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1); - return positive ? (res < 0) : (res > 0); + return reverse ? -res : res; } }; -void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const +template +void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const { size_t s = offsets.size(); res.resize(s); @@ -309,23 +310,16 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio if (limit >= s) limit = 0; + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); - } + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); else - { - if (reverse) - std::sort(res.begin(), res.end(), less(*this)); - else - std::sort(res.begin(), res.end(), less(*this)); - } + std::sort(res.begin(), res.end(), less); } -void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const +template +void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const { if (equal_ranges.empty()) return; @@ -340,21 +334,17 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc if (limit) --number_of_ranges; + auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + for (size_t i = 0; i < number_of_ranges; ++i) { const auto & [first, last] = equal_ranges[i]; - - if (reverse) - std::sort(res.begin() + first, res.begin() + last, less(*this)); - else - std::sort(res.begin() + first, res.begin() + last, less(*this)); + std::sort(res.begin() + first, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < last; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0) + if (comparator(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -375,17 +365,12 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0) + if (comparator(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -395,9 +380,7 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc size_t new_last = limit; for (size_t j = limit; j < last; ++j) { - if (memcmpSmallAllowOverflow15( - chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1, - chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) == 0) + if (comparator(res[j], res[new_first]) == 0) { std::swap(res[j], res[new_last]); ++new_last; @@ -408,6 +391,45 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc } } +void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const +{ + getPermutationImpl(limit, res, cmp(*this, reverse)); +} + +void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(limit, res, equal_ranges, cmp(*this, reverse)); +} + +struct ColumnString::cmpWithCollation +{ + const ColumnString & parent; + const Collator & collator; + bool reverse; + + cmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + + int operator()(size_t lhs, size_t rhs) const + { + int res = collator.compare( + reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), + reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); + + return reverse ? -res : res; + } +}; + +void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const +{ + getPermutationImpl(limit, res, cmpWithCollation(*this, collator, reverse)); +} + +void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(limit, res, equal_ranges, cmpWithCollation(*this, collator, reverse)); +} + + ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const { size_t col_size = size(); @@ -476,13 +498,13 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - less less_op(*this); + cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { - if (less_op(i, min_idx)) + if (cmp_op(i, min_idx) < 0) min_idx = i; - else if (less_op(max_idx, i)) + else if (cmp_op(max_idx, i) < 0) max_idx = i; } @@ -500,134 +522,6 @@ int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs reinterpret_cast(&rhs.chars[rhs.offsetAt(m)]), rhs.sizeAt(m)); } - -template -struct ColumnString::lessWithCollation -{ - const ColumnString & parent; - const Collator & collator; - - lessWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {} - - bool operator()(size_t lhs, size_t rhs) const - { - int res = collator.compare( - reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), - reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); - - return positive ? (res < 0) : (res > 0); - } -}; - -void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const -{ - size_t s = offsets.size(); - res.resize(s); - for (size_t i = 0; i < s; ++i) - res[i] = i; - - if (limit >= s) - limit = 0; - - if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation(*this, collator)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation(*this, collator)); - } - else - { - if (reverse) - std::sort(res.begin(), res.end(), lessWithCollation(*this, collator)); - else - std::sort(res.begin(), res.end(), lessWithCollation(*this, collator)); - } -} - -void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const -{ - if (equal_ranges.empty()) - return; - - if (limit >= size() || limit >= equal_ranges.back().second) - limit = 0; - - size_t number_of_ranges = equal_ranges.size(); - if (limit) - --number_of_ranges; - - EqualRanges new_ranges; - SCOPE_EXIT({equal_ranges = std::move(new_ranges);}); - - for (size_t i = 0; i < number_of_ranges; ++i) - { - const auto& [first, last] = equal_ranges[i]; - - if (reverse) - std::sort(res.begin() + first, res.begin() + last, lessWithCollation(*this, collator)); - else - std::sort(res.begin() + first, res.begin() + last, lessWithCollation(*this, collator)); - auto new_first = first; - for (auto j = first + 1; j < last; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0) - { - if (j - new_first > 1) - new_ranges.emplace_back(new_first, j); - - new_first = j; - } - } - if (last - new_first > 1) - new_ranges.emplace_back(new_first, last); - } - - if (limit) - { - const auto & [first, last] = equal_ranges.back(); - - if (limit < first || limit > last) - return; - - /// Since then we are working inside the interval. - - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation(*this, collator)); - - auto new_first = first; - for (auto j = first + 1; j < limit; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0) - { - if (j - new_first > 1) - new_ranges.emplace_back(new_first, j); - - new_first = j; - } - } - auto new_last = limit; - for (auto j = limit; j < last; ++j) - { - if (collator.compare( - reinterpret_cast(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]), - reinterpret_cast(&chars[offsetAt(res[j])]), sizeAt(res[j])) == 0) - { - std::swap(res[new_last], res[j]); - ++new_last; - } - } - if (new_last - new_first > 1) - new_ranges.emplace_back(new_first, new_last); - } -} - void ColumnString::protect() { getChars().protect(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 19398e07b83..c91d982f126 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,15 +42,19 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } - template - struct less; + struct cmp; - template - struct lessWithCollation; + struct cmpWithCollation; ColumnString() = default; ColumnString(const ColumnString & src); + template + void getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + public: const char * getFamilyName() const override { return "String"; } TypeIndex getDataType() const override { return TypeIndex::String; } @@ -233,12 +237,12 @@ public: void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; + void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; /// Sorting with respect of collation. void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const; ColumnPtr replicate(const Offsets & replicate_offsets) const override; diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index c2436806fcd..5d114c746e5 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include @@ -20,6 +22,24 @@ static bool isCollationRequired(const SortColumnDescription & description) return description.collator != nullptr; } +static bool isCollationSupported(const IColumn * column) +{ + if (column->getDataType() == TypeIndex::String) + return true; + + if (column->getDataType() == TypeIndex::Nullable) + { + const ColumnNullable * column_nullable = assert_cast(column); + return isCollationSupported(&column_nullable->getNestedColumn()); + } + + if (column->getDataType() == TypeIndex::LowCardinality) + { + const ColumnLowCardinality * column_low_cardinality = assert_cast(column); + return isCollationSupported(column_low_cardinality->getDictionary().getNestedColumn().get()); + } + return false; +} ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) { @@ -101,7 +121,6 @@ struct PartialSortingLessWithCollation } }; - void sortBlock(Block & block, const SortDescription & description, UInt64 limit) { if (!block) @@ -120,14 +139,18 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) bool is_column_const = false; if (isCollationRequired(description[0])) { - /// it it's real string column, than we need sort + /// Check if column supports collations + if (!isCollationSupported(column)) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (const ColumnString * column_string = checkAndGetColumn(column)) column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm); - else if (checkAndGetColumnConstData(column)) + else if (const ColumnNullable * column_nullable = checkAndGetColumn(column)) + column_nullable->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); + else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column)) + column_low_cardinality->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); + else if (isColumnConst(*column)) is_column_const = true; - else - throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); - } else if (!isColumnConst(*column)) { @@ -163,8 +186,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) const IColumn * column = columns_with_sort_desc[i].column; if (isCollationRequired(description[i])) { - if (!checkAndGetColumn(column) && !checkAndGetColumnConstData(column)) - throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION); + if (!isCollationSupported(column)) + throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); need_collation = true; } @@ -187,10 +210,20 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) if (isCollationRequired(column.description)) { - const ColumnString & column_string = assert_cast(*column.column); - column_string.updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + if (const ColumnString * column_string = checkAndGetColumn(column.column)) + column_string->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + + else if (const ColumnNullable * column_nullable = checkAndGetColumn(column.column)) + column_nullable->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + + else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column.column)) + column_low_cardinality->updatePermutationWithCollation( + *column.description.collator, + column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); } else { diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference new file mode 100644 index 00000000000..b7a4830f9cf --- /dev/null +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference @@ -0,0 +1,28 @@ +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +Order by tuple without collate +1 Ё +1 а +1 ё +2 А +2 Я +2 я +Order by tuple with collate +1 а +1 ё +1 Ё +2 А +2 я +2 Я diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql new file mode 100644 index 00000000000..0f4194ee671 --- /dev/null +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test_collate; + +CREATE TABLE test_collate (x UInt32, s LowCardinality(String)) ENGINE=Memory(); + +INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'); + +SELECT 'Order by without collate'; +SELECT * FROM test_collate ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate ORDER BY s COLLATE 'ru'; + +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; + +DROP TABLE test_collate; + diff --git a/tests/queries/0_stateless/01533_collate_in_nullable.reference b/tests/queries/0_stateless/01533_collate_in_nullable.reference new file mode 100644 index 00000000000..6bb06cbc8b5 --- /dev/null +++ b/tests/queries/0_stateless/01533_collate_in_nullable.reference @@ -0,0 +1,36 @@ +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +1 \N +2 \N +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +1 \N +2 \N +Order by tuple without collate +1 Ё +1 а +1 ё +1 \N +2 А +2 Я +2 я +2 \N +Order by tuple with collate +1 а +1 ё +1 Ё +1 \N +2 А +2 я +2 Я +2 \N diff --git a/tests/queries/0_stateless/01533_collate_in_nullable.sql b/tests/queries/0_stateless/01533_collate_in_nullable.sql new file mode 100644 index 00000000000..40b48bee465 --- /dev/null +++ b/tests/queries/0_stateless/01533_collate_in_nullable.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test_collate; + +CREATE TABLE test_collate (x UInt32, s Nullable(String)) ENGINE=Memory(); + +INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (1, null), (2, 'А'), (2, 'я'), (2, 'Я'), (2, null); + +SELECT 'Order by without collate'; +SELECT * FROM test_collate ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate ORDER BY s COLLATE 'ru'; + +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; + +DROP TABLE test_collate; + From 18d8d5d81f5844f77ba3bac972dc7f893fc014d7 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 23 Oct 2020 11:02:40 +0300 Subject: [PATCH 063/149] Add tests in arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index f5b81c08520..cc03dee9eb0 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -155,5 +155,9 @@ 01509_dictionary_preallocate 01526_max_untracked_memory 01530_drop_database_atomic_sync +01532_collate_in_low_cardinality +01533_collate_in_nullable +01542_collate_in_array +01543_collate_in_tuple 01547_query_log_current_database 01548_query_log_query_execution_ms From 97a6e3dde2bb0b99a1323e05370ae07d6fc3012c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 24 Oct 2020 20:15:03 +0300 Subject: [PATCH 064/149] Skip collate test in fasttest --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 8300c31681e..f11e51dee98 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -240,6 +240,8 @@ TESTS_TO_SKIP=( 01354_order_by_tuple_collate_const 01355_ilike 01411_bayesian_ab_testing + 01532_collate_in_low_cardinality + 01533_collate_in_nullable _orc_ arrow avro From 5c296365e2c808e4ab601c389f38c7485f072e0b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 27 Oct 2020 14:12:48 +0300 Subject: [PATCH 065/149] Fix build error --- src/Columns/ColumnString.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 27dd9e54685..3093ae10646 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -288,7 +288,7 @@ struct ColumnString::cmp { const ColumnString & parent; bool reverse; - cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( From 4d399fff3ee8a0922ca3b33e80daf6fae5730d69 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 29 Oct 2020 14:24:01 +0300 Subject: [PATCH 066/149] Support collation for Array and Tuple --- docker/test/fasttest/run.sh | 2 + src/Columns/ColumnArray.cpp | 120 +++++++++++------- src/Columns/ColumnArray.h | 15 +++ src/Columns/ColumnLowCardinality.cpp | 33 ++--- src/Columns/ColumnLowCardinality.h | 19 ++- src/Columns/ColumnNullable.cpp | 30 +++-- src/Columns/ColumnNullable.h | 20 +-- src/Columns/ColumnString.cpp | 41 +++--- src/Columns/ColumnString.h | 20 +-- src/Columns/ColumnTuple.cpp | 89 ++++++++++--- src/Columns/ColumnTuple.h | 16 ++- src/Columns/IColumn.h | 23 +++- src/Core/SortCursor.h | 7 +- src/Interpreters/sortBlock.cpp | 56 ++------ ...01532_collate_in_low_cardinality.reference | 36 ++++++ .../01532_collate_in_low_cardinality.sql | 17 ++- .../01542_collate_in_array.reference | 50 ++++++++ .../0_stateless/01542_collate_in_array.sql | 34 +++++ .../01543_collate_in_tuple.reference | 60 +++++++++ .../0_stateless/01543_collate_in_tuple.sql | 34 +++++ 20 files changed, 524 insertions(+), 198 deletions(-) create mode 100644 tests/queries/0_stateless/01542_collate_in_array.reference create mode 100644 tests/queries/0_stateless/01542_collate_in_array.sql create mode 100644 tests/queries/0_stateless/01543_collate_in_tuple.reference create mode 100644 tests/queries/0_stateless/01543_collate_in_tuple.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index f11e51dee98..ad25be9e2de 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -242,6 +242,8 @@ TESTS_TO_SKIP=( 01411_bayesian_ab_testing 01532_collate_in_low_cardinality 01533_collate_in_nullable + 01542_collate_in_array + 01543_collate_in_tuple _orc_ arrow avro diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index e4d17c586ac..c061dd50642 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -324,8 +324,7 @@ void ColumnArray::popBack(size_t n) offsets_data.resize_assume_reserved(offsets_data.size() - n); } - -int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +int ColumnArray::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator) const { const ColumnArray & rhs = assert_cast(rhs_); @@ -334,8 +333,15 @@ int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_dir size_t rhs_size = rhs.sizeAt(m); size_t min_size = std::min(lhs_size, rhs_size); for (size_t i = 0; i < min_size; ++i) - if (int res = getData().compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint)) + { + int res; + if (collator) + res = getData().compareAtWithCollation(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint, *collator); + else + res = getData().compareAt(offsetAt(n) + i, rhs.offsetAt(m) + i, *rhs.data.get(), nan_direction_hint); + if (res) return res; + } return lhs_size < rhs_size ? -1 @@ -344,6 +350,16 @@ int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_dir : 1); } +int ColumnArray::compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs_, nan_direction_hint); +} + +int ColumnArray::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs_, nan_direction_hint, &collator); +} + void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -352,27 +368,25 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -namespace -{ - template - struct Less +struct ColumnArray::Cmp { + const ColumnArray & parent; + int nan_direction_hint; + bool reverse; + const Collator * collator; + + Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} + + int operator()(size_t lhs, size_t rhs) const { - const ColumnArray & parent; - int nan_direction_hint; - - Less(const ColumnArray & parent_, int nan_direction_hint_) - : parent(parent_), nan_direction_hint(nan_direction_hint_) {} - - bool operator()(size_t lhs, size_t rhs) const - { - if (positive) - return parent.compareAt(lhs, rhs, parent, nan_direction_hint) < 0; - else - return parent.compareAt(lhs, rhs, parent, nan_direction_hint) > 0; - } - }; -} - + int res; + if (collator) + res = parent.compareAtWithCollation(lhs, rhs, parent, nan_direction_hint, *collator); + else + res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); + return reverse ? -res : res; + } +}; void ColumnArray::reserve(size_t n) { @@ -753,7 +767,8 @@ ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit INSTANTIATE_INDEX_IMPL(ColumnArray) -void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +template +void ColumnArray::getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const { size_t s = size(); if (limit >= s) @@ -763,23 +778,16 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h for (size_t i = 0; i < s; ++i) res[i] = i; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; + if (limit) - { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(*this, nan_direction_hint)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(*this, nan_direction_hint)); - } + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); else - { - if (reverse) - std::sort(res.begin(), res.end(), Less(*this, nan_direction_hint)); - else - std::sort(res.begin(), res.end(), Less(*this, nan_direction_hint)); - } + std::sort(res.begin(), res.end(), less); } -void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +template +void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const { if (equal_range.empty()) return; @@ -792,20 +800,19 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio if (limit) --number_of_ranges; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; + EqualRanges new_ranges; for (size_t i = 0; i < number_of_ranges; ++i) { const auto & [first, last] = equal_range[i]; - if (reverse) - std::sort(res.begin() + first, res.begin() + last, Less(*this, nan_direction_hint)); - else - std::sort(res.begin() + first, res.begin() + last, Less(*this, nan_direction_hint)); + std::sort(res.begin() + first, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (cmp(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -827,14 +834,11 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio /// Since then we are working inside the interval. - if (reverse) - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, Less(*this, nan_direction_hint)); - else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, Less(*this, nan_direction_hint)); + std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0) + if (cmp(res[new_first], res[j]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -845,7 +849,7 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio auto new_last = limit; for (auto j = limit; j < last; ++j) { - if (compareAt(res[new_first], res[j], *this, nan_direction_hint) == 0) + if (cmp(res[new_first], res[j]) == 0) { std::swap(res[new_last], res[j]); ++new_last; @@ -859,6 +863,26 @@ void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_directio equal_range = std::move(new_ranges); } +void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse)); +} + +void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse)); +} + +void ColumnArray::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse, &collator)); +} + +void ColumnArray::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse, &collator)); +} + ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const { if (replicate_offsets.empty()) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index cec8387ab66..028eaba73c5 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -77,8 +77,11 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_range) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; @@ -132,6 +135,8 @@ public: return false; } + bool isCollationSupported() const override { return getData().isCollationSupported(); } + private: WrappedPtr data; WrappedPtr offsets; @@ -169,6 +174,16 @@ private: ColumnPtr filterTuple(const Filter & filt, ssize_t result_size_hint) const; ColumnPtr filterNullable(const Filter & filt, ssize_t result_size_hint) const; ColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const; + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator * collator=nullptr) const; + + template + void getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const; + + struct Cmp; }; diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 2e941a3ef8a..37e97da88b9 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; - extern const int BAD_COLLATION; } namespace @@ -280,14 +279,26 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const return ColumnLowCardinality::create(IColumn::mutate(std::move(unique_ptr)), getIndexes().cloneResized(size)); } -int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnLowCardinality::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator) const { const auto & low_cardinality_column = assert_cast(rhs); size_t n_index = getIndexes().getUInt(n); size_t m_index = low_cardinality_column.getIndexes().getUInt(m); + if (collator) + return getDictionary().getNestedColumn()->compareAtWithCollation(n_index, m_index, *low_cardinality_column.getDictionary().getNestedColumn(), nan_direction_hint, *collator); return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint); } +int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint); +} + +int ColumnLowCardinality::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); +} + void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -306,12 +317,7 @@ void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int na Permutation unique_perm; if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->getPermutationWithCollation(*collator, reverse, unique_limit, unique_perm); + getDictionary().getNestedColumn()->getPermutationWithCollation(*collator, reverse, unique_limit, nan_direction_hint, unique_perm); } else getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); @@ -438,16 +444,11 @@ void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator getPermutationImpl(reverse, limit, nan_direction_hint, res, &collator); } -void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const +void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getDictionary().getNestedColumn().get()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - auto comparator = [this, &column_string, &collator, reverse](size_t lhs, size_t rhs) + auto comparator = [this, &collator, reverse, nan_direction_hint](size_t lhs, size_t rhs) { - int ret = column_string->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *column_string, collator); + int ret = getDictionary().getNestedColumn()->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *getDictionary().getNestedColumn(), nan_direction_hint, collator); return reverse ? -ret : ret; }; diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e45449873fc..0874f619b8a 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -31,11 +31,6 @@ class ColumnLowCardinality final : public COWHelper - void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; - public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -130,13 +125,15 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator &) const override; + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override; - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_range) const override; ColumnPtr replicate(const Offsets & offsets) const override { @@ -179,6 +176,7 @@ public: size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } bool isNumeric() const override { return getDictionary().isNumeric(); } bool lowCardinality() const override { return true; } + bool isCollationSupported() const override { return getDictionary().getNestedColumn()->isCollationSupported(); } /** * Checks if the dictionary column is Nullable(T). @@ -318,6 +316,13 @@ private: void compactInplace(); void compactIfSharedDictionary(); + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; + + void getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; }; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 63b86f38342..cbb82264694 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int ILLEGAL_COLUMN; extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT; - extern const int BAD_COLLATION; } @@ -225,7 +224,7 @@ ColumnPtr ColumnNullable::index(const IColumn & indexes, size_t limit) const return ColumnNullable::create(indexed_data, indexed_null_map); } -int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +int ColumnNullable::compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator * collator) const { /// NULL values share the properties of NaN values. /// Here the last parameter of compareAt is called null_direction_hint @@ -247,9 +246,22 @@ int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null } const IColumn & nested_rhs = nullable_rhs.getNestedColumn(); + if (collator) + return getNestedColumn().compareAtWithCollation(n, m, nested_rhs, null_direction_hint, *collator); + return getNestedColumn().compareAt(n, m, nested_rhs, null_direction_hint); } +int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +{ + return compareAtImpl(n, m, rhs_, null_direction_hint); +} + +int ColumnNullable::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs_, null_direction_hint, &collator); +} + void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -264,12 +276,7 @@ void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_dir if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(&getNestedColumn()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->getPermutationWithCollation(*collator, reverse, 0, res); + getNestedColumn().getPermutationWithCollation(*collator, reverse, 0, null_direction_hint, res); } else getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); @@ -447,12 +454,7 @@ void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_ if (collator) { - /// Collations are supported only for ColumnString - const ColumnString * column_string = checkAndGetColumn(getNestedColumn()); - if (!column_string) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); - - column_string->updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); + getNestedColumn().updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); } else getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 3d7a7970bd3..47b0103eab4 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -31,11 +31,6 @@ private: ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_); ColumnNullable(const ColumnNullable &) = default; - void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; - - void updatePermutationImpl( - bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; - public: /** Create immutable column using immutable arguments. This arguments may be shared with other columns. * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. @@ -98,11 +93,12 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const; + void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutationWithCollation( - const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const; + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; @@ -138,6 +134,7 @@ public: bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); } size_t sizeOfValueIfFixed() const override { return null_map->sizeOfValueIfFixed() + nested_column->sizeOfValueIfFixed(); } bool onlyNull() const override { return nested_column->isDummy(); } + bool isCollationSupported() const override { return nested_column->isCollationSupported(); } /// Return the column that represents values. @@ -173,6 +170,13 @@ private: template void applyNullMapImpl(const ColumnUInt8 & map); + + int compareAtImpl(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint, const Collator * collator=nullptr) const; + + void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const; }; ColumnPtr makeNullable(const ColumnPtr & column); diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 3093ae10646..9ea12041d85 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,11 +284,11 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } -struct ColumnString::cmp +struct ColumnString::Cmp { const ColumnString & parent; bool reverse; - explicit cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit Cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( @@ -299,8 +299,8 @@ struct ColumnString::cmp } }; -template -void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const +template +void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const { size_t s = offsets.size(); res.resize(s); @@ -310,7 +310,7 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp compa if (limit >= s) limit = 0; - auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); @@ -318,8 +318,8 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp compa std::sort(res.begin(), res.end(), less); } -template -void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const +template +void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Comparator cmp) const { if (equal_ranges.empty()) return; @@ -334,7 +334,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR if (limit) --number_of_ranges; - auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; }; + auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; for (size_t i = 0; i < number_of_ranges; ++i) { @@ -344,7 +344,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_first = first; for (size_t j = first + 1; j < last; ++j) { - if (comparator(res[j], res[new_first]) != 0) + if (cmp(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -370,7 +370,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) { - if (comparator(res[j], res[new_first]) != 0) + if (cmp(res[j], res[new_first]) != 0) { if (j - new_first > 1) new_ranges.emplace_back(new_first, j); @@ -380,7 +380,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR size_t new_last = limit; for (size_t j = limit; j < last; ++j) { - if (comparator(res[j], res[new_first]) == 0) + if (cmp(res[j], res[new_first]) == 0) { std::swap(res[j], res[new_last]); ++new_last; @@ -393,21 +393,21 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const { - getPermutationImpl(limit, res, cmp(*this, reverse)); + getPermutationImpl(limit, res, Cmp(*this, reverse)); } void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, cmp(*this, reverse)); + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this, reverse)); } -struct ColumnString::cmpWithCollation +struct ColumnString::CmpWithCollation { const ColumnString & parent; const Collator & collator; bool reverse; - cmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { @@ -419,17 +419,16 @@ struct ColumnString::cmpWithCollation } }; -void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const +void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const { - getPermutationImpl(limit, res, cmpWithCollation(*this, collator, reverse)); + getPermutationImpl(limit, res, CmpWithCollation(*this, collator, reverse)); } void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, cmpWithCollation(*this, collator, reverse)); + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator, reverse)); } - ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const { size_t col_size = size(); @@ -498,7 +497,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - cmp cmp_op(*this); + Cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { @@ -513,7 +512,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const } -int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const +int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int, const Collator & collator) const { const ColumnString & rhs = assert_cast(rhs_); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index c91d982f126..1e6f60e63b3 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,18 +42,18 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } - struct cmp; + struct Cmp; - struct cmpWithCollation; + struct CmpWithCollation; ColumnString() = default; ColumnString(const ColumnString & src); - template - void getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const; + template + void getPermutationImpl(size_t limit, Permutation & res, Comparator cmp) const; - template - void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const; + template + void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Comparator cmp) const; public: const char * getFamilyName() const override { return "String"; } @@ -233,16 +233,16 @@ public: int direction, int nan_direction_hint) const override; /// Variant of compareAt for string comparison with respect of collation. - int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int, const Collator & collator) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; /// Sorting with respect of collation. - void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const override; - void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; @@ -274,6 +274,8 @@ public: // Throws an exception if offsets/chars are messed up void validate() const; + + bool isCollationSupported() const override { return true; } }; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 98a6611edb7..f588762fb67 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -275,16 +275,27 @@ MutableColumns ColumnTuple::scatter(ColumnIndex num_columns, const Selector & se return res; } -int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +int ColumnTuple::compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator) const { const size_t tuple_size = columns.size(); for (size_t i = 0; i < tuple_size; ++i) - if (int res = columns[i]->compareAt(n, m, *assert_cast(rhs).columns[i], nan_direction_hint)) + { + int res; + if (collator && columns[i]->isCollationSupported()) + res = columns[i]->compareAtWithCollation(n, m, *assert_cast(rhs).columns[i], nan_direction_hint, *collator); + else + res = columns[i]->compareAt(n, m, *assert_cast(rhs).columns[i], nan_direction_hint); + if (res) return res; - + } return 0; } +int ColumnTuple::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint); +} + void ColumnTuple::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const @@ -293,14 +304,20 @@ void ColumnTuple::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -template +int ColumnTuple::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const +{ + return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); +} + struct ColumnTuple::Less { TupleColumns columns; int nan_direction_hint; + bool reverse; + const Collator * collator; - Less(const TupleColumns & columns_, int nan_direction_hint_) - : columns(columns_), nan_direction_hint(nan_direction_hint_) + Less(const TupleColumns & columns_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + : columns(columns_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) { } @@ -308,17 +325,22 @@ struct ColumnTuple::Less { for (const auto & column : columns) { - int res = column->compareAt(a, b, *column, nan_direction_hint); + int res; + if (collator && column->isCollationSupported()) + res = column->compareAtWithCollation(a, b, *column, nan_direction_hint, *collator); + else + res = column->compareAt(a, b, *column, nan_direction_hint); if (res < 0) - return positive; + return !reverse; else if (res > 0) - return !positive; + return reverse; } return false; } }; -void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +template +void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperator less) const { size_t rows = size(); res.resize(rows); @@ -330,28 +352,25 @@ void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_h if (limit) { - if (reverse) - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(columns, nan_direction_hint)); - else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), Less(columns, nan_direction_hint)); + std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); } else { - if (reverse) - std::sort(res.begin(), res.end(), Less(columns, nan_direction_hint)); - else - std::sort(res.begin(), res.end(), Less(columns, nan_direction_hint)); + std::sort(res.begin(), res.end(), less); } } -void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const { if (equal_ranges.empty()) return; for (const auto & column : columns) { - column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); + if (collator && column->isCollationSupported()) + column->updatePermutationWithCollation(*collator, reverse, limit, nan_direction_hint, res, equal_ranges); + else + column->updatePermutation(reverse, limit, nan_direction_hint, res, equal_ranges); while (limit && !equal_ranges.empty() && limit <= equal_ranges.back().first) equal_ranges.pop_back(); @@ -361,6 +380,26 @@ void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_directio } } +void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse)); +} + +void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, nan_direction_hint, res, equal_ranges); +} + +void ColumnTuple::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const +{ + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse, &collator)); +} + +void ColumnTuple::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const +{ + updatePermutationImpl(reverse, limit, nan_direction_hint, res, equal_ranges, &collator); +} + void ColumnTuple::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); @@ -433,5 +472,15 @@ bool ColumnTuple::structureEquals(const IColumn & rhs) const return false; } +bool ColumnTuple::isCollationSupported() const +{ + for (const auto& column : columns) + { + if (column->isCollationSupported()) + return true; + } + return false; +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index e8dfd4c8e44..c34768d85a4 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -20,7 +20,6 @@ private: using TupleColumns = std::vector; TupleColumns columns; - template struct Less; explicit ColumnTuple(MutableColumns && columns); @@ -75,15 +74,19 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; - void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const override; + void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; + void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges& equal_ranges) const override; void reserve(size_t n) override; size_t byteSize() const override; size_t allocatedBytes() const override; void protect() override; void forEachSubcolumn(ColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; + bool isCollationSupported() const override; size_t tupleSize() const { return columns.size(); } @@ -94,6 +97,15 @@ public: Columns getColumnsCopy() const { return {columns.begin(), columns.end()}; } const ColumnPtr & getColumnPtr(size_t idx) const { return columns[idx]; } + +private: + int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; + + template + void getPermutationImpl(size_t limit, Permutation & res, LessOperator less) const; + + void updatePermutationImpl( + bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator=nullptr) const; }; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 14e6a9d7eed..6dbcfacefe9 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -9,7 +9,7 @@ class SipHash; - +class Collator; namespace DB { @@ -18,6 +18,7 @@ namespace ErrorCodes { extern const int CANNOT_GET_SIZE_OF_FIELD; extern const int NOT_IMPLEMENTED; + extern const int BAD_COLLATION; } class Arena; @@ -250,6 +251,12 @@ public: */ virtual int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const = 0; + /// Equivalent to compareAt, but collator is used to compare values. + virtual int compareAtWithCollation(size_t, size_t, const IColumn &, int, const Collator &) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing it.", ErrorCodes::BAD_COLLATION); + } + /// Compare the whole column with single value from rhs column. /// If row_indexes is nullptr, it's ignored. Otherwise, it is a set of rows to compare. /// compare_results[i] will be equal to compareAt(row_indexes[i], rhs_row_num, rhs, nan_direction_hint) * direction @@ -277,6 +284,18 @@ public: */ virtual void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const = 0; + /** Equivalent to getPermutation and updatePermutation but collator is used to compare values. + * Supported for String, LowCardinality(String), Nullable(String) and for Array and Tuple, containing them. + */ + virtual void getPermutationWithCollation(const Collator &, bool, size_t, int, Permutation &) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); + } + virtual void updatePermutationWithCollation(const Collator &, bool, size_t, int, Permutation &, EqualRanges&) const + { + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); + } + /** Copies each element according offsets parameter. * (i-th element should be copied offsets[i] - offsets[i - 1] times.) * It is necessary in ARRAY JOIN operation. @@ -402,6 +421,8 @@ public: virtual bool lowCardinality() const { return false; } + virtual bool isCollationSupported() const { return false; } + virtual ~IColumn() = default; IColumn() = default; IColumn(const IColumn &) = default; diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index 4c90cc723bf..7a222f70199 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -96,7 +96,7 @@ struct SortCursorImpl : column_desc.column_number; sort_columns.push_back(columns[column_number].get()); - need_collation[j] = desc[j].collator != nullptr && typeid_cast(sort_columns.back()); /// TODO Nullable(String) + need_collation[j] = desc[j].collator != nullptr && sort_columns.back()->isCollationSupported(); /// TODO Nullable(String) has_collation |= need_collation[j]; } @@ -201,10 +201,7 @@ struct SortCursorWithCollation : SortCursorHelper int nulls_direction = desc.nulls_direction; int res; if (impl->need_collation[i]) - { - const ColumnString & column_string = assert_cast(*impl->sort_columns[i]); - res = column_string.compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), *impl->desc[i].collator); - } + res = impl->sort_columns[i]->compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), nulls_direction, *impl->desc[i].collator); else res = impl->sort_columns[i]->compareAt(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), nulls_direction); diff --git a/src/Interpreters/sortBlock.cpp b/src/Interpreters/sortBlock.cpp index 5d114c746e5..edf911fa61c 100644 --- a/src/Interpreters/sortBlock.cpp +++ b/src/Interpreters/sortBlock.cpp @@ -22,24 +22,6 @@ static bool isCollationRequired(const SortColumnDescription & description) return description.collator != nullptr; } -static bool isCollationSupported(const IColumn * column) -{ - if (column->getDataType() == TypeIndex::String) - return true; - - if (column->getDataType() == TypeIndex::Nullable) - { - const ColumnNullable * column_nullable = assert_cast(column); - return isCollationSupported(&column_nullable->getNestedColumn()); - } - - if (column->getDataType() == TypeIndex::LowCardinality) - { - const ColumnLowCardinality * column_low_cardinality = assert_cast(column); - return isCollationSupported(column_low_cardinality->getDictionary().getNestedColumn().get()); - } - return false; -} ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description) { @@ -106,8 +88,7 @@ struct PartialSortingLessWithCollation } else if (isCollationRequired(elem.description)) { - const ColumnString & column_string = assert_cast(*elem.column); - res = column_string.compareAtWithCollation(a, b, *elem.column, *elem.description.collator); + res = elem.column->compareAtWithCollation(a, b, *elem.column, elem.description.nulls_direction, *elem.description.collator); } else res = elem.column->compareAt(a, b, *elem.column, elem.description.nulls_direction); @@ -139,18 +120,13 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) bool is_column_const = false; if (isCollationRequired(description[0])) { - /// Check if column supports collations - if (!isCollationSupported(column)) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (!column->isCollationSupported()) + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); - if (const ColumnString * column_string = checkAndGetColumn(column)) - column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm); - else if (const ColumnNullable * column_nullable = checkAndGetColumn(column)) - column_nullable->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); - else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column)) - column_low_cardinality->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); - else if (isColumnConst(*column)) + if (isColumnConst(*column)) is_column_const = true; + else + column->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm); } else if (!isColumnConst(*column)) { @@ -186,8 +162,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) const IColumn * column = columns_with_sort_desc[i].column; if (isCollationRequired(description[i])) { - if (!isCollationSupported(column)) - throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION); + if (!column->isCollationSupported()) + throw Exception("Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, containing them.", ErrorCodes::BAD_COLLATION); need_collation = true; } @@ -210,20 +186,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit) if (isCollationRequired(column.description)) { - if (const ColumnString * column_string = checkAndGetColumn(column.column)) - column_string->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); - - else if (const ColumnNullable * column_nullable = checkAndGetColumn(column.column)) - column_nullable->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); - - else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn(column.column)) - column_low_cardinality->updatePermutationWithCollation( - *column.description.collator, - column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); + column.column->updatePermutationWithCollation( + *column.description.collator, column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges); } else { diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference index b7a4830f9cf..fbffea8df5a 100644 --- a/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.reference @@ -26,3 +26,39 @@ Order by tuple with collate 2 А 2 я 2 Я +Order by without collate +1 Ё +2 А +2 Я +1 а +2 я +1 ё +1 \N +2 \N +Order by with collate +1 а +2 А +1 ё +1 Ё +2 я +2 Я +1 \N +2 \N +Order by tuple without collate +1 Ё +1 а +1 ё +1 \N +2 А +2 Я +2 я +2 \N +Order by tuple with collate +1 а +1 ё +1 Ё +1 \N +2 А +2 я +2 Я +2 \N diff --git a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql index 0f4194ee671..b6fba26eb2d 100644 --- a/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql +++ b/tests/queries/0_stateless/01532_collate_in_low_cardinality.sql @@ -1,8 +1,12 @@ DROP TABLE IF EXISTS test_collate; +DROP TABLE IF EXISTS test_collate_null; CREATE TABLE test_collate (x UInt32, s LowCardinality(String)) ENGINE=Memory(); +CREATE TABLE test_collate_null (x UInt32, s LowCardinality(Nullable(String))) ENGINE=Memory(); INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'); +INSERT INTO test_collate_null VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я'), (1, null), (2, null); + SELECT 'Order by without collate'; SELECT * FROM test_collate ORDER BY s; @@ -14,5 +18,16 @@ SELECT * FROM test_collate ORDER BY x, s; SELECT 'Order by tuple with collate'; SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru'; -DROP TABLE test_collate; +SELECT 'Order by without collate'; +SELECT * FROM test_collate_null ORDER BY s; +SELECT 'Order by with collate'; +SELECT * FROM test_collate_null ORDER BY s COLLATE 'ru'; +SELECT 'Order by tuple without collate'; +SELECT * FROM test_collate_null ORDER BY x, s; +SELECT 'Order by tuple with collate'; +SELECT * FROM test_collate_null ORDER BY x, s COLLATE 'ru'; + + +DROP TABLE test_collate; +DROP TABLE test_collate_null; diff --git a/tests/queries/0_stateless/01542_collate_in_array.reference b/tests/queries/0_stateless/01542_collate_in_array.reference new file mode 100644 index 00000000000..2c5a23066f3 --- /dev/null +++ b/tests/queries/0_stateless/01542_collate_in_array.reference @@ -0,0 +1,50 @@ +1 ['а'] +2 ['А'] +1 ['ё'] +1 ['ё','а'] +2 ['ё','а','а'] +1 ['ё','я'] +1 ['Ё'] +2 ['я','а'] +2 ['Я'] + +1 ['а'] +1 ['ё'] +1 ['ё','а'] +1 ['ё','я'] +1 ['Ё'] +2 ['А'] +2 ['ё','а','а'] +2 ['я','а'] +2 ['Я'] + +1 ['а'] +2 ['А'] +1 ['ё'] +1 ['ё','а'] +2 ['ё','а','а',NULL] +1 ['ё',NULL,'я'] +1 ['Ё'] +2 ['я'] +2 [NULL,'Я'] + +1 ['а'] +1 ['ё'] +1 ['ё','а'] +1 ['ё',NULL,'я'] +1 ['Ё'] +2 ['А'] +2 ['ё','а','а',NULL] +2 ['я'] +2 [NULL,'Я'] + +2 [['а','а'],['я','ё']] +1 [['а','Ё'],['ё','я']] +1 [['а','я'],['а','ё']] +2 [['ё']] + +1 [['а','Ё'],['ё','я']] +1 [['а','я'],['а','ё']] +2 [['а','а'],['я','ё']] +2 [['ё']] + diff --git a/tests/queries/0_stateless/01542_collate_in_array.sql b/tests/queries/0_stateless/01542_collate_in_array.sql new file mode 100644 index 00000000000..dd0ec769e7d --- /dev/null +++ b/tests/queries/0_stateless/01542_collate_in_array.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS collate_test1; +DROP TABLE IF EXISTS collate_test2; +DROP TABLE IF EXISTS collate_test3; + +CREATE TABLE collate_test1 (x UInt32, s Array(String)) ENGINE=Memory(); +CREATE TABLE collate_test2 (x UInt32, s Array(LowCardinality(Nullable(String)))) ENGINE=Memory(); +CREATE TABLE collate_test3 (x UInt32, s Array(Array(String))) ENGINE=Memory(); + +INSERT INTO collate_test1 VALUES (1, ['Ё']), (1, ['ё']), (1, ['а']), (2, ['А']), (2, ['я', 'а']), (2, ['Я']), (1, ['ё','а']), (1, ['ё', 'я']), (2, ['ё', 'а', 'а']); +INSERT INTO collate_test2 VALUES (1, ['Ё']), (1, ['ё']), (1, ['а']), (2, ['А']), (2, ['я']), (2, [null, 'Я']), (1, ['ё','а']), (1, ['ё', null, 'я']), (2, ['ё', 'а', 'а', null]); +INSERT INTO collate_test3 VALUES (1, [['а', 'я'], ['а', 'ё']]), (1, [['а', 'Ё'], ['ё', 'я']]), (2, [['ё']]), (2, [['а', 'а'], ['я', 'ё']]); + +SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test1 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +DROP TABLE collate_test1; +DROP TABLE collate_test2; +DROP TABLE collate_test3; + diff --git a/tests/queries/0_stateless/01543_collate_in_tuple.reference b/tests/queries/0_stateless/01543_collate_in_tuple.reference new file mode 100644 index 00000000000..fe8f935f0a6 --- /dev/null +++ b/tests/queries/0_stateless/01543_collate_in_tuple.reference @@ -0,0 +1,60 @@ +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +2 (1,'я') +1 (2,'а') +2 (2,'А') +2 (2,'Я') +1 (3,'я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +1 (2,'а') +1 (3,'я') +2 (1,'я') +2 (2,'А') +2 (2,'Я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +2 (1,'я') +1 (1,NULL) +2 (2,'А') +2 (2,'Я') +1 (2,NULL) +2 (2,NULL) +1 (3,'я') + +1 (1,'а') +1 (1,'ё') +1 (1,'Ё') +1 (1,NULL) +1 (2,NULL) +1 (3,'я') +2 (1,'я') +2 (2,'А') +2 (2,'Я') +2 (2,NULL) + +2 (1,(1,['А'])) +2 (1,(1,['ё','а','а'])) +1 (1,(1,['Ё'])) +2 (1,(1,['Я'])) +1 (1,(2,['а'])) +1 (1,(2,['ё','я'])) +1 (2,(1,['ё'])) +1 (2,(1,['ё','а'])) +2 (2,(1,['я'])) + +1 (1,(1,['Ё'])) +1 (1,(2,['а'])) +1 (1,(2,['ё','я'])) +1 (2,(1,['ё'])) +1 (2,(1,['ё','а'])) +2 (1,(1,['А'])) +2 (1,(1,['ё','а','а'])) +2 (1,(1,['Я'])) +2 (2,(1,['я'])) + diff --git a/tests/queries/0_stateless/01543_collate_in_tuple.sql b/tests/queries/0_stateless/01543_collate_in_tuple.sql new file mode 100644 index 00000000000..17d9426cf45 --- /dev/null +++ b/tests/queries/0_stateless/01543_collate_in_tuple.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS collate_test1; +DROP TABLE IF EXISTS collate_test2; +DROP TABLE IF EXISTS collate_test3; + +CREATE TABLE collate_test1 (x UInt32, s Tuple(UInt32, String)) ENGINE=Memory(); +CREATE TABLE collate_test2 (x UInt32, s Tuple(UInt32, LowCardinality(Nullable(String)))) ENGINE=Memory(); +CREATE TABLE collate_test3 (x UInt32, s Tuple(UInt32, Tuple(UInt32, Array(String)))) ENGINE=Memory(); + +INSERT INTO collate_test1 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), (2, (2, 'А')), (2, (1, 'я')), (2, (2, 'Я')), (1, (2,'а')), (1, (3, 'я')); +INSERT INTO collate_test2 VALUES (1, (1, 'Ё')), (1, (1, 'ё')), (1, (1, 'а')), (2, (2, 'А')), (2, (1, 'я')), (2, (2, 'Я')), (1, (2, null)), (1, (3, 'я')), (1, (1, null)), (2, (2, null)); +INSERT INTO collate_test3 VALUES (1, (1, (1, ['Ё']))), (1, (2, (1, ['ё']))), (1, (1, (2, ['а']))), (2, (1, (1, ['А']))), (2, (2, (1, ['я']))), (2, (1, (1, ['Я']))), (1, (2, (1, ['ё','а']))), (1, (1, (2, ['ё', 'я']))), (2, (1, (1, ['ё', 'а', 'а']))); + +SELECT * FROM collate_test1 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test1 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test2 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY s COLLATE 'ru'; +SELECT ''; + +SELECT * FROM collate_test3 ORDER BY x, s COLLATE 'ru'; +SELECT ''; + +DROP TABLE collate_test1; +DROP TABLE collate_test2; +DROP TABLE collate_test3; + From cf3f39ed29bf5a127c1dcc5c97ded68fb75befc8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 29 Oct 2020 14:37:00 +0300 Subject: [PATCH 067/149] Small changes --- src/Columns/ColumnLowCardinality.cpp | 2 -- src/Columns/ColumnNullable.cpp | 4 ---- 2 files changed, 6 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 37e97da88b9..3f03734b738 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -316,9 +316,7 @@ void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int na size_t unique_limit = getDictionary().size(); Permutation unique_perm; if (collator) - { getDictionary().getNestedColumn()->getPermutationWithCollation(*collator, reverse, unique_limit, nan_direction_hint, unique_perm); - } else getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index cbb82264694..4f2117b1405 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -275,9 +275,7 @@ void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_dir /// Cannot pass limit because of unknown amount of NULLs. if (collator) - { getNestedColumn().getPermutationWithCollation(*collator, reverse, 0, null_direction_hint, res); - } else getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res); @@ -453,9 +451,7 @@ void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_ } if (collator) - { getNestedColumn().updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges); - } else getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges); From bcd660bb57862b2aae0572518c1ecde2be59c21b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Oct 2020 08:35:18 +0300 Subject: [PATCH 068/149] Minor fixes --- src/Columns/ColumnArray.cpp | 5 +++-- src/Columns/ColumnConst.h | 2 ++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index c061dd50642..b420d337701 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -368,13 +368,14 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -struct ColumnArray::Cmp { +struct ColumnArray::Cmp +{ const ColumnArray & parent; int nan_direction_hint; bool reverse; const Collator * collator; - Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) + Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_ = false, const Collator * collator_ = nullptr) : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 4942d27b6c9..d7a8842bf01 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -248,6 +248,8 @@ public: /// The constant value. It is valid even if the size of the column is 0. template T getValue() const { return getField().safeGet>(); } + + bool isCollationSupported() const override { return true; } }; } From cd86f98aec6ff47185d6972d3509e508db4fdd3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 30 Oct 2020 08:36:27 +0300 Subject: [PATCH 069/149] Minor fixes --- src/Columns/ColumnConst.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index d7a8842bf01..3680926cd9b 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -249,7 +249,7 @@ public: template T getValue() const { return getField().safeGet>(); } - bool isCollationSupported() const override { return true; } + bool isCollationSupported() const override { return data->isCollationSupported(); } }; } From 9868b58531b7ce726e11e71a0b0b068cbb73cd06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 31 Oct 2020 03:59:58 +0300 Subject: [PATCH 070/149] Minor change --- src/Columns/ColumnString.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 9ea12041d85..477c098f067 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -407,7 +407,8 @@ struct ColumnString::CmpWithCollation const Collator & collator; bool reverse; - CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_ = false) + : parent(parent_), collator(collator_), reverse(reverse_) {} int operator()(size_t lhs, size_t rhs) const { From 726be794a4546101c078976427e77c7ce657876e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 17:14:34 +0300 Subject: [PATCH 071/149] Update ExpressionAnalyzer.cpp --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 516c12b2be7..45230c53e81 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -835,8 +835,8 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - step.required_output.push_back(select_query->having()->getColumnName()); getRootActionsForHaving(select_query->having(), only_types, step.actions()); + step.required_output.push_back(select_query->having()->getColumnName()); return true; } From 822bbcfdba6544dbcddd2f6985b92a3827504862 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 3 Nov 2020 17:25:52 +0300 Subject: [PATCH 072/149] Make Cmp with template --- src/Columns/ColumnArray.cpp | 32 ++++++++++++++++++++++---------- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnString.cpp | 35 +++++++++++++++++++++++------------ src/Columns/ColumnString.h | 2 ++ src/Columns/ColumnTuple.cpp | 20 +++++++++++++------- src/Columns/ColumnTuple.h | 1 + 6 files changed, 62 insertions(+), 29 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index b420d337701..f03a51e0681 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -368,15 +368,14 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } -struct ColumnArray::Cmp -{ +template +struct ColumnArray::Cmp { const ColumnArray & parent; int nan_direction_hint; - bool reverse; const Collator * collator; - Cmp(const ColumnArray & parent_, int nan_direction_hint_, bool reverse_ = false, const Collator * collator_ = nullptr) - : parent(parent_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) {} + Cmp(const ColumnArray & parent_, int nan_direction_hint_, const Collator * collator_=nullptr) + : parent(parent_), nan_direction_hint(nan_direction_hint_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const { @@ -385,7 +384,7 @@ struct ColumnArray::Cmp res = parent.compareAtWithCollation(lhs, rhs, parent, nan_direction_hint, *collator); else res = parent.compareAt(lhs, rhs, parent, nan_direction_hint); - return reverse ? -res : res; + return positive ? res : -res; } }; @@ -866,22 +865,35 @@ void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRa void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint)); + else + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint)); + } void ColumnArray::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const { - updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint)); + else + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint)); } void ColumnArray::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, reverse, &collator)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, &collator)); + else + getPermutationImpl(limit, res, Cmp(*this, nan_direction_hint, &collator)); } void ColumnArray::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const { - updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, reverse, &collator)); + if (reverse) + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, &collator)); + else + updatePermutationImpl(limit, res, equal_range, Cmp(*this, nan_direction_hint, &collator)); } ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 028eaba73c5..8a02af92dce 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -183,6 +183,7 @@ private: template void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_range, Comparator cmp) const; + template struct Cmp; }; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 477c098f067..23798f64a9c 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -284,18 +284,18 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } +template struct ColumnString::Cmp { const ColumnString & parent; - bool reverse; - explicit Cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {} + explicit Cmp(const ColumnString & parent_) : parent(parent_) {} int operator()(size_t lhs, size_t rhs) const { int res = memcmpSmallAllowOverflow15( parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1, parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1); - return reverse ? -res : res; + return positive ? res : -res; } }; @@ -393,22 +393,27 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const { - getPermutationImpl(limit, res, Cmp(*this, reverse)); + if (reverse) + getPermutationImpl(limit, res, Cmp(*this)); + else + getPermutationImpl(limit, res, Cmp(*this)); } void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, Cmp(*this, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this)); + else + updatePermutationImpl(limit, res, equal_ranges, Cmp(*this)); } +template struct ColumnString::CmpWithCollation { const ColumnString & parent; const Collator & collator; - bool reverse; - CmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_ = false) - : parent(parent_), collator(collator_), reverse(reverse_) {} + CmpWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {} int operator()(size_t lhs, size_t rhs) const { @@ -416,18 +421,24 @@ struct ColumnString::CmpWithCollation reinterpret_cast(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs), reinterpret_cast(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs)); - return reverse ? -res : res; + return positive ? res : -res; } }; void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res) const { - getPermutationImpl(limit, res, CmpWithCollation(*this, collator, reverse)); + if (reverse) + getPermutationImpl(limit, res, CmpWithCollation(*this, collator)); + else + getPermutationImpl(limit, res, CmpWithCollation(*this, collator)); } void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const { - updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator, reverse)); + if (reverse) + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator)); + else + updatePermutationImpl(limit, res, equal_ranges, CmpWithCollation(*this, collator)); } ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const @@ -498,7 +509,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const size_t min_idx = 0; size_t max_idx = 0; - Cmp cmp_op(*this); + Cmp cmp_op(*this); for (size_t i = 1; i < col_size; ++i) { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 1e6f60e63b3..b71751dbc4e 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -42,8 +42,10 @@ private: /// Size of i-th element, including terminating zero. size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; } + template struct Cmp; + template struct CmpWithCollation; ColumnString() = default; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index f588762fb67..d6e1ca982d6 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -309,15 +309,15 @@ int ColumnTuple::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); } +template struct ColumnTuple::Less { TupleColumns columns; int nan_direction_hint; - bool reverse; const Collator * collator; - Less(const TupleColumns & columns_, int nan_direction_hint_, bool reverse_=false, const Collator * collator_=nullptr) - : columns(columns_), nan_direction_hint(nan_direction_hint_), reverse(reverse_), collator(collator_) + Less(const TupleColumns & columns_, int nan_direction_hint_, const Collator * collator_=nullptr) + : columns(columns_), nan_direction_hint(nan_direction_hint_), collator(collator_) { } @@ -331,9 +331,9 @@ struct ColumnTuple::Less else res = column->compareAt(a, b, *column, nan_direction_hint); if (res < 0) - return !reverse; + return positive; else if (res > 0) - return reverse; + return !positive; } return false; } @@ -382,7 +382,10 @@ void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_dire void ColumnTuple::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse)); + if (reverse) + getPermutationImpl(limit, res, Less(columns, nan_direction_hint)); + else + getPermutationImpl(limit, res, Less(columns, nan_direction_hint)); } void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const @@ -392,7 +395,10 @@ void ColumnTuple::updatePermutation(bool reverse, size_t limit, int nan_directio void ColumnTuple::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { - getPermutationImpl(limit, res, Less(columns, nan_direction_hint, reverse, &collator)); + if (reverse) + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, &collator)); + else + getPermutationImpl(limit, res, Less(columns, nan_direction_hint, &collator)); } void ColumnTuple::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_ranges) const diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index c34768d85a4..0bee3463f2f 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -20,6 +20,7 @@ private: using TupleColumns = std::vector; TupleColumns columns; + template struct Less; explicit ColumnTuple(MutableColumns && columns); From 4245c470ac579d1d5f7b13083b32fd4a78c7a4fd Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 3 Nov 2020 17:37:54 +0300 Subject: [PATCH 073/149] fixes --- docker/test/performance-comparison/compare.sh | 11 ++++++----- docker/test/performance-comparison/entrypoint.sh | 3 +++ 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 20720e0b976..ca52fbd5891 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1085,7 +1085,7 @@ function upload_results # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 # so I have to do this instead. I tried to use Poco URI parser for this, # but it's also broken and can't parse host:port. - IFS=':' read host port <<<"${CHPC_DATABASE_URL}" + IFS=':' read -r host port <<<"${CHPC_DATABASE_URL}" upload_client=(clickhouse-client --host "${host}" @@ -1096,14 +1096,14 @@ function upload_results --config "ch/tests/config/client_config.xml" --database perftest -m - --date_time_input_format=best_effort) + "--date_time_input_format=best_effort") set +x # Don't show password in the log - cat "report/all-query-metrics.tsv" | "${upload_client[@]}" --query " + "${upload_client[@]}" --query " insert into query_metrics_tmp select toDate(event_time) event_date, - toDateTime('$(cd ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, $PR_TO_TEST pr_number, '$REF_SHA' old_sha, '$SHA_TO_TEST' new_sha, @@ -1120,7 +1120,8 @@ function upload_results test text, query_index int, query_display_name text') settings date_time_input_format='best_effort' format TSV - settings date_time_input_format='best_effort'" + settings date_time_input_format='best_effort' + " < "report/all-query-metrics.tsv" set -x } diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index ed2e542eadd..570a1c21514 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -121,6 +121,9 @@ set +e PATH="$(readlink -f right/)":"$PATH" export PATH +export REF_PR +export REF_SHA + # Start the main comparison script. { \ time ../download.sh "$REF_PR" "$REF_SHA" "$PR_TO_TEST" "$SHA_TO_TEST" && \ From b67465b010048d2c46cb0cf19e16aae9b13035fc Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 3 Nov 2020 18:00:51 +0300 Subject: [PATCH 074/149] Fix style --- src/Columns/ColumnArray.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index f03a51e0681..9b948236943 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -369,7 +369,8 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, } template -struct ColumnArray::Cmp { +struct ColumnArray::Cmp +{ const ColumnArray & parent; int nan_direction_hint; const Collator * collator; From d7de4509a3b2e26ca8f5f417ad4abb09818797fe Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 19:07:27 +0300 Subject: [PATCH 075/149] Update ActionsVisitor.cpp --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3054f4781d0..67ef37ba319 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1049,7 +1049,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. * - this function shows the expression IN_data1. - * + * * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. */ From 5cb708a50ab8025bf7110d4c64b38a8494c62e44 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 4 Nov 2020 04:19:53 +0300 Subject: [PATCH 076/149] fixup --- docker/test/performance-comparison/compare.sh | 74 +++++++++---------- 1 file changed, 35 insertions(+), 39 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index ca52fbd5891..08462298093 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1083,45 +1083,41 @@ function upload_results fi # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 - # so I have to do this instead. I tried to use Poco URI parser for this, - # but it's also broken and can't parse host:port. - IFS=':' read -r host port <<<"${CHPC_DATABASE_URL}" - - upload_client=(clickhouse-client - --host "${host}" - --port "${port}" - --secure - --user "${CHPC_DATABASE_USER}" - --password "${CHPC_DATABASE_PASSWORD}" - --config "ch/tests/config/client_config.xml" - --database perftest - -m - "--date_time_input_format=best_effort") - - set +x # Don't show password in the log - "${upload_client[@]}" --query " - insert into query_metrics_tmp - select - toDate(event_time) event_date, - toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, - $PR_TO_TEST pr_number, - '$REF_SHA' old_sha, - '$SHA_TO_TEST' new_sha, - test, - query_index, - query_display_name, - metric_name, - old_value, - new_value, - diff, - stat_threshold - from input('metric_name text, old_value float, new_value float, diff float, - ratio_display_text text, stat_threshold float, - test text, query_index int, query_display_name text') - settings date_time_input_format='best_effort' - format TSV - settings date_time_input_format='best_effort' - " < "report/all-query-metrics.tsv" + # so I have to extract host and port with clickhouse-local. I tried to use + # Poco URI parser to support this in the client, but it's broken and can't + # parse host:port. + set +x # Don't show password in the log + clickhouse-client \ + $(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") \ + --secure \ + --user "${CHPC_DATABASE_USER}" \ + --password "${CHPC_DATABASE_PASSWORD}" \ + --config "ch/tests/config/client_config.xml" \ + --database perftest \ + --date_time_input_format=best_effort \ + --query " + insert into query_metrics_tmp + select + toDate(event_time) event_date, + toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, + $PR_TO_TEST pr_number, + '$REF_SHA' old_sha, + '$SHA_TO_TEST' new_sha, + test, + query_index, + query_display_name, + metric_name, + old_value, + new_value, + diff, + stat_threshold + from input('metric_name text, old_value float, new_value float, diff float, + ratio_display_text text, stat_threshold float, + test text, query_index int, query_display_name text') + settings date_time_input_format='best_effort' + format TSV + settings date_time_input_format='best_effort' +" < report/all-query-metrics.tsv # Don't leave whitespace after INSERT: https://github.com/ClickHouse/ClickHouse/issues/16652 set -x } From b161127fc1223c40d5225f7deac594dc4bc95179 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 21:19:50 +0300 Subject: [PATCH 077/149] Add a test for spreading parts between threads for ReadInOrderOptimizer --- ...1551_mergetree_read_in_order_spread.reference | 11 +++++++++++ .../01551_mergetree_read_in_order_spread.sql | 16 ++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference create mode 100644 tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference new file mode 100644 index 00000000000..fc10b4707a9 --- /dev/null +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -0,0 +1,11 @@ +(Expression) +ExpressionTransform + (Expression) + ExpressionTransform + (Aggregating) + FinalizingSimpleTransform + AggregatingSortedTransform 3 → 1 + AggregatingInOrderTransform × 3 + (Expression) + ExpressionTransform × 3 + (ReadFromStorage) diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql new file mode 100644 index 00000000000..831a7282861 --- /dev/null +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS data_01551; + +CREATE TABLE data_01551 +( + key UInt32 +) engine=AggregatingMergeTree() +PARTITION BY key%2 +ORDER BY (key, key/2) +SETTINGS index_granularity=10; + +INSERT INTO data_01551 SELECT number FROM numbers(100000); +SET max_threads=3; +SET merge_tree_min_rows_for_concurrent_read=10000; +SET optimize_aggregation_in_order=1; +SET read_in_order_two_level_merge_threshold=1; +EXPLAIN PIPELINE SELECT key FROM data_01551 GROUP BY key, key/2; From 2389406c21848d07da7f2fc670a24612c018f6e4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 3 Nov 2020 21:19:50 +0300 Subject: [PATCH 078/149] Fix spreading for ReadInOrderOptimizer with expression in ORDER BY This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 56 ++++++++++++++----- 1 file changed, 43 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f06bfb97b2c..a38d50e56fb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -82,6 +82,17 @@ static Block getBlockWithPartColumn(const MergeTreeData::DataPartsVector & parts return Block{ColumnWithTypeAndName(std::move(column), std::make_shared(), "_part")}; } +/// Check if ORDER BY clause of the query has some expression. +static bool sortingDescriptionHasExpressions(const SortDescription & sort_description, const StorageMetadataPtr & metadata_snapshot) +{ + auto all_columns = metadata_snapshot->getColumns(); + for (const auto & sort_column : sort_description) + { + if (!all_columns.has(sort_column.column_name)) + return true; + } + return false; +} size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, @@ -1065,6 +1076,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const size_t min_marks_per_stream = (sum_marks - 1) / num_streams + 1; bool need_preliminary_merge = (parts.size() > settings.read_in_order_two_level_merge_threshold); + size_t max_output_ports = 0; for (size_t i = 0; i < num_streams && !parts.empty(); ++i) { @@ -1174,25 +1186,43 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( }); } - if (pipe.numOutputPorts() > 1 && need_preliminary_merge) + max_output_ports = std::max(pipe.numOutputPorts(), max_output_ports); + res.emplace_back(std::move(pipe)); + } + + if (need_preliminary_merge) + { + /// If ORDER BY clause of the query contains some expression, + /// then those new columns should be added for the merge step, + /// and this should be done always, if there is at least one pipe that + /// has multiple output ports. + bool sorting_key_has_expression = sortingDescriptionHasExpressions(input_order_info->order_key_prefix_descr, metadata_snapshot); + bool force_sorting_key_transform = res.size() > 1 && max_output_ports > 1 && sorting_key_has_expression; + + for (auto & pipe : res) { SortDescription sort_description; - for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], - input_order_info->direction, 1); - /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipe, data); - pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + if (pipe.numOutputPorts() > 1 || force_sorting_key_transform) { - return std::make_shared(header, sorting_key_prefix_expr); - }); + for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(metadata_snapshot->getSortingKey().column_names[j], + input_order_info->direction, 1); - pipe.addTransform(std::make_shared( - pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + /// Drop temporary columns, added by 'sorting_key_prefix_expr' + out_projection = createProjection(pipe, data); + pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) + { + return std::make_shared(header, sorting_key_prefix_expr); + }); + } + + if (pipe.numOutputPorts() > 1) + { + pipe.addTransform(std::make_shared( + pipe.getHeader(), pipe.numOutputPorts(), sort_description, max_block_size)); + } } - - res.emplace_back(std::move(pipe)); } return Pipe::unitePipes(std::move(res)); From cd711039fb6a9a4d382801e089281a0dba9115ae Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 07:56:27 +0000 Subject: [PATCH 079/149] Allow reinterpret between integers and floats of the same size --- .../registerFunctionsReinterpret.cpp | 4 +- ...nterpretStringAs.cpp => reinterpretAs.cpp} | 108 +++++++++++++----- src/Functions/ya.make | 2 +- 3 files changed, 84 insertions(+), 30 deletions(-) rename src/Functions/{reinterpretStringAs.cpp => reinterpretAs.cpp} (54%) diff --git a/src/Functions/registerFunctionsReinterpret.cpp b/src/Functions/registerFunctionsReinterpret.cpp index d2e43fbd52a..d82274ce9ed 100644 --- a/src/Functions/registerFunctionsReinterpret.cpp +++ b/src/Functions/registerFunctionsReinterpret.cpp @@ -3,13 +3,13 @@ namespace DB class FunctionFactory; -void registerFunctionsReinterpretStringAs(FunctionFactory & factory); +void registerFunctionsReinterpretAs(FunctionFactory & factory); void registerFunctionReinterpretAsString(FunctionFactory & factory); void registerFunctionReinterpretAsFixedString(FunctionFactory & factory); void registerFunctionsReinterpret(FunctionFactory & factory) { - registerFunctionsReinterpretStringAs(factory); + registerFunctionsReinterpretAs(factory); registerFunctionReinterpretAsString(factory); registerFunctionReinterpretAsFixedString(factory); } diff --git a/src/Functions/reinterpretStringAs.cpp b/src/Functions/reinterpretAs.cpp similarity index 54% rename from src/Functions/reinterpretStringAs.cpp rename to src/Functions/reinterpretAs.cpp index cd36d63dd46..1efadd4f491 100644 --- a/src/Functions/reinterpretStringAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -25,30 +26,60 @@ namespace ErrorCodes namespace { - -template -class FunctionReinterpretStringAs : public IFunction +template +class FunctionReinterpretAs : public IFunction { + template + static bool castType(const IDataType * type, F && f) + { + return castTypeToEither( + type, std::forward(f)); + } + + template + static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) + { + const auto * from_reinterpret = reinterpret_cast(const_cast(from.data())); + to.resize(from.size()); + for (size_t i = 0; i < from.size(); ++i) + { + to[i] = from_reinterpret[i]; + } + } + public: static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared(); } using ToFieldType = typename ToDataType::FieldType; using ColumnType = typename ToDataType::ColumnType; - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { const IDataType & type = *arguments[0]; - if (!isStringOrFixedString(type)) - throw Exception("Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if constexpr (support_between_float_integer) + { + if (!isStringOrFixedString(type) && !isNumber(type)) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (isNumber(type)) + { + if (type.getSizeOfValueInMemory() != ToDataType{}.getSizeOfValueInMemory()) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + else + { + if (!isStringOrFixedString(type)) + throw Exception( + "Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } return std::make_shared(); } @@ -99,10 +130,34 @@ public: return col_res; } + else if constexpr (support_between_float_integer) + { + ColumnPtr res; + if (castType(arguments[0].type.get(), [&](const auto & type) { + using DataType = std::decay_t; + using T = typename DataType::FieldType; + + const ColumnVector * col = checkAndGetColumn>(arguments[0].column.get()); + auto col_res = ColumnType::create(); + reinterpretImpl(col->getData(), col_res->getData()); + res = std::move(col_res); + + return true; + })) + { + return res; + } + else + { + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + } else { - throw Exception("Illegal column " + arguments[0].column->getName() - + " of argument of function " + getName(), + throw Exception( + "Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } @@ -123,23 +178,22 @@ struct NameReinterpretAsDate { static constexpr auto name = "reinterpretA struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; }; struct NameReinterpretAsUUID { static constexpr auto name = "reinterpretAsUUID"; }; -using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsUInt64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt8 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt16 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsInt64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs; -using FunctionReinterpretAsFloat64 = FunctionReinterpretStringAs; -using FunctionReinterpretAsDate = FunctionReinterpretStringAs; -using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs; -using FunctionReinterpretAsUUID = FunctionReinterpretStringAs; - +using FunctionReinterpretAsUInt8 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt16 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt32 = FunctionReinterpretAs; +using FunctionReinterpretAsUInt64 = FunctionReinterpretAs; +using FunctionReinterpretAsInt8 = FunctionReinterpretAs; +using FunctionReinterpretAsInt16 = FunctionReinterpretAs; +using FunctionReinterpretAsInt32 = FunctionReinterpretAs; +using FunctionReinterpretAsInt64 = FunctionReinterpretAs; +using FunctionReinterpretAsFloat32 = FunctionReinterpretAs; +using FunctionReinterpretAsFloat64 = FunctionReinterpretAs; +using FunctionReinterpretAsDate = FunctionReinterpretAs; +using FunctionReinterpretAsDateTime = FunctionReinterpretAs; +using FunctionReinterpretAsUUID = FunctionReinterpretAs; } -void registerFunctionsReinterpretStringAs(FunctionFactory & factory) +void registerFunctionsReinterpretAs(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 4c2cbaf5b1f..bc5af88b6af 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -359,7 +359,7 @@ SRCS( registerFunctionsVisitParam.cpp reinterpretAsFixedString.cpp reinterpretAsString.cpp - reinterpretStringAs.cpp + reinterpretAs.cpp repeat.cpp replaceAll.cpp replaceOne.cpp From b127752102594baab9ae9bc91a182b90adf76a59 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 07:58:26 +0000 Subject: [PATCH 080/149] add test --- .../0_stateless/01554_interpreter_integer_float.reference | 8 ++++++++ .../0_stateless/01554_interpreter_integer_float.sql | 8 ++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01554_interpreter_integer_float.reference create mode 100644 tests/queries/0_stateless/01554_interpreter_integer_float.sql diff --git a/tests/queries/0_stateless/01554_interpreter_integer_float.reference b/tests/queries/0_stateless/01554_interpreter_integer_float.reference new file mode 100644 index 00000000000..0aab8e60c7b --- /dev/null +++ b/tests/queries/0_stateless/01554_interpreter_integer_float.reference @@ -0,0 +1,8 @@ +1.72999e-40 +1067320848 +1.72999e-40 +1067320848 +6.09954e-319 +4608238783128613432 +6.09954e-319 +4608238783128613432 diff --git a/tests/queries/0_stateless/01554_interpreter_integer_float.sql b/tests/queries/0_stateless/01554_interpreter_integer_float.sql new file mode 100644 index 00000000000..69b00fc68d4 --- /dev/null +++ b/tests/queries/0_stateless/01554_interpreter_integer_float.sql @@ -0,0 +1,8 @@ +SELECT reinterpretAsFloat32(CAST(123456 AS UInt32)); +SELECT reinterpretAsUInt32(CAST(1.23456 AS Float32)); +SELECT reinterpretAsFloat32(CAST(123456 AS Int32)); +SELECT reinterpretAsInt32(CAST(1.23456 AS Float32)); +SELECT reinterpretAsFloat64(CAST(123456 AS UInt64)); +SELECT reinterpretAsUInt64(CAST(1.23456 AS Float64)); +SELECT reinterpretAsFloat64(CAST(123456 AS Int64)); +SELECT reinterpretAsInt64(CAST(1.23456 AS Float64)); From 9b8dbf37cc336d8d59c6b8fd5b5faf2252191686 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 08:30:09 +0000 Subject: [PATCH 081/149] fix fix --- src/Functions/reinterpretAs.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 1efadd4f491..76ee6155048 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -15,6 +15,7 @@ #include #include +#include namespace DB { @@ -39,11 +40,10 @@ class FunctionReinterpretAs : public IFunction template static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) { - const auto * from_reinterpret = reinterpret_cast(const_cast(from.data())); to.resize(from.size()); for (size_t i = 0; i < from.size(); ++i) { - to[i] = from_reinterpret[i]; + to[i] = unalignedLoad(&(from.data()[i])); } } @@ -133,7 +133,8 @@ public: else if constexpr (support_between_float_integer) { ColumnPtr res; - if (castType(arguments[0].type.get(), [&](const auto & type) { + if (castType(arguments[0].type.get(), [&](const auto & type) + { using DataType = std::decay_t; using T = typename DataType::FieldType; From d7b4e036d2ef94b4316cf3ff688f36ba25bbf7cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 12:27:34 +0300 Subject: [PATCH 082/149] Fix flaky tests after "max_memory_usage_for_user" --- .../01541_max_memory_usage_for_user.sh | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh index ecae442b134..3b1ced03473 100755 --- a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh +++ b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh @@ -9,26 +9,30 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # - max_memory_usage_for_user # - one users' query in background (to avoid reseting max_memory_usage_for_user) -query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number%10 FORMAT JSON" +# We have to create a separate user to run this tests isolated from other tests. +${CLICKHOUSE_CLIENT} -n --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; + + +query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number % 10 FORMAT JSON" function execute_http() { for _ in {1..100}; do - $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&max_memory_usage_for_user=100Mi&max_threads=1" -d@- <<<"$query" | grep -F DB::Exception: + $CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL&user=test_01541&max_memory_usage_for_user=100Mi&max_threads=1" -d@- <<<"$query" | grep -F DB::Exception: done } function execute_tcp() { # slow in debug, but should trigger the problem in ~10 iterations, so 20 is ok for _ in {1..20}; do - ${CLICKHOUSE_CLIENT} --max_memory_usage_for_user=100Mi --max_threads=1 -q "$query" | grep -F DB::Exception: + ${CLICKHOUSE_CLIENT} --user=test_01541 --max_memory_usage_for_user=100Mi --max_threads=1 -q "$query" | grep -F DB::Exception: done } function execute_tcp_one_session() { for _ in {1..30}; do echo "$query;" - done | ${CLICKHOUSE_CLIENT} -nm --max_memory_usage_for_user=100Mi --max_threads=1 | grep -F DB::Exception: + done | ${CLICKHOUSE_CLIENT} --user=test_01541 -nm --max_memory_usage_for_user=100Mi --max_threads=1 | grep -F DB::Exception: } @@ -36,7 +40,7 @@ function execute_tcp_one_session() # --max_block_size=1 to make it killable (check the state each 1 second, 1 row) # (the test takes ~40 seconds in debug build, so 60 seconds is ok) query_id=$$-$RANDOM-$SECONDS -${CLICKHOUSE_CLIENT} --max_block_size=1 --format Null --query_id $query_id -q 'SELECT sleepEachRow(1) FROM numbers(600)' & +${CLICKHOUSE_CLIENT} --user=test_01541 --max_block_size=1 --format Null --query_id $query_id -q 'SELECT sleepEachRow(1) FROM numbers(600)' & # trap sleep_query_pid=$! function cleanup() @@ -59,4 +63,6 @@ echo 'TCP' execute_tcp echo 'OK' +${CLICKHOUSE_CLIENT} --query "DROP USER test_01541"; + exit 0 From d99940b7f48ccdf0a1304b8493b4c50f8b22bf2e Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Wed, 4 Nov 2020 12:57:49 +0300 Subject: [PATCH 083/149] Development instructions updated to GCC 10 from GCC 9 --- docs/en/development/build.md | 14 +++++++------- docs/en/development/developer-instruction.md | 8 ++++---- docs/es/development/build.md | 12 ++++++------ docs/es/development/developer-instruction.md | 8 ++++---- docs/fa/development/build.md | 10 +++++----- docs/fa/development/developer-instruction.md | 8 ++++---- docs/fr/development/build.md | 12 ++++++------ docs/fr/development/developer-instruction.md | 8 ++++---- docs/ja/development/build.md | 10 +++++----- docs/ja/development/developer-instruction.md | 4 ++-- docs/ru/development/developer-instruction.md | 4 ++-- docs/tr/development/build.md | 12 ++++++------ docs/tr/development/developer-instruction.md | 4 ++-- docs/zh/development/build.md | 10 +++++----- docs/zh/development/developer-instruction.md | 4 ++-- 15 files changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index e9b6ecebae8..d6e06be3e38 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -23,7 +23,7 @@ $ sudo apt-get install git cmake python ninja-build Or cmake3 instead of cmake on older systems. -### Install GCC 9 {#install-gcc-9} +### Install GCC 10 {#install-gcc-10} There are several ways to do this. @@ -32,7 +32,7 @@ There are several ways to do this. On Ubuntu 19.10 or newer: $ sudo apt-get update - $ sudo apt-get install gcc-9 g++-9 + $ sudo apt-get install gcc-10 g++-10 #### Install from a PPA Package {#install-from-a-ppa-package} @@ -42,18 +42,18 @@ On older Ubuntu: $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` #### Install from Sources {#install-from-sources} See [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -### Use GCC 9 for Builds {#use-gcc-9-for-builds} +### Use GCC 10 for Builds {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ### Checkout ClickHouse Sources {#checkout-clickhouse-sources} @@ -88,7 +88,7 @@ The build requires the following components: - Git (is used only to checkout the sources, it’s not needed for the build) - CMake 3.10 or newer - Ninja (recommended) or Make -- C++ compiler: gcc 9 or clang 8 or newer +- C++ compiler: gcc 10 or clang 8 or newer - Linker: lld or gold (the classic GNU ld won’t work) - Python (is only used inside LLVM build and it is optional) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 3dc7c138520..dc95c3ec50b 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -131,13 +131,13 @@ ClickHouse uses several external libraries for building. All of them do not need ## C++ Compiler {#c-compiler} -Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse. +Compilers GCC starting from version 10 and Clang version 8 or above are supported for building ClickHouse. Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. To install GCC on Ubuntu run: `sudo apt install gcc g++` -Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/#install-gcc-9. +Check the version of gcc: `gcc --version`. If it is below 10, then follow the instruction here: https://clickhouse.tech/docs/en/development/build/#install-gcc-10. Mac OS X build is supported only for Clang. Just run `brew install llvm` @@ -152,11 +152,11 @@ Now that you are ready to build ClickHouse we recommend you to create a separate You can have several different directories (build_release, build_debug, etc.) for different types of build. -While inside the `build` directory, configure your build by running CMake. Before the first run, you need to define environment variables that specify compiler (version 9 gcc compiler in this example). +While inside the `build` directory, configure your build by running CMake. Before the first run, you need to define environment variables that specify compiler (version 10 gcc compiler in this example). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/es/development/build.md b/docs/es/development/build.md index 7a1479874eb..42cd9b5433f 100644 --- a/docs/es/development/build.md +++ b/docs/es/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build O cmake3 en lugar de cmake en sistemas más antiguos. -## Instalar GCC 9 {#install-gcc-9} +## Instalar GCC 10 {#install-gcc-10} Hay varias formas de hacer esto. @@ -29,18 +29,18 @@ Hay varias formas de hacer esto. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Instalar desde fuentes {#install-from-sources} Mira [Sistema abierto.](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Usar GCC 9 para compilaciones {#use-gcc-9-for-builds} +## Usar GCC 10 para compilaciones {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Fuentes de ClickHouse de pago {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ La compilación requiere los siguientes componentes: - Git (se usa solo para verificar las fuentes, no es necesario para la compilación) - CMake 3.10 o más reciente - Ninja (recomendado) o Hacer -- Compilador de C ++: gcc 9 o clang 8 o más reciente +- Compilador de C ++: gcc 10 o clang 8 o más reciente - Enlazador: lld u oro (el clásico GNU ld no funcionará) - Python (solo se usa dentro de la compilación LLVM y es opcional) diff --git a/docs/es/development/developer-instruction.md b/docs/es/development/developer-instruction.md index 8570fc1924c..390ac55602d 100644 --- a/docs/es/development/developer-instruction.md +++ b/docs/es/development/developer-instruction.md @@ -135,13 +135,13 @@ ClickHouse utiliza varias bibliotecas externas para la construcción. Todos ello # Compilador de C ++ {#c-compiler} -Los compiladores GCC a partir de la versión 9 y Clang versión 8 o superior son compatibles para construir ClickHouse. +Los compiladores GCC a partir de la versión 10 y Clang versión 8 o superior son compatibles para construir ClickHouse. Las compilaciones oficiales de Yandex actualmente usan GCC porque genera código de máquina de un rendimiento ligeramente mejor (con una diferencia de hasta varios por ciento según nuestros puntos de referencia). Y Clang es más conveniente para el desarrollo generalmente. Sin embargo, nuestra plataforma de integración continua (CI) ejecuta verificaciones de aproximadamente una docena de combinaciones de compilación. Para instalar GCC en Ubuntu, ejecute: `sudo apt install gcc g++` -Compruebe la versión de gcc: `gcc --version`. Si está por debajo de 9, siga las instrucciones aquí: https://clickhouse.tech/docs/es/development/build/#install-gcc-9. +Compruebe la versión de gcc: `gcc --version`. Si está por debajo de 9, siga las instrucciones aquí: https://clickhouse.tech/docs/es/development/build/#install-gcc-10. La compilación de Mac OS X solo es compatible con Clang. Sólo tiene que ejecutar `brew install llvm` @@ -156,11 +156,11 @@ Ahora que está listo para construir ClickHouse, le recomendamos que cree un dir Puede tener varios directorios diferentes (build_release, build_debug, etc.) para diferentes tipos de construcción. -Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 9 en este ejemplo). +Mientras que dentro de la `build` directorio, configure su compilación ejecutando CMake. Antes de la primera ejecución, debe definir variables de entorno que especifiquen el compilador (compilador gcc versión 10 en este ejemplo). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/fa/development/build.md b/docs/fa/development/build.md index db3aa96285f..b3d4fa1764e 100644 --- a/docs/fa/development/build.md +++ b/docs/fa/development/build.md @@ -20,7 +20,7 @@ $ sudo apt-get install git cmake python ninja-build یا سیمک 3 به جای کیک در سیستم های قدیمی تر. -## نصب شورای همکاری خلیج فارس 9 {#install-gcc-9} +## نصب شورای همکاری خلیج فارس 10 {install-gcc-10} راه های مختلفی برای انجام این کار وجود دارد. @@ -30,18 +30,18 @@ $ sudo apt-get install git cmake python ninja-build $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### نصب از منابع {#install-from-sources} نگاه کن [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## استفاده از شورای همکاری خلیج فارس 9 برای ساخت {#use-gcc-9-for-builds} +## استفاده از شورای همکاری خلیج فارس 10 برای ساخت {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## پرداخت منابع کلیک {#checkout-clickhouse-sources} diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index c9461748098..01fb24ac006 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -137,13 +137,13 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ # ج ++ کامپایلر {#c-compiler} -کامپایلر شورای همکاری خلیج فارس با شروع از نسخه 9 و صدای شیپور نسخه 8 یا بالاتر برای ساخت و ساز خانه عروسکی پشتیبانی می کند. +کامپایلر شورای همکاری خلیج فارس با شروع از نسخه 10 و صدای شیپور نسخه 8 یا بالاتر برای ساخت و ساز خانه عروسکی پشتیبانی می کند. یاندکس رسمی ایجاد شده در حال حاضر با استفاده از شورای همکاری خلیج فارس به دلیل تولید کد ماشین از عملکرد کمی بهتر (بازده تفاوت تا چند درصد با توجه به معیار ما). و کلانگ معمولا برای توسعه راحت تر است. هر چند, ادغام مداوم ما (سی) پلت فرم اجرا می شود چک برای حدود یک دوجین از ترکیب ساخت. برای نصب شورای همکاری خلیج فارس در اوبونتو اجرای: `sudo apt install gcc g++` -بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 9, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-9. +بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 9, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-10. سیستم عامل مک ایکس ساخت فقط برای صدای جرنگ جرنگ پشتیبانی می شود. فقط فرار کن `brew install llvm` @@ -158,11 +158,11 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ شما می توانید چندین دایرکتوری های مختلف (build_release, build_debug ، ) برای انواع مختلف ساخت. -در حالی که در داخل `build` فهرست, پیکربندی ساخت خود را با در حال اجرا کیک. قبل از اولین اجرا, شما نیاز به تعریف متغیرهای محیطی که کامپایلر را مشخص (نسخه 9 کامپایلر شورای همکاری خلیج فارس در این مثال). +در حالی که در داخل `build` فهرست, پیکربندی ساخت خود را با در حال اجرا کیک. قبل از اولین اجرا, شما نیاز به تعریف متغیرهای محیطی که کامپایلر را مشخص (نسخه 10 کامپایلر شورای همکاری خلیج فارس در این مثال). لینوکس: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. سیستم عامل مک ایکس: diff --git a/docs/fr/development/build.md b/docs/fr/development/build.md index 0d2713aaaa5..4889373c52a 100644 --- a/docs/fr/development/build.md +++ b/docs/fr/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build Ou cmake3 au lieu de cmake sur les systèmes plus anciens. -## Installer GCC 9 {#install-gcc-9} +## Installer GCC 10 {#install-gcc-10} Il y a plusieurs façons de le faire. @@ -29,18 +29,18 @@ Il y a plusieurs façons de le faire. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Installer à partir de Sources {#install-from-sources} Regarder [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Utilisez GCC 9 pour les Builds {#use-gcc-9-for-builds} +## Utilisez GCC 10 pour les Builds {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Commander Clickhouse Sources {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ La construction nécessite les composants suivants: - Git (est utilisé uniquement pour extraire les sources, ce n'est pas nécessaire pour la construction) - CMake 3.10 ou plus récent - Ninja (recommandé) ou faire -- Compilateur C++: gcc 9 ou clang 8 ou plus récent +- Compilateur C++: gcc 10 ou clang 8 ou plus récent - Linker: lld ou gold (le classique GNU LD ne fonctionnera pas) - Python (est seulement utilisé dans la construction LLVM et il est facultatif) diff --git a/docs/fr/development/developer-instruction.md b/docs/fr/development/developer-instruction.md index 576188941bd..e78ed7ba6d9 100644 --- a/docs/fr/development/developer-instruction.md +++ b/docs/fr/development/developer-instruction.md @@ -135,13 +135,13 @@ ClickHouse utilise plusieurs bibliothèques externes pour la construction. Tous # Compilateur C++ {#c-compiler} -Les compilateurs GCC à partir de la version 9 et Clang version 8 ou supérieure sont pris en charge pour construire ClickHouse. +Les compilateurs GCC à partir de la version 10 et Clang version 8 ou supérieure sont pris en charge pour construire ClickHouse. Les builds officiels de Yandex utilisent actuellement GCC car ils génèrent du code machine de performances légèrement meilleures (ce qui donne une différence allant jusqu'à plusieurs pour cent selon nos benchmarks). Et Clang est plus pratique pour le développement habituellement. Cependant, notre plate-forme d'intégration continue (CI) vérifie environ une douzaine de combinaisons de construction. Pour installer GCC sur Ubuntu Exécutez: `sudo apt install gcc g++` -Vérifiez la version de gcc: `gcc --version`. Si elle est inférieure à 9, suivez les instructions ici: https://clickhouse.tech/docs/fr/development/build/#install-gcc-9. +Vérifiez la version de gcc: `gcc --version`. Si elle est inférieure à 10, suivez les instructions ici: https://clickhouse.tech/docs/fr/development/build/#install-gcc-10. Mac OS X build est pris en charge uniquement pour Clang. Il suffit d'exécuter `brew install llvm` @@ -156,11 +156,11 @@ Maintenant que vous êtes prêt à construire ClickHouse nous vous conseillons d Vous pouvez avoir plusieurs répertoires différents (build_release, build_debug, etc.) pour les différents types de construction. -Tandis qu'à l'intérieur de la `build` répertoire, configurez votre build en exécutant CMake. Avant la première exécution, vous devez définir des variables d'environnement qui spécifient le compilateur (compilateur gcc version 9 dans cet exemple). +Tandis qu'à l'intérieur de la `build` répertoire, configurez votre build en exécutant CMake. Avant la première exécution, vous devez définir des variables d'environnement qui spécifient le compilateur (compilateur gcc version 10 dans cet exemple). Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/ja/development/build.md b/docs/ja/development/build.md index 33709cb65f3..e44ba45485e 100644 --- a/docs/ja/development/build.md +++ b/docs/ja/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build 古いシステムではcmakeの代わりにcmake3。 -## GCC9のインストール {#install-gcc-9} +## GCC9のインストール {#install-gcc-10} これを行うにはいくつかの方法があります。 @@ -29,18 +29,18 @@ $ sudo apt-get install git cmake python ninja-build $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### ソースからインスト {#install-from-sources} 見て [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## ビルドにGCC9を使用する {#use-gcc-9-for-builds} +## ビルドにGCC9を使用する {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## ツつィツ姪"ツ債ツつケ {#checkout-clickhouse-sources} diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index f3a703f9879..988becf98c3 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -141,7 +141,7 @@ ClickHouseのビルドには、バージョン9以降のGCCとClangバージョ UBUNTUにGCCをインストールするには: `sudo apt install gcc g++` -Gccのバージョンを確認する: `gcc --version`. の場合は下記9その指示に従う。https://clickhouse.tech/docs/ja/development/build/#install-gcc-9. +Gccのバージョンを確認する: `gcc --version`. の場合は下記9その指示に従う。https://clickhouse.tech/docs/ja/development/build/#install-gcc-10. Mac OS XのビルドはClangでのみサポートされています。 ちょうど実行 `brew install llvm` @@ -160,7 +160,7 @@ ClickHouseを構築する準備ができたので、別のディレクトリを Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index b0c84d9de7c..4bdcf89004d 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -142,7 +142,7 @@ ClickHouse использует для сборки некоторое коли Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. -Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.tech/docs/ru/development/build/#install-gcc-9. +Проверьте версию gcc: `gcc --version`. Если версия меньше 10, то следуйте инструкции: https://clickhouse.tech/docs/ru/development/build/#install-gcc-10. Сборка под Mac OS X поддерживается только для компилятора Clang. Чтобы установить его выполните `brew install llvm` @@ -162,7 +162,7 @@ ClickHouse использует для сборки некоторое коли Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/tr/development/build.md b/docs/tr/development/build.md index 92eaaf30ced..93ddebffa75 100644 --- a/docs/tr/development/build.md +++ b/docs/tr/development/build.md @@ -19,7 +19,7 @@ $ sudo apt-get install git cmake python ninja-build Veya eski sistemlerde cmake yerine cmake3. -## Gcc 9'u yükle {#install-gcc-9} +## Gcc 10'u yükle {#install-gcc-10} Bunu yapmak için çeşitli yollar vardır. @@ -29,18 +29,18 @@ Bunu yapmak için çeşitli yollar vardır. $ sudo apt-get install software-properties-common $ sudo apt-add-repository ppa:ubuntu-toolchain-r/test $ sudo apt-get update -$ sudo apt-get install gcc-9 g++-9 +$ sudo apt-get install gcc-10 g++-10 ``` ### Kaynaklardan yükleyin {#install-from-sources} Bakmak [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Yapılar için GCC 9 kullanın {#use-gcc-9-for-builds} +## Yapılar için GCC 10 kullanın {#use-gcc-10-for-builds} ``` bash -$ export CC=gcc-9 -$ export CXX=g++-9 +$ export CC=gcc-10 +$ export CXX=g++-10 ``` ## Checkout ClickHouse Kaynakları {#checkout-clickhouse-sources} @@ -76,7 +76,7 @@ Yapı aşağıdaki bileşenleri gerektirir: - Git (yalnızca kaynakları kontrol etmek için kullanılır, yapı için gerekli değildir) - Cmake 3.10 veya daha yeni - Ninja (önerilir) veya yapmak -- C ++ derleyici: gcc 9 veya clang 8 veya daha yeni +- C ++ derleyici: gcc 10 veya clang 8 veya daha yeni - Linker :lld veya altın (klasik GNU ld çalışmaz) - Python (sadece LLVM yapısında kullanılır ve isteğe bağlıdır) diff --git a/docs/tr/development/developer-instruction.md b/docs/tr/development/developer-instruction.md index 10b1ea35758..51a6c4345c6 100644 --- a/docs/tr/development/developer-instruction.md +++ b/docs/tr/development/developer-instruction.md @@ -141,7 +141,7 @@ Resmi Yandex şu anda GCC'Yİ kullanıyor çünkü biraz daha iyi performansa sa Ubuntu run GCC yüklemek için: `sudo apt install gcc g++` -Gcc sürümünü kontrol edin: `gcc --version`. 9'un altındaysa, buradaki talimatları izleyin: https://clickhouse.tech/docs/tr/development/build/#install-gcc-9. +Gcc sürümünü kontrol edin: `gcc --version`. 10'un altındaysa, buradaki talimatları izleyin: https://clickhouse.tech/docs/tr/development/build/#install-gcc-10. Mac OS X build sadece Clang için desteklenir. Sadece koş `brew install llvm` @@ -160,7 +160,7 @@ Birkaç farklı dizine (build_release, build_debug, vb.) sahip olabilirsiniz.) f Linux: - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. Mac OS X: diff --git a/docs/zh/development/build.md b/docs/zh/development/build.md index 8cf371b7be5..1aa5c1c97b7 100644 --- a/docs/zh/development/build.md +++ b/docs/zh/development/build.md @@ -35,7 +35,7 @@ sudo apt-get install git cmake ninja-build 或cmake3而不是旧系统上的cmake。 或者在早期版本的系统中用 cmake3 替代 cmake -## 安装 GCC 9 {#an-zhuang-gcc-9} +## 安装 GCC 10 {#an-zhuang-gcc-10} 有几种方法可以做到这一点。 @@ -45,18 +45,18 @@ sudo apt-get install git cmake ninja-build sudo apt-get install software-properties-common sudo apt-add-repository ppa:ubuntu-toolchain-r/test sudo apt-get update -sudo apt-get install gcc-9 g++-9 +sudo apt-get install gcc-10 g++-10 ``` ### 源码安装 gcc {#yuan-ma-an-zhuang-gcc} 请查看 [utils/ci/build-gcc-from-sources.sh](https://github.com/ClickHouse/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## 使用 GCC 9 来编译 {#shi-yong-gcc-9-lai-bian-yi} +## 使用 GCC 10 来编译 {#shi-yong-gcc-10-lai-bian-yi} ``` bash -export CC=gcc-9 -export CXX=g++-9 +export CC=gcc-10 +export CXX=g++-10 ``` ## 拉取 ClickHouse 源码 {#la-qu-clickhouse-yuan-ma-1} diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index faa333f8e31..3e2ccf5da35 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -129,7 +129,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 在Ubuntu上安装GCC,请执行:`sudo apt install gcc g++` -请使用`gcc --version`查看gcc的版本。如果gcc版本低于9,请参考此处的指示:https://clickhouse.tech/docs/zh/development/build/#an-zhuang-gcc-9 。 +请使用`gcc --version`查看gcc的版本。如果gcc版本低于9,请参考此处的指示:https://clickhouse.tech/docs/zh/development/build/#an-zhuang-gcc-10 。 在Mac OS X上安装GCC,请执行:`brew install gcc` @@ -146,7 +146,7 @@ Yandex官方当前使用GCC构建ClickHouse,因为它生成的机器代码性 在`build`目录下,通过运行CMake配置构建。 在第一次运行之前,请定义用于指定编译器的环境变量(本示例中为gcc 9 编译器)。 - export CC=gcc-9 CXX=g++-9 + export CC=gcc-10 CXX=g++-10 cmake .. `CC`变量指代C的编译器(C Compiler的缩写),而`CXX`变量指代要使用哪个C++编译器进行编译。 From 7ec73e1f206f34627e48279a7d1eb9da3646e0ef Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 11:21:59 +0000 Subject: [PATCH 084/149] fix build --- src/Interpreters/TableJoin.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index cd837cc15d6..5db914bc457 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -28,8 +28,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , temporary_files_codec(settings.temporary_files_codec) , tmp_volume(tmp_volume_) { - if (settings.partial_merge_join) - join_algorithm = JoinAlgorithm::PREFER_PARTIAL_MERGE; } void TableJoin::resetCollected() From 269e96f17ee8ffd0f02081d20b004fb6bfbe7cf0 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 4 Nov 2020 12:15:31 +0000 Subject: [PATCH 085/149] fix --- src/Interpreters/TreeRewriter.cpp | 8 +++++--- ...8_setting_aggregate_functions_null_for_empty.reference | 8 ++++++++ .../01528_setting_aggregate_functions_null_for_empty.sql | 8 ++++++++ 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 7c37365a384..c8691c25f1b 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -120,10 +120,12 @@ struct CustomizeAggregateFunctionsSuffixData void visit(ASTFunction & func, ASTPtr &) { - if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name) - && !endsWith(func.name, customized_func_suffix)) + const auto & instance = AggregateFunctionFactory::instance(); + if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix)) { - func.name = func.name + customized_func_suffix; + auto properties = instance.tryGetProperties(func.name); + if (properties && !properties->returns_default_when_only_null) + func.name = func.name + customized_func_suffix; } } }; diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference index 570ea20ffad..9c6ae9c65ab 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference @@ -1,8 +1,16 @@ 0 \N +0 +\N \N \N +0 +\N 45 45 +10 +10 45 45 +10 +10 diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql index c59b592e701..e76ce667bbc 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql @@ -7,11 +7,15 @@ CREATE TABLE defaults SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; SET aggregate_functions_null_for_empty=1; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; INSERT INTO defaults SELECT * FROM numbers(10); @@ -19,10 +23,14 @@ SET aggregate_functions_null_for_empty=0; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; SET aggregate_functions_null_for_empty=1; SELECT sum(n) FROM defaults; SELECT sumOrNull(n) FROM defaults; +SELECT count(n) FROM defaults; +SELECT countOrNull(n) FROM defaults; DROP TABLE defaults; From 4ff1dacaa0529410558a33141fade0f6fc91cc9c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 4 Nov 2020 14:20:11 +0000 Subject: [PATCH 086/149] Fix the test case for maxMap function --- src/AggregateFunctions/AggregateFunctionSumMap.h | 1 + tests/queries/0_stateless/01280_min_map_max_map.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 33fd87c0a87..c991474d0e6 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -256,6 +256,7 @@ public: // Final step does compaction of keys that have zero values, this mutates the state auto & merged_maps = this->data(place).merged_maps; + // Remove keys which are zeros or empty. This should be enabled only for sumMap. if constexpr (compact) { for (auto it = merged_maps.cbegin(); it != merged_maps.cend();) diff --git a/tests/queries/0_stateless/01280_min_map_max_map.sql b/tests/queries/0_stateless/01280_min_map_max_map.sql index 61d4a8ee4c9..cc07998bb79 100644 --- a/tests/queries/0_stateless/01280_min_map_max_map.sql +++ b/tests/queries/0_stateless/01280_min_map_max_map.sql @@ -36,5 +36,5 @@ select maxMap(val, cnt) from values ('val Array(Enum16(\'a\'=1)), cnt Array(Int1 select minMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); select maxMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); -select minMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); +select maxMap(val, cnt) from values ('val Array(String), cnt Array(String)', (['A'], ['']), (['B'], [''])); select sumMap(val, cnt) from values ('val Array(UInt64), cnt Array(UInt64)', ([1], [0]), ([2], [0])); From b8b9c6d40997416b5f5fabdaf7af27d53e07b6cc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 17:55:05 +0300 Subject: [PATCH 087/149] Update reinterpretAs.cpp --- src/Functions/reinterpretAs.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/reinterpretAs.cpp b/src/Functions/reinterpretAs.cpp index 76ee6155048..1c309505ac9 100644 --- a/src/Functions/reinterpretAs.cpp +++ b/src/Functions/reinterpretAs.cpp @@ -40,8 +40,9 @@ class FunctionReinterpretAs : public IFunction template static void reinterpretImpl(const PaddedPODArray & from, PaddedPODArray & to) { - to.resize(from.size()); - for (size_t i = 0; i < from.size(); ++i) + size_t size = from.size(); + to.resize(size); + for (size_t i = 0; i < size; ++i) { to[i] = unalignedLoad(&(from.data()[i])); } From 8d5fe1148c721a3065cbebcea5f02ae9689876f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 4 Nov 2020 18:07:34 +0300 Subject: [PATCH 088/149] add client config --- docker/test/performance-comparison/compare.sh | 2 +- .../config/client_config.xml | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 docker/test/performance-comparison/config/client_config.xml diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 08462298093..60d226f025d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1092,7 +1092,7 @@ function upload_results --secure \ --user "${CHPC_DATABASE_USER}" \ --password "${CHPC_DATABASE_PASSWORD}" \ - --config "ch/tests/config/client_config.xml" \ + --config "right/config/client_config.xml" \ --database perftest \ --date_time_input_format=best_effort \ --query " diff --git a/docker/test/performance-comparison/config/client_config.xml b/docker/test/performance-comparison/config/client_config.xml new file mode 100644 index 00000000000..9f590389dc3 --- /dev/null +++ b/docker/test/performance-comparison/config/client_config.xml @@ -0,0 +1,17 @@ + + + + + true + true + sslv2,sslv3 + true + + AcceptCertificateHandler + + + + From c496aaddb286bcac40aa649fbe2d1e8e9f0803d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 18:29:43 +0300 Subject: [PATCH 089/149] Add a test for #16231 --- tests/queries/0_stateless/01556_if_null.reference | 1 + tests/queries/0_stateless/01556_if_null.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01556_if_null.reference create mode 100644 tests/queries/0_stateless/01556_if_null.sql diff --git a/tests/queries/0_stateless/01556_if_null.reference b/tests/queries/0_stateless/01556_if_null.reference new file mode 100644 index 00000000000..a0c5e7faf40 --- /dev/null +++ b/tests/queries/0_stateless/01556_if_null.reference @@ -0,0 +1 @@ +([1],[5]) 4 4 diff --git a/tests/queries/0_stateless/01556_if_null.sql b/tests/queries/0_stateless/01556_if_null.sql new file mode 100644 index 00000000000..1952b4b7617 --- /dev/null +++ b/tests/queries/0_stateless/01556_if_null.sql @@ -0,0 +1,5 @@ +SELECT + sumMapIf([1], [1], nullIf(number, 3) > 0) AS col1, + countIf(1, nullIf(number, 3) > 0) AS col2, + sumIf(1, nullIf(number, 3) > 0) AS col3 +FROM numbers(1, 5); From 1bcf22d42fc211350f2ba830a508f67966fd6f93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 18:59:14 +0300 Subject: [PATCH 090/149] Fix 'max_parallel_replicas' without sampling. --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f06bfb97b2c..2ca989e12e6 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -366,6 +366,15 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( * It is also important that the entire universe can be covered using SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 and similar decimals. */ + /// Parallel replicas has been requested but there is no way to sample data. + /// Select all data from first replica and no data from other replicas. + if (settings.parallel_replicas_count > 1 && !data.supportsSampling() && settings.parallel_replica_offset > 0) + { + LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" + " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); + return {}; + } + bool use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); bool no_data = false; /// There is nothing left after sampling. From a4cfc240e3a76166dbde8f4cc9b80279af35c97d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 19:19:59 +0300 Subject: [PATCH 091/149] Add a test --- ..._max_parallel_replicas_no_sample.reference | 4 ++++ .../01557_max_parallel_replicas_no_sample.sql | 22 +++++++++++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference create mode 100644 tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference new file mode 100644 index 00000000000..2715babfff2 --- /dev/null +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.reference @@ -0,0 +1,4 @@ +Hello +1000 +1000 +1000 diff --git a/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql new file mode 100644 index 00000000000..d86f692a1ea --- /dev/null +++ b/tests/queries/0_stateless/01557_max_parallel_replicas_no_sample.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY x; +INSERT INTO t VALUES ('Hello'); + +SET max_parallel_replicas = 3; +SELECT * FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +DROP TABLE t; + +CREATE TABLE t (x String) ENGINE = MergeTree ORDER BY cityHash64(x) SAMPLE BY cityHash64(x); +INSERT INTO t SELECT toString(number) FROM numbers(1000); + +SET max_parallel_replicas = 1; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +SET max_parallel_replicas = 2; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +SET max_parallel_replicas = 3; +SELECT count() FROM remote('127.0.0.{2|3|4}', currentDatabase(), t); + +DROP TABLE t; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 8ece63c419c..900cc82b33f 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -159,3 +159,4 @@ 01547_query_log_current_database 01548_query_log_query_execution_ms 01552_dict_fixedstring +01557_max_parallel_replicas_no_sample.sql From 1c961c281cd7365a977e36f7360652fb4fcff101 Mon Sep 17 00:00:00 2001 From: Jacob Hayes Date: Wed, 4 Nov 2020 11:32:58 -0500 Subject: [PATCH 092/149] Add farmFingerprint64 test --- tests/queries/0_stateless/00751_hashing_ints.reference | 2 ++ tests/queries/0_stateless/00751_hashing_ints.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/00751_hashing_ints.reference b/tests/queries/0_stateless/00751_hashing_ints.reference index 8eccccd0449..31acc13a762 100644 --- a/tests/queries/0_stateless/00751_hashing_ints.reference +++ b/tests/queries/0_stateless/00751_hashing_ints.reference @@ -2,6 +2,8 @@ 11717965186011240346 13379111408315310133 13379111408315310133 +7773179648686038998 +13379111408315310133 13379111408315310133 623211862 9052087431341907723 diff --git a/tests/queries/0_stateless/00751_hashing_ints.sql b/tests/queries/0_stateless/00751_hashing_ints.sql index d2f0a26cef4..be3268da29d 100644 --- a/tests/queries/0_stateless/00751_hashing_ints.sql +++ b/tests/queries/0_stateless/00751_hashing_ints.sql @@ -1,6 +1,8 @@ SELECT halfMD5(123456); SELECT sipHash64(123456); SELECT cityHash64(123456); +SELECT farmFingerprint64(123456); +SELECT farmFingerprint64('123456'); SELECT farmHash64(123456); SELECT metroHash64(123456); SELECT murmurHash2_32(123456); From 22954cc12393f8a0ad9c345eec094cacd8d01fa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 4 Nov 2020 19:48:36 +0300 Subject: [PATCH 093/149] Fix UBSan report when trying to convert infinite floating point number to integer --- src/Common/FieldVisitors.h | 5 +++++ .../01557_field_infinite_convert_to_number.reference | 0 .../0_stateless/01557_field_infinite_convert_to_number.sql | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference create mode 100644 tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 4fcb3091833..421423131a2 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -3,6 +3,7 @@ #include #include #include +#include class SipHash; @@ -142,6 +143,10 @@ public: T operator() (const Float64 & x) const { + if constexpr (!std::is_floating_point_v) + if (!isFinite(x)) + throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE); + if constexpr (std::is_same_v) return Int256(x); else diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql new file mode 100644 index 00000000000..edc4d5cbc91 --- /dev/null +++ b/tests/queries/0_stateless/01557_field_infinite_convert_to_number.sql @@ -0,0 +1 @@ +SET max_threads = nan; -- { serverError 70 } From 0e0b154d244c204d0abf071d4ce9804051de6532 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 4 Nov 2020 23:07:56 +0300 Subject: [PATCH 094/149] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Внес поправки согласно комментариям ревьюера. --- .../server-configuration-parameters/settings.md | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 31a8e896438..ad7f92a76f4 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -471,7 +471,7 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa ## max_concurrent_queries {#max-concurrent-queries} -The maximum number of simultaneously processed requests. +The maximum number of simultaneously processed queries. **Example** @@ -483,11 +483,18 @@ The maximum number of simultaneously processed requests. Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. -Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - Modifying the setting for one query or user does not affect other queries. -Default value: `0` that means no limit. +Possible values: + +- Positive integer. +- 0 — Unlimited. + +Default value: `0`. + +**Usage** + +`max_concurrent_queries_for_all_users` can be set to 99 for all users, and database administrator can set it to 100 or even 0 for himself to run queries for investigation even when the ClickHouse server is overloaded. **Example** From 7dd749fc4c6ae8533b2e4f475899de46ec37f4b2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 4 Nov 2020 23:41:10 +0300 Subject: [PATCH 095/149] BloomFilter index added big integers and UUID support --- src/Interpreters/BloomFilterHash.h | 16 +++++++++--- src/Interpreters/convertFieldToType.cpp | 4 +++ .../MergeTree/MergeTreeIndexBloomFilter.cpp | 2 +- ...om_filter_index_big_integer_uuid.reference | 12 +++++++++ ...54_bloom_filter_index_big_integer_uuid.sql | 25 +++++++++++++++++++ 5 files changed, 55 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference create mode 100644 tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 43f5d7b5e87..b64f7bae79d 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -38,7 +38,7 @@ struct BloomFilterHash static UInt64 getNumberTypeHash(const Field & field) { /// For negative, we should convert the type to make sure the symbol is in right place - return field.isNull() ? intHash64(0) : intHash64(ext::bit_cast(FieldType(field.safeGet()))); + return field.isNull() ? intHash64(0) : DefaultHash64(FieldType(field.safeGet())); } static UInt64 getStringTypeHash(const Field & field) @@ -79,16 +79,21 @@ struct BloomFilterHash else if (which.isUInt16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isUInt32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isUInt64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUInt128()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUInt256()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt8()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isInt64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isInt128()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isInt256()) return build_hash_column(getNumberTypeHash(field)); else if (which.isEnum8()) return build_hash_column(getNumberTypeHash(field)); else if (which.isEnum16()) return build_hash_column(getNumberTypeHash(field)); else if (which.isDate()) return build_hash_column(getNumberTypeHash(field)); else if (which.isDateTime()) return build_hash_column(getNumberTypeHash(field)); else if (which.isFloat32()) return build_hash_column(getNumberTypeHash(field)); else if (which.isFloat64()) return build_hash_column(getNumberTypeHash(field)); + else if (which.isUUID()) return build_hash_column(getNumberTypeHash(field)); else if (which.isString()) return build_hash_column(getStringTypeHash(field)); else if (which.isFixedString()) return build_hash_column(getFixedStringTypeHash(field, data_type)); else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS); @@ -135,16 +140,21 @@ struct BloomFilterHash else if (which.isUInt16()) getNumberTypeHash(column, vec, pos); else if (which.isUInt32()) getNumberTypeHash(column, vec, pos); else if (which.isUInt64()) getNumberTypeHash(column, vec, pos); + else if (which.isUInt128()) getNumberTypeHash(column, vec, pos); + else if (which.isUInt256()) getNumberTypeHash(column, vec, pos); else if (which.isInt8()) getNumberTypeHash(column, vec, pos); else if (which.isInt16()) getNumberTypeHash(column, vec, pos); else if (which.isInt32()) getNumberTypeHash(column, vec, pos); else if (which.isInt64()) getNumberTypeHash(column, vec, pos); + else if (which.isInt128()) getNumberTypeHash(column, vec, pos); + else if (which.isInt256()) getNumberTypeHash(column, vec, pos); else if (which.isEnum8()) getNumberTypeHash(column, vec, pos); else if (which.isEnum16()) getNumberTypeHash(column, vec, pos); else if (which.isDate()) getNumberTypeHash(column, vec, pos); else if (which.isDateTime()) getNumberTypeHash(column, vec, pos); else if (which.isFloat32()) getNumberTypeHash(column, vec, pos); else if (which.isFloat64()) getNumberTypeHash(column, vec, pos); + else if (which.isUUID()) getNumberTypeHash(column, vec, pos); else if (which.isString()) getStringTypeHash(column, vec, pos); else if (which.isFixedString()) getStringTypeHash(column, vec, pos); else throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS); @@ -166,7 +176,7 @@ struct BloomFilterHash { for (size_t index = 0, size = vec.size(); index < size; ++index) { - UInt64 hash = intHash64(ext::bit_cast(Float64(vec_from[index + pos]))); + UInt64 hash = DefaultHash64(Float64(vec_from[index + pos])); if constexpr (is_first) vec[index] = hash; @@ -178,7 +188,7 @@ struct BloomFilterHash { for (size_t index = 0, size = vec.size(); index < size; ++index) { - UInt64 hash = intHash64(ext::bit_cast(vec_from[index + pos])); + UInt64 hash = DefaultHash64(vec_from[index + pos]); if constexpr (is_first) vec[index] = hash; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index c0e9d7edc13..18a3eeecb46 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -152,10 +152,14 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); if (which_type.isUInt64()) return convertNumericType(src, type); + if (which_type.isUInt128()) return convertNumericType(src, type); + if (which_type.isUInt256()) return convertNumericType(src, type); if (which_type.isInt8()) return convertNumericType(src, type); if (which_type.isInt16()) return convertNumericType(src, type); if (which_type.isInt32()) return convertNumericType(src, type); if (which_type.isInt64()) return convertNumericType(src, type); + if (which_type.isInt128()) return convertNumericType(src, type); + if (which_type.isInt256()) return convertNumericType(src, type); if (which_type.isFloat32()) return convertNumericType(src, type); if (which_type.isFloat64()) return convertNumericType(src, type); if (const auto * ptype = typeid_cast *>(&type)) return convertDecimalType(src, *ptype); diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 5a5dfffe287..a98ba16978d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -85,7 +85,7 @@ static void assertIndexColumnsType(const Block & header) WhichDataType which(actual_type); if (!which.isUInt() && !which.isInt() && !which.isString() && !which.isFixedString() && !which.isFloat() && - !which.isDateOrDateTime() && !which.isEnum()) + !which.isDateOrDateTime() && !which.isEnum() && !which.isUUID()) throw Exception("Unexpected type " + type->getName() + " of bloom filter index.", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference new file mode 100644 index 00000000000..39beae69d65 --- /dev/null +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.reference @@ -0,0 +1,12 @@ +1 +1 +2 +1 +1 +2 +1 +1 +2 +00000000-0000-0001-0000-000000000000 +00000000-0000-0001-0000-000000000000 +00000000-0000-0002-0000-000000000000 diff --git a/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql new file mode 100644 index 00000000000..265f931c556 --- /dev/null +++ b/tests/queries/0_stateless/01554_bloom_filter_index_big_integer_uuid.sql @@ -0,0 +1,25 @@ +SET allow_experimental_bigint_types = 1; + +CREATE TABLE 01154_test (x Int128, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x Int256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x UInt256, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (1), (2), (3); +SELECT x FROM 01154_test WHERE x = 1; +SELECT x FROM 01154_test WHERE x IN (1, 2); +DROP TABLE 01154_test; + +CREATE TABLE 01154_test (x UUID, INDEX ix_x x TYPE bloom_filter(0.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY x SETTINGS index_granularity=8192; +INSERT INTO 01154_test VALUES (toUUID(1)), (toUUID(2)), (toUUID(3)); +SELECT x FROM 01154_test WHERE x = toUUID(1); +SELECT x FROM 01154_test WHERE x IN (toUUID(1), toUUID(2)); +DROP TABLE 01154_test; From cd19509b5c2f1d2e538934b03ae5aa2f23eff12f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 4 Nov 2020 23:42:17 +0300 Subject: [PATCH 096/149] Update 01541_max_memory_usage_for_user.sh --- tests/queries/0_stateless/01541_max_memory_usage_for_user.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh index 3b1ced03473..64a90d871e7 100755 --- a/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh +++ b/tests/queries/0_stateless/01541_max_memory_usage_for_user.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # - one users' query in background (to avoid reseting max_memory_usage_for_user) # We have to create a separate user to run this tests isolated from other tests. -${CLICKHOUSE_CLIENT} -n --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; +${CLICKHOUSE_CLIENT} -n --allow_introspection_functions 1 --query "DROP USER IF EXISTS test_01541; CREATE USER test_01541; GRANT ALL ON *.* TO test_01541;"; query="SELECT groupArray(repeat('a', 1000)) FROM numbers(10000) GROUP BY number % 10 FORMAT JSON" From b1e75ec6f5b34de8ac8d53f8f7a4a1110edd2f84 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 4 Nov 2020 23:43:11 +0300 Subject: [PATCH 097/149] CreateQuery compound column type indendation fix --- src/Parsers/ASTColumnDeclaration.cpp | 6 +++++- .../0_stateless/01458_named_tuple_millin.reference | 4 ++-- .../queries/0_stateless/01458_named_tuple_millin.sql | 4 +--- ...548_create_table_compound_column_format.reference | 12 ++++++++++++ .../01548_create_table_compound_column_format.sh | 8 ++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01548_create_table_compound_column_format.reference create mode 100755 tests/queries/0_stateless/01548_create_table_compound_column_format.sh diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 730e892f8f7..0e0847713c2 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -55,7 +55,11 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta if (type) { settings.ostr << ' '; - type->formatImpl(settings, state, frame); + + FormatStateStacked typeFrame = frame; + typeFrame.indent = 0; + + type->formatImpl(settings, state, typeFrame); } if (null_modifier) diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.reference b/tests/queries/0_stateless/01458_named_tuple_millin.reference index b826566c74b..d6d6d7ae8d4 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.reference +++ b/tests/queries/0_stateless/01458_named_tuple_millin.reference @@ -1,12 +1,12 @@ CREATE TABLE default.tuple ( - `j` Tuple( a Int8, b String) + `j` Tuple(a Int8, b String) ) ENGINE = Memory j Tuple(a Int8, b String) CREATE TABLE default.tuple ( - `j` Tuple( a Int8, b String) + `j` Tuple(a Int8, b String) ) ENGINE = Memory j Tuple(a Int8, b String) diff --git a/tests/queries/0_stateless/01458_named_tuple_millin.sql b/tests/queries/0_stateless/01458_named_tuple_millin.sql index 7687dd4c158..ea730e65bb7 100644 --- a/tests/queries/0_stateless/01458_named_tuple_millin.sql +++ b/tests/queries/0_stateless/01458_named_tuple_millin.sql @@ -10,9 +10,7 @@ SHOW CREATE TABLE tuple FORMAT TSVRaw; DESC tuple; DROP TABLE tuple; -CREATE TABLE tuple -ENGINE = Memory AS -SELECT CAST((1, 'Test'), 'Tuple(a Int8, b String)') AS j; +CREATE TABLE tuple ENGINE = Memory AS SELECT CAST((1, 'Test'), 'Tuple(a Int8, b String)') AS j; SHOW CREATE TABLE tuple FORMAT TSVRaw; DESC tuple; diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.reference b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference new file mode 100644 index 00000000000..c6c4dcdfa4a --- /dev/null +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.reference @@ -0,0 +1,12 @@ +CREATE TABLE test +( + `a` Int64, + `b` NESTED(a Int64) +) +ENGINE = TinyLog +CREATE TABLE test +( + `a` Int64, + `b` TUPLE(a Int64) +) +ENGINE = TinyLog diff --git a/tests/queries/0_stateless/01548_create_table_compound_column_format.sh b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh new file mode 100755 index 00000000000..6c9384e01c1 --- /dev/null +++ b/tests/queries/0_stateless/01548_create_table_compound_column_format.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +echo "CREATE TABLE test(a Int64, b NESTED(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT + +echo "CREATE TABLE test(a Int64, b TUPLE(a Int64)) ENGINE=TinyLog" | $CLICKHOUSE_FORMAT \ No newline at end of file From 6b5c30ecf2084669415825f051931dadc4a52856 Mon Sep 17 00:00:00 2001 From: Grigory Buteyko Date: Thu, 5 Nov 2020 01:36:05 +0300 Subject: [PATCH 098/149] reverted tiny mistake added in previous commit. More reference to GCC 9 replaced --- docs/fa/development/build.md | 4 ++-- docs/fa/development/developer-instruction.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/fa/development/build.md b/docs/fa/development/build.md index b3d4fa1764e..8126578f9d5 100644 --- a/docs/fa/development/build.md +++ b/docs/fa/development/build.md @@ -20,7 +20,7 @@ $ sudo apt-get install git cmake python ninja-build یا سیمک 3 به جای کیک در سیستم های قدیمی تر. -## نصب شورای همکاری خلیج فارس 10 {install-gcc-10} +## نصب شورای همکاری خلیج فارس 9 {#install-gcc-10} راه های مختلفی برای انجام این کار وجود دارد. @@ -77,7 +77,7 @@ $ cd .. - دستگاه گوارش (استفاده می شود تنها به پرداخت منابع مورد نیاز برای ساخت) - کیک 3.10 یا جدیدتر - نینجا (توصیه می شود) و یا -- ج ++ کامپایلر: شورای همکاری خلیج فارس 9 یا صدای شیپور 8 یا جدیدتر +- ج ++ کامپایلر: شورای همکاری خلیج فارس 10 یا صدای شیپور 8 یا جدیدتر - لینکر: لیلند یا طلا (کلاسیک گنو الدی کار نخواهد کرد) - پایتون (فقط در داخل ساخت لورم استفاده می شود و اختیاری است) diff --git a/docs/fa/development/developer-instruction.md b/docs/fa/development/developer-instruction.md index 01fb24ac006..ee78050da07 100644 --- a/docs/fa/development/developer-instruction.md +++ b/docs/fa/development/developer-instruction.md @@ -143,7 +143,7 @@ toc_title: "\u062F\u0633\u062A\u0648\u0631\u0627\u0644\u0639\u0645\u0644 \u062A\ برای نصب شورای همکاری خلیج فارس در اوبونتو اجرای: `sudo apt install gcc g++` -بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 9, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-10. +بررسی نسخه شورای همکاری خلیج فارس: `gcc --version`. اگر زیر است 10, سپس دستورالعمل اینجا را دنبال کنید: https://clickhouse.tech/docs/fa/development/build/#install-gcc-10. سیستم عامل مک ایکس ساخت فقط برای صدای جرنگ جرنگ پشتیبانی می شود. فقط فرار کن `brew install llvm` From 52463063fc454ed8a3bbaff62724a6680e2c4844 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 5 Nov 2020 06:07:36 +0000 Subject: [PATCH 099/149] Bump mkdocs-macros-plugin from 0.4.17 to 0.4.20 in /docs/tools Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 0.4.17 to 0.4.20. - [Release notes](https://github.com/fralau/mkdocs_macros_plugin/releases) - [Commits](https://github.com/fralau/mkdocs_macros_plugin/commits/v0.4.20) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 3c59a601ff2..3560b6ad40a 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -18,7 +18,7 @@ Markdown==3.3.2 MarkupSafe==1.1.1 mkdocs==1.1.2 mkdocs-htmlproofer-plugin==0.0.3 -mkdocs-macros-plugin==0.4.17 +mkdocs-macros-plugin==0.4.20 nltk==3.5 nose==1.3.7 protobuf==3.13.0 From 65aa226bcd09fd9bccb1c9cb91d2ac183c8ed9c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 12:26:51 +0300 Subject: [PATCH 100/149] Fix test --- src/Common/FieldVisitors.h | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/FieldVisitors.h b/src/Common/FieldVisitors.h index 421423131a2..79d5af88904 100644 --- a/src/Common/FieldVisitors.h +++ b/src/Common/FieldVisitors.h @@ -144,8 +144,17 @@ public: T operator() (const Float64 & x) const { if constexpr (!std::is_floating_point_v) + { if (!isFinite(x)) + { + /// When converting to bool it's ok (non-zero converts to true, NaN including). + if (std::is_same_v) + return true; + + /// Conversion of infinite values to integer is undefined. throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE); + } + } if constexpr (std::is_same_v) return Int256(x); From f06abe03dafc3740768554544b15278f3ee6e1e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 12:48:53 +0300 Subject: [PATCH 101/149] utils/generate-ya-make/generate-ya-make.sh --- src/Functions/ya.make | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index bc5af88b6af..3a4e7b6b5c1 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -357,9 +357,9 @@ SRCS( registerFunctionsTuple.cpp registerFunctionsUnixTimestamp64.cpp registerFunctionsVisitParam.cpp + reinterpretAs.cpp reinterpretAsFixedString.cpp reinterpretAsString.cpp - reinterpretAs.cpp repeat.cpp replaceAll.cpp replaceOne.cpp From b56486510f553f7032a8373527ea5a85f1617de7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 12:55:01 +0300 Subject: [PATCH 102/149] Improve Arcadia --- base/common/ya.make | 10 +- src/Access/ya.make | 6 +- src/AggregateFunctions/ya.make | 8 +- src/Columns/ya.make | 5 +- src/Columns/ya.make.in | 19 +++ src/Common/ya.make | 60 +++---- src/Compression/ya.make | 4 +- src/Core/ya.make | 2 +- src/DataStreams/ya.make | 8 +- src/DataTypes/ya.make | 12 +- src/Databases/ya.make | 2 +- src/Dictionaries/ya.make | 12 +- src/Disks/ya.make | 4 +- src/Functions/ya.make | 182 ++++++++++----------- src/IO/ya.make | 20 +-- src/Interpreters/ya.make | 40 ++--- src/Parsers/ya.make | 28 ++-- src/Processors/ya.make | 14 +- src/Storages/ya.make | 36 ++-- src/TableFunctions/ya.make | 4 +- utils/generate-ya-make/generate-ya-make.sh | 3 + 21 files changed, 250 insertions(+), 229 deletions(-) create mode 100644 src/Columns/ya.make.in diff --git a/base/common/ya.make b/base/common/ya.make index cbb6b5f64ac..02e0e90fe58 100644 --- a/base/common/ya.make +++ b/base/common/ya.make @@ -35,25 +35,25 @@ PEERDIR( CFLAGS(-g0) SRCS( - argsToConfig.cpp - coverage.cpp DateLUT.cpp DateLUTImpl.cpp + JSON.cpp + LineReader.cpp + StringRef.cpp + argsToConfig.cpp + coverage.cpp demangle.cpp errnoToString.cpp getFQDNOrHostName.cpp getMemoryAmount.cpp getResource.cpp getThreadId.cpp - JSON.cpp - LineReader.cpp mremap.cpp phdr_cache.cpp preciseExp10.cpp setTerminalEcho.cpp shift10.cpp sleep.cpp - StringRef.cpp terminalColors.cpp ) diff --git a/src/Access/ya.make b/src/Access/ya.make index 1ec8cb32c97..97640344498 100644 --- a/src/Access/ya.make +++ b/src/Access/ya.make @@ -27,14 +27,14 @@ SRCS( LDAPClient.cpp MemoryAccessStorage.cpp MultipleAccessStorage.cpp - QuotaCache.cpp Quota.cpp + QuotaCache.cpp QuotaUsage.cpp - RoleCache.cpp Role.cpp + RoleCache.cpp RolesOrUsersSet.cpp - RowPolicyCache.cpp RowPolicy.cpp + RowPolicyCache.cpp SettingsConstraints.cpp SettingsProfile.cpp SettingsProfileElement.cpp diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make index 80b8de0eea7..e6aedc513f9 100644 --- a/src/AggregateFunctions/ya.make +++ b/src/AggregateFunctions/ya.make @@ -26,10 +26,10 @@ SRCS( AggregateFunctionGroupUniqArray.cpp AggregateFunctionHistogram.cpp AggregateFunctionIf.cpp + AggregateFunctionMLMethod.cpp AggregateFunctionMaxIntersections.cpp AggregateFunctionMerge.cpp AggregateFunctionMinMaxAny.cpp - AggregateFunctionMLMethod.cpp AggregateFunctionNull.cpp AggregateFunctionOrFill.cpp AggregateFunctionQuantile.cpp @@ -45,14 +45,14 @@ SRCS( AggregateFunctionSumMap.cpp AggregateFunctionTimeSeriesGroupSum.cpp AggregateFunctionTopK.cpp - AggregateFunctionUniqCombined.cpp AggregateFunctionUniq.cpp + AggregateFunctionUniqCombined.cpp AggregateFunctionUniqUpTo.cpp AggregateFunctionWindowFunnel.cpp - parseAggregateFunctionParameters.cpp - registerAggregateFunctions.cpp UniqCombinedBiasData.cpp UniqVariadicHash.cpp + parseAggregateFunctionParameters.cpp + registerAggregateFunctions.cpp ) diff --git a/src/Columns/ya.make b/src/Columns/ya.make index 1463bbc69e2..56c25529cfd 100644 --- a/src/Columns/ya.make +++ b/src/Columns/ya.make @@ -13,7 +13,6 @@ PEERDIR( contrib/libs/pdqsort ) - SRCS( Collator.cpp ColumnAggregateFunction.cpp @@ -24,13 +23,13 @@ SRCS( ColumnFunction.cpp ColumnLowCardinality.cpp ColumnNullable.cpp - ColumnsCommon.cpp ColumnString.cpp ColumnTuple.cpp ColumnVector.cpp + ColumnsCommon.cpp FilterDescription.cpp - getLeastSuperColumn.cpp IColumn.cpp + getLeastSuperColumn.cpp ) diff --git a/src/Columns/ya.make.in b/src/Columns/ya.make.in new file mode 100644 index 00000000000..4e667b49b7c --- /dev/null +++ b/src/Columns/ya.make.in @@ -0,0 +1,19 @@ +LIBRARY() + +ADDINCL( + contrib/libs/icu/common + contrib/libs/icu/i18n + contrib/libs/pdqsort +) + +PEERDIR( + clickhouse/src/Common + contrib/libs/icu + contrib/libs/pdqsort +) + +SRCS( + +) + +END() diff --git a/src/Common/ya.make b/src/Common/ya.make index b19a5183201..0d6caa22f3a 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -26,60 +26,36 @@ SRCS( ActionLock.cpp AlignedBuffer.cpp Allocator.cpp - checkStackSize.cpp - clearPasswordFromCommandLine.cpp ClickHouseRevision.cpp Config/AbstractConfigurationComparison.cpp Config/ConfigProcessor.cpp - Config/configReadClient.cpp Config/ConfigReloader.cpp - createHardLink.cpp + Config/configReadClient.cpp CurrentMetrics.cpp CurrentThread.cpp DNSResolver.cpp Dwarf.cpp Elf.cpp ErrorCodes.cpp - escapeForFileName.cpp Exception.cpp ExternalLoaderStatus.cpp FieldVisitors.cpp FileChecker.cpp - filesystemHelpers.cpp - formatIPv6.cpp - formatReadable.cpp - getExecutablePath.cpp - getMappedArea.cpp - getMultipleKeysFromConfig.cpp - getNumberOfPhysicalCPUCores.cpp - hasLinuxCapability.cpp - hex.cpp - IntervalKind.cpp IPv6ToBinary.cpp - isLocalAddress.cpp + IntervalKind.cpp Macros.cpp - malloc.cpp MemoryStatisticsOS.cpp MemoryTracker.cpp - new_delete.cpp OpenSSLHelpers.cpp OptimizedRegularExpression.cpp - parseAddress.cpp - parseGlobs.cpp - parseRemoteDescription.cpp - PipeFDs.cpp PODArray.cpp + PipeFDs.cpp ProcfsMetricsProvider.cpp ProfileEvents.cpp QueryProfiler.cpp - quoteString.cpp - randomSeed.cpp - remapExecutable.cpp - RemoteHostFilter.cpp - renameat2.cpp RWLock.cpp + RemoteHostFilter.cpp SensitiveDataMasker.cpp - setThreadName.cpp SettingsChanges.cpp SharedLibrary.cpp ShellCommand.cpp @@ -93,19 +69,43 @@ SRCS( TaskStatsInfoGetter.cpp TerminalSize.cpp ThreadFuzzer.cpp - thread_local_rng.cpp ThreadPool.cpp ThreadProfileEvents.cpp ThreadStatus.cpp TraceCollector.cpp - UnicodeBar.cpp UTF8Helpers.cpp + UnicodeBar.cpp WeakHash.cpp ZooKeeper/IKeeper.cpp ZooKeeper/TestKeeper.cpp ZooKeeper/ZooKeeper.cpp ZooKeeper/ZooKeeperImpl.cpp ZooKeeper/ZooKeeperNodeCache.cpp + checkStackSize.cpp + clearPasswordFromCommandLine.cpp + createHardLink.cpp + escapeForFileName.cpp + filesystemHelpers.cpp + formatIPv6.cpp + formatReadable.cpp + getExecutablePath.cpp + getMappedArea.cpp + getMultipleKeysFromConfig.cpp + getNumberOfPhysicalCPUCores.cpp + hasLinuxCapability.cpp + hex.cpp + isLocalAddress.cpp + malloc.cpp + new_delete.cpp + parseAddress.cpp + parseGlobs.cpp + parseRemoteDescription.cpp + quoteString.cpp + randomSeed.cpp + remapExecutable.cpp + renameat2.cpp + setThreadName.cpp + thread_local_rng.cpp ) diff --git a/src/Compression/ya.make b/src/Compression/ya.make index 8ffcb6be547..ed762bcd35f 100644 --- a/src/Compression/ya.make +++ b/src/Compression/ya.make @@ -15,8 +15,8 @@ PEERDIR( SRCS( CachedCompressedReadBuffer.cpp - CompressedReadBufferBase.cpp CompressedReadBuffer.cpp + CompressedReadBufferBase.cpp CompressedReadBufferFromFile.cpp CompressedWriteBuffer.cpp CompressionCodecDelta.cpp @@ -28,9 +28,9 @@ SRCS( CompressionCodecT64.cpp CompressionCodecZSTD.cpp CompressionFactory.cpp - getCompressionCodecForFile.cpp ICompressionCodec.cpp LZ4_decompress_faster.cpp + getCompressionCodecForFile.cpp ) diff --git a/src/Core/ya.make b/src/Core/ya.make index 626662e992c..6bef761a193 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -17,7 +17,6 @@ SRCS( ExternalResultDescription.cpp ExternalTable.cpp Field.cpp - iostream_debug_helpers.cpp MySQL/Authentication.cpp MySQL/IMySQLReadPacket.cpp MySQL/IMySQLWritePacket.cpp @@ -35,6 +34,7 @@ SRCS( SettingsEnums.cpp SettingsFields.cpp SortDescription.cpp + iostream_debug_helpers.cpp ) diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index adef8246f33..bb6dd3f9357 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -19,21 +19,17 @@ SRCS( CheckSortedBlockInputStream.cpp ColumnGathererStream.cpp ConvertingBlockInputStream.cpp - copyData.cpp CountingBlockOutputStream.cpp DistinctSortedBlockInputStream.cpp ExecutionSpeedLimits.cpp ExpressionBlockInputStream.cpp - finalizeBlock.cpp IBlockInputStream.cpp InputStreamFromASTInsertQuery.cpp InternalTextLogsRowOutputStream.cpp LimitBlockInputStream.cpp - materializeBlock.cpp MaterializingBlockInputStream.cpp MergingSortedBlockInputStream.cpp MongoDBBlockInputStream.cpp - narrowBlockInputStreams.cpp NativeBlockInputStream.cpp NativeBlockOutputStream.cpp ParallelParsingBlockInputStream.cpp @@ -46,6 +42,10 @@ SRCS( SquashingBlockOutputStream.cpp SquashingTransform.cpp TTLBlockInputStream.cpp + copyData.cpp + finalizeBlock.cpp + materializeBlock.cpp + narrowBlockInputStreams.cpp ) diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 97b600f70ba..91d28a08f22 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -8,7 +8,6 @@ PEERDIR( SRCS( - convertMySQLDataType.cpp DataTypeAggregateFunction.cpp DataTypeArray.cpp DataTypeCustomGeo.cpp @@ -16,8 +15,8 @@ SRCS( DataTypeCustomSimpleAggregateFunction.cpp DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp - DataTypeDateTime64.cpp DataTypeDateTime.cpp + DataTypeDateTime64.cpp DataTypeDecimalBase.cpp DataTypeEnum.cpp DataTypeFactory.cpp @@ -29,16 +28,17 @@ SRCS( DataTypeNothing.cpp DataTypeNullable.cpp DataTypeNumberBase.cpp - DataTypesDecimal.cpp - DataTypesNumber.cpp DataTypeString.cpp DataTypeTuple.cpp DataTypeUUID.cpp + DataTypesDecimal.cpp + DataTypesNumber.cpp FieldToDataType.cpp - getLeastSupertype.cpp - getMostSubtype.cpp IDataType.cpp NestedUtils.cpp + convertMySQLDataType.cpp + getLeastSupertype.cpp + getMostSubtype.cpp registerDataTypeDateTime.cpp ) diff --git a/src/Databases/ya.make b/src/Databases/ya.make index e3c5daeb6bc..0c6cfae29c3 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -14,8 +14,8 @@ SRCS( DatabaseMemory.cpp DatabaseOnDisk.cpp DatabaseOrdinary.cpp - DatabasesCommon.cpp DatabaseWithDictionaries.cpp + DatabasesCommon.cpp MySQL/ConnectionMySQLSettings.cpp MySQL/DatabaseConnectionMySQL.cpp MySQL/DatabaseMaterializeMySQL.cpp diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 9edf156c015..d12db283cae 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -37,11 +37,11 @@ SRCS( DictionarySourceHelpers.cpp DictionaryStructure.cpp DirectDictionary.cpp + Embedded/GeoDictionariesLoader.cpp Embedded/GeodataProviders/HierarchiesProvider.cpp Embedded/GeodataProviders/HierarchyFormatReader.cpp Embedded/GeodataProviders/NamesFormatReader.cpp Embedded/GeodataProviders/NamesProvider.cpp - Embedded/GeoDictionariesLoader.cpp Embedded/RegionsHierarchies.cpp Embedded/RegionsHierarchy.cpp Embedded/RegionsNames.cpp @@ -49,9 +49,8 @@ SRCS( ExternalQueryBuilder.cpp FileDictionarySource.cpp FlatDictionary.cpp - getDictionaryConfigurationFromAST.cpp - HashedDictionary.cpp HTTPDictionarySource.cpp + HashedDictionary.cpp LibraryDictionarySource.cpp LibraryDictionarySourceExternal.cpp MongoDBDictionarySource.cpp @@ -60,14 +59,15 @@ SRCS( PolygonDictionaryImplementations.cpp PolygonDictionaryUtils.cpp RangeHashedDictionary.cpp - readInvalidateQuery.cpp RedisBlockInputStream.cpp RedisDictionarySource.cpp - registerDictionaries.cpp SSDCacheDictionary.cpp SSDComplexKeyCacheDictionary.cpp - writeParenthesisedString.cpp XDBCDictionarySource.cpp + getDictionaryConfigurationFromAST.cpp + readInvalidateQuery.cpp + registerDictionaries.cpp + writeParenthesisedString.cpp ) diff --git a/src/Disks/ya.make b/src/Disks/ya.make index d14bc0d05c8..ea204ff09ec 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -7,7 +7,6 @@ PEERDIR( SRCS( - createVolume.cpp DiskCacheWrapper.cpp DiskDecorator.cpp DiskFactory.cpp @@ -16,11 +15,12 @@ SRCS( DiskSelector.cpp IDisk.cpp IVolume.cpp - registerDisks.cpp SingleDiskVolume.cpp StoragePolicy.cpp VolumeJBOD.cpp VolumeRAID1.cpp + createVolume.cpp + registerDisks.cpp ) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 3aea31aa538..a756cdbf7b1 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -34,6 +34,70 @@ PEERDIR( # "Arcadia" build is slightly deficient. It lacks many libraries that we need. SRCS( + CRC.cpp + FunctionFQDN.cpp + FunctionFactory.cpp + FunctionHelpers.cpp + FunctionJoinGet.cpp + FunctionsAES.cpp + FunctionsCoding.cpp + FunctionsConversion.cpp + FunctionsEmbeddedDictionaries.cpp + FunctionsExternalDictionaries.cpp + FunctionsExternalModels.cpp + FunctionsHashing.cpp + FunctionsJSON.cpp + FunctionsLogical.cpp + FunctionsRandom.cpp + FunctionsRound.cpp + FunctionsStringArray.cpp + FunctionsStringSimilarity.cpp + GatherUtils/concat.cpp + GatherUtils/createArraySink.cpp + GatherUtils/createArraySource.cpp + GatherUtils/createValueSource.cpp + GatherUtils/has_all.cpp + GatherUtils/has_any.cpp + GatherUtils/has_substr.cpp + GatherUtils/push.cpp + GatherUtils/resizeConstantSize.cpp + GatherUtils/resizeDynamicSize.cpp + GatherUtils/sliceDynamicOffsetBounded.cpp + GatherUtils/sliceDynamicOffsetUnbounded.cpp + GatherUtils/sliceFromLeftConstantOffsetBounded.cpp + GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp + GatherUtils/sliceFromRightConstantOffsetBounded.cpp + GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp + GeoHash.cpp + IFunction.cpp + TargetSpecific.cpp + URL/URLHierarchy.cpp + URL/URLPathHierarchy.cpp + URL/basename.cpp + URL/cutFragment.cpp + URL/cutQueryString.cpp + URL/cutQueryStringAndFragment.cpp + URL/cutToFirstSignificantSubdomain.cpp + URL/cutURLParameter.cpp + URL/cutWWW.cpp + URL/decodeURLComponent.cpp + URL/domain.cpp + URL/domainWithoutWWW.cpp + URL/extractURLParameter.cpp + URL/extractURLParameterNames.cpp + URL/extractURLParameters.cpp + URL/firstSignificantSubdomain.cpp + URL/fragment.cpp + URL/netloc.cpp + URL/path.cpp + URL/pathFull.cpp + URL/port.cpp + URL/protocol.cpp + URL/queryString.cpp + URL/queryStringAndFragment.cpp + URL/registerFunctionsURL.cpp + URL/tldLookup.generated.cpp + URL/topLevelDomain.cpp abs.cpp acos.cpp addDays.cpp @@ -41,20 +105,20 @@ SRCS( addMinutes.cpp addMonths.cpp addQuarters.cpp - addressToLine.cpp - addressToSymbol.cpp addSeconds.cpp addWeeks.cpp addYears.cpp + addressToLine.cpp + addressToSymbol.cpp aes_decrypt_mysql.cpp aes_encrypt_mysql.cpp appendTrailingCharIfAbsent.cpp - array/arrayAll.cpp + array/array.cpp array/arrayAUC.cpp + array/arrayAll.cpp array/arrayCompact.cpp array/arrayConcat.cpp array/arrayCount.cpp - array/array.cpp array/arrayCumSum.cpp array/arrayCumSumNonNegative.cpp array/arrayDifference.cpp @@ -93,9 +157,9 @@ SRCS( array/countEqual.cpp array/emptyArray.cpp array/emptyArrayToSingle.cpp + array/has.cpp array/hasAll.cpp array/hasAny.cpp - array/has.cpp array/hasSubstr.cpp array/indexOf.cpp array/length.cpp @@ -113,7 +177,6 @@ SRCS( bitBoolMaskAnd.cpp bitBoolMaskOr.cpp bitCount.cpp - bitmaskToList.cpp bitNot.cpp bitOr.cpp bitRotateLeft.cpp @@ -121,11 +184,12 @@ SRCS( bitShiftLeft.cpp bitShiftRight.cpp bitSwapLastTwo.cpp + bitTest.cpp bitTestAll.cpp bitTestAny.cpp - bitTest.cpp bitWrapperFunc.cpp bitXor.cpp + bitmaskToList.cpp blockNumber.cpp blockSerializedSize.cpp blockSize.cpp @@ -137,7 +201,6 @@ SRCS( convertCharset.cpp cos.cpp countDigits.cpp - CRC.cpp currentDatabase.cpp currentUser.cpp dateDiff.cpp @@ -153,16 +216,16 @@ SRCS( encrypt.cpp endsWith.cpp equals.cpp - erfc.cpp erf.cpp + erfc.cpp errorCodeToName.cpp evalMLMethod.cpp + exp.cpp exp10.cpp exp2.cpp - exp.cpp + extract.cpp extractAllGroupsHorizontal.cpp extractAllGroupsVertical.cpp - extract.cpp extractGroups.cpp extractTimeZoneFromFunctionArguments.cpp filesystem.cpp @@ -176,47 +239,13 @@ SRCS( fromUnixTimestamp64Micro.cpp fromUnixTimestamp64Milli.cpp fromUnixTimestamp64Nano.cpp - FunctionFactory.cpp - FunctionFQDN.cpp - FunctionHelpers.cpp - FunctionJoinGet.cpp - FunctionsAES.cpp - FunctionsCoding.cpp - FunctionsConversion.cpp - FunctionsEmbeddedDictionaries.cpp - FunctionsExternalDictionaries.cpp - FunctionsExternalModels.cpp - FunctionsHashing.cpp - FunctionsJSON.cpp - FunctionsLogical.cpp - FunctionsRandom.cpp - FunctionsRound.cpp - FunctionsStringArray.cpp - FunctionsStringSimilarity.cpp fuzzBits.cpp - GatherUtils/concat.cpp - GatherUtils/createArraySink.cpp - GatherUtils/createArraySource.cpp - GatherUtils/createValueSource.cpp - GatherUtils/has_all.cpp - GatherUtils/has_any.cpp - GatherUtils/has_substr.cpp - GatherUtils/push.cpp - GatherUtils/resizeConstantSize.cpp - GatherUtils/resizeDynamicSize.cpp - GatherUtils/sliceDynamicOffsetBounded.cpp - GatherUtils/sliceDynamicOffsetUnbounded.cpp - GatherUtils/sliceFromLeftConstantOffsetBounded.cpp - GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp - GatherUtils/sliceFromRightConstantOffsetBounded.cpp - GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp gcd.cpp generateUUIDv4.cpp - GeoHash.cpp + geoToH3.cpp geohashDecode.cpp geohashEncode.cpp geohashesInBox.cpp - geoToH3.cpp getMacro.cpp getScalar.cpp getSetting.cpp @@ -233,20 +262,19 @@ SRCS( h3HexAreaM2.cpp h3IndexesAreNeighbors.cpp h3IsValid.cpp - h3kRing.cpp h3ToChildren.cpp h3ToParent.cpp h3ToString.cpp + h3kRing.cpp hasColumnInTable.cpp hasThreadFuzzer.cpp - hasTokenCaseInsensitive.cpp hasToken.cpp + hasTokenCaseInsensitive.cpp hostName.cpp identity.cpp if.cpp ifNotFinite.cpp ifNull.cpp - IFunction.cpp ignore.cpp ilike.cpp in.cpp @@ -272,9 +300,9 @@ SRCS( lessOrEquals.cpp lgamma.cpp like.cpp + log.cpp log10.cpp log2.cpp - log.cpp logTrace.cpp lowCardinalityIndices.cpp lowCardinalityKeys.cpp @@ -292,56 +320,56 @@ SRCS( multiMatchAllIndices.cpp multiMatchAny.cpp multiMatchAnyIndex.cpp - multiply.cpp + multiSearchAllPositions.cpp multiSearchAllPositionsCaseInsensitive.cpp multiSearchAllPositionsCaseInsensitiveUTF8.cpp - multiSearchAllPositions.cpp multiSearchAllPositionsUTF8.cpp + multiSearchAny.cpp multiSearchAnyCaseInsensitive.cpp multiSearchAnyCaseInsensitiveUTF8.cpp - multiSearchAny.cpp multiSearchAnyUTF8.cpp + multiSearchFirstIndex.cpp multiSearchFirstIndexCaseInsensitive.cpp multiSearchFirstIndexCaseInsensitiveUTF8.cpp - multiSearchFirstIndex.cpp multiSearchFirstIndexUTF8.cpp + multiSearchFirstPosition.cpp multiSearchFirstPositionCaseInsensitive.cpp multiSearchFirstPositionCaseInsensitiveUTF8.cpp - multiSearchFirstPosition.cpp multiSearchFirstPositionUTF8.cpp + multiply.cpp negate.cpp neighbor.cpp - normalizedQueryHash.cpp normalizeQuery.cpp + normalizedQueryHash.cpp notEmpty.cpp notEquals.cpp notILike.cpp notLike.cpp - now64.cpp now.cpp + now64.cpp nullIf.cpp pi.cpp plus.cpp pointInEllipses.cpp pointInPolygon.cpp + position.cpp positionCaseInsensitive.cpp positionCaseInsensitiveUTF8.cpp - position.cpp positionUTF8.cpp pow.cpp + rand.cpp rand64.cpp randConstant.cpp - rand.cpp randomFixedString.cpp randomPrintableASCII.cpp randomString.cpp randomStringUTF8.cpp regexpQuoteMeta.cpp + registerFunctions.cpp registerFunctionsArithmetic.cpp registerFunctionsComparison.cpp registerFunctionsConditional.cpp registerFunctionsConsistentHashing.cpp - registerFunctions.cpp registerFunctionsDateTime.cpp registerFunctionsFormatting.cpp registerFunctionsGeo.cpp @@ -395,7 +423,6 @@ SRCS( subtractYears.cpp tan.cpp tanh.cpp - TargetSpecific.cpp tgamma.cpp throwIf.cpp tid.cpp @@ -404,7 +431,6 @@ SRCS( timezone.cpp toColumnTypeName.cpp toCustomWeek.cpp - today.cpp toDayOfMonth.cpp toDayOfWeek.cpp toDayOfYear.cpp @@ -431,8 +457,8 @@ SRCS( toStartOfFifteenMinutes.cpp toStartOfFiveMinute.cpp toStartOfHour.cpp - toStartOfInterval.cpp toStartOfISOYear.cpp + toStartOfInterval.cpp toStartOfMinute.cpp toStartOfMonth.cpp toStartOfQuarter.cpp @@ -446,10 +472,11 @@ SRCS( toUnixTimestamp64Milli.cpp toUnixTimestamp64Nano.cpp toValidUTF8.cpp - toYear.cpp toYYYYMM.cpp toYYYYMMDD.cpp toYYYYMMDDhhmmss.cpp + toYear.cpp + today.cpp transform.cpp trap.cpp trim.cpp @@ -459,33 +486,6 @@ SRCS( upper.cpp upperUTF8.cpp uptime.cpp - URL/basename.cpp - URL/cutFragment.cpp - URL/cutQueryStringAndFragment.cpp - URL/cutQueryString.cpp - URL/cutToFirstSignificantSubdomain.cpp - URL/cutURLParameter.cpp - URL/cutWWW.cpp - URL/decodeURLComponent.cpp - URL/domain.cpp - URL/domainWithoutWWW.cpp - URL/extractURLParameter.cpp - URL/extractURLParameterNames.cpp - URL/extractURLParameters.cpp - URL/firstSignificantSubdomain.cpp - URL/fragment.cpp - URL/netloc.cpp - URL/path.cpp - URL/pathFull.cpp - URL/port.cpp - URL/protocol.cpp - URL/queryStringAndFragment.cpp - URL/queryString.cpp - URL/registerFunctionsURL.cpp - URL/tldLookup.generated.cpp - URL/topLevelDomain.cpp - URL/URLHierarchy.cpp - URL/URLPathHierarchy.cpp version.cpp visibleWidth.cpp visitParamExtractBool.cpp diff --git a/src/IO/ya.make b/src/IO/ya.make index 3796494ff33..d20c01faaa3 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -10,45 +10,40 @@ PEERDIR( SRCS( - AIOContextPool.cpp AIO.cpp + AIOContextPool.cpp BrotliReadBuffer.cpp BrotliWriteBuffer.cpp CascadeWriteBuffer.cpp CompressionMethod.cpp - copyData.cpp - createReadBufferFromFileBase.cpp - createWriteBufferFromFileBase.cpp DoubleConverter.cpp + HTTPCommon.cpp HashingWriteBuffer.cpp HexWriteBuffer.cpp - HTTPCommon.cpp LimitReadBuffer.cpp - MemoryReadWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp + MemoryReadWriteBuffer.cpp MySQLBinlogEventReadBuffer.cpp MySQLPacketPayloadReadBuffer.cpp MySQLPacketPayloadWriteBuffer.cpp NullWriteBuffer.cpp - parseDateTimeBestEffort.cpp PeekableReadBuffer.cpp Progress.cpp ReadBufferAIO.cpp - ReadBufferFromFileBase.cpp ReadBufferFromFile.cpp + ReadBufferFromFileBase.cpp ReadBufferFromFileDescriptor.cpp ReadBufferFromIStream.cpp ReadBufferFromMemory.cpp ReadBufferFromPocoSocket.cpp - readFloatText.cpp ReadHelpers.cpp ReadWriteBufferFromHTTP.cpp SeekAvoidingReadBuffer.cpp UseSSL.cpp WriteBufferAIO.cpp - WriteBufferFromFileBase.cpp WriteBufferFromFile.cpp + WriteBufferFromFileBase.cpp WriteBufferFromFileDescriptor.cpp WriteBufferFromFileDescriptorDiscardOnFailure.cpp WriteBufferFromHTTP.cpp @@ -60,6 +55,11 @@ SRCS( WriteHelpers.cpp ZlibDeflatingWriteBuffer.cpp ZlibInflatingReadBuffer.cpp + copyData.cpp + createReadBufferFromFileBase.cpp + createWriteBufferFromFileBase.cpp + parseDateTimeBestEffort.cpp + readFloatText.cpp ) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 1c463eff7e4..02e67116233 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -18,8 +18,6 @@ NO_COMPILER_WARNINGS() SRCS( ActionLocksManager.cpp ActionsVisitor.cpp - addMissingDefaults.cpp - addTypeConversionToAST.cpp AggregateDescription.cpp Aggregator.cpp ApplyWithAliasVisitor.cpp @@ -30,26 +28,21 @@ SRCS( AsynchronousMetricLog.cpp AsynchronousMetrics.cpp BloomFilter.cpp - castColumn.cpp CatBoostModel.cpp ClientInfo.cpp Cluster.cpp - ClusterProxy/executeQuery.cpp ClusterProxy/SelectStreamFactory.cpp + ClusterProxy/executeQuery.cpp CollectJoinOnKeysVisitor.cpp Context.cpp - convertFieldToType.cpp CrashLog.cpp - createBlockSelector.cpp CrossToInnerJoinVisitor.cpp + DDLWorker.cpp + DNSCacheUpdater.cpp DatabaseAndTableWithAlias.cpp DatabaseCatalog.cpp - DDLWorker.cpp DictionaryReader.cpp - DNSCacheUpdater.cpp EmbeddedDictionaries.cpp - evaluateConstantExpression.cpp - executeQuery.cpp ExecuteScalarSubqueriesVisitor.cpp ExpressionActions.cpp ExpressionAnalyzer.cpp @@ -61,14 +54,10 @@ SRCS( ExternalModelsLoader.cpp ExtractExpressionInfoVisitor.cpp FillingRow.cpp - getClusterName.cpp - getHeaderForProcessingStage.cpp - getTableExpressions.cpp HashJoin.cpp - IdentifierSemantic.cpp IExternalLoadable.cpp + IdentifierSemantic.cpp InJoinSubqueriesPreprocessor.cpp - inplaceBlockConversions.cpp InternalTextLogsQueue.cpp InterpreterAlterQuery.cpp InterpreterCheckQuery.cpp @@ -105,12 +94,9 @@ SRCS( InterpreterSystemQuery.cpp InterpreterUseQuery.cpp InterpreterWatchQuery.cpp - interpretSubquery.cpp - join_common.cpp - JoinedTables.cpp JoinSwitcher.cpp JoinToSubqueryTransformVisitor.cpp - loadMetadata.cpp + JoinedTables.cpp LogicalExpressionsOptimizer.cpp MarkTableIdentifiersVisitor.cpp MergeJoin.cpp @@ -140,7 +126,6 @@ SRCS( RowRefs.cpp Set.cpp SetVariants.cpp - sortBlock.cpp SortedBlocksWriter.cpp StorageID.cpp SubqueryForSet.cpp @@ -153,6 +138,21 @@ SRCS( TranslateQualifiedNamesVisitor.cpp TreeOptimizer.cpp TreeRewriter.cpp + addMissingDefaults.cpp + addTypeConversionToAST.cpp + castColumn.cpp + convertFieldToType.cpp + createBlockSelector.cpp + evaluateConstantExpression.cpp + executeQuery.cpp + getClusterName.cpp + getHeaderForProcessingStage.cpp + getTableExpressions.cpp + inplaceBlockConversions.cpp + interpretSubquery.cpp + join_common.cpp + loadMetadata.cpp + sortBlock.cpp ) diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 4f8b8a82210..3fd173d8e18 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -19,8 +19,8 @@ SRCS( ASTCreateRowPolicyQuery.cpp ASTCreateSettingsProfileQuery.cpp ASTCreateUserQuery.cpp - ASTDictionaryAttributeDeclaration.cpp ASTDictionary.cpp + ASTDictionaryAttributeDeclaration.cpp ASTDropAccessEntityQuery.cpp ASTDropQuery.cpp ASTExpressionList.cpp @@ -56,21 +56,17 @@ SRCS( ASTShowTablesQuery.cpp ASTSubquery.cpp ASTSystemQuery.cpp - ASTTablesInSelectQuery.cpp ASTTTLElement.cpp + ASTTablesInSelectQuery.cpp ASTUserNameWithHost.cpp ASTWithAlias.cpp ASTWithElement.cpp CommonParsers.cpp ExpressionElementParsers.cpp ExpressionListParsers.cpp - formatAST.cpp - formatSettingName.cpp IAST.cpp - iostream_debug_helpers.cpp IParserBase.cpp Lexer.cpp - makeASTForLogicalFunction.cpp MySQL/ASTAlterCommand.cpp MySQL/ASTAlterQuery.cpp MySQL/ASTCreateDefines.cpp @@ -84,11 +80,6 @@ SRCS( MySQL/ASTDeclareReference.cpp MySQL/ASTDeclareSubPartition.cpp MySQL/ASTDeclareTableOptions.cpp - obfuscateQueries.cpp - parseDatabaseAndTableName.cpp - parseIdentifierOrStringLiteral.cpp - parseIntervalKind.cpp - parseQuery.cpp ParserAlterQuery.cpp ParserCase.cpp ParserCheckQuery.cpp @@ -100,8 +91,8 @@ SRCS( ParserCreateUserQuery.cpp ParserDataType.cpp ParserDescribeTableQuery.cpp - ParserDictionaryAttributeDeclaration.cpp ParserDictionary.cpp + ParserDictionaryAttributeDeclaration.cpp ParserDropAccessEntityQuery.cpp ParserDropQuery.cpp ParserExplainQuery.cpp @@ -135,10 +126,19 @@ SRCS( ParserUserNameWithHost.cpp ParserWatchQuery.cpp ParserWithElement.cpp - parseUserName.cpp - queryToString.cpp QueryWithOutputSettingsPushDownVisitor.cpp TokenIterator.cpp + formatAST.cpp + formatSettingName.cpp + iostream_debug_helpers.cpp + makeASTForLogicalFunction.cpp + obfuscateQueries.cpp + parseDatabaseAndTableName.cpp + parseIdentifierOrStringLiteral.cpp + parseIntervalKind.cpp + parseQuery.cpp + parseUserName.cpp + queryToString.cpp ) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 7898576ad2d..9f022d26d3a 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -19,11 +19,14 @@ SRCS( Executors/PullingPipelineExecutor.cpp ForkProcessor.cpp Formats/IInputFormat.cpp + Formats/IOutputFormat.cpp + Formats/IRowInputFormat.cpp + Formats/IRowOutputFormat.cpp Formats/Impl/BinaryRowInputFormat.cpp Formats/Impl/BinaryRowOutputFormat.cpp - Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/CSVRowInputFormat.cpp Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/ConstantExpressionTemplate.cpp Formats/Impl/JSONAsStringRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowInputFormat.cpp Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -49,19 +52,16 @@ SRCS( Formats/Impl/RawBLOBRowInputFormat.cpp Formats/Impl/RawBLOBRowOutputFormat.cpp Formats/Impl/RegexpRowInputFormat.cpp + Formats/Impl/TSKVRowInputFormat.cpp + Formats/Impl/TSKVRowOutputFormat.cpp Formats/Impl/TabSeparatedRowInputFormat.cpp Formats/Impl/TabSeparatedRowOutputFormat.cpp Formats/Impl/TemplateBlockOutputFormat.cpp Formats/Impl/TemplateRowInputFormat.cpp - Formats/Impl/TSKVRowInputFormat.cpp - Formats/Impl/TSKVRowOutputFormat.cpp Formats/Impl/ValuesBlockInputFormat.cpp Formats/Impl/ValuesRowOutputFormat.cpp Formats/Impl/VerticalRowOutputFormat.cpp Formats/Impl/XMLRowOutputFormat.cpp - Formats/IOutputFormat.cpp - Formats/IRowInputFormat.cpp - Formats/IRowOutputFormat.cpp Formats/LazyOutputFormat.cpp Formats/OutputStreamToOutputFormat.cpp Formats/PullingOutputFormat.cpp @@ -87,7 +87,6 @@ SRCS( OffsetTransform.cpp Pipe.cpp Port.cpp - printPipeline.cpp QueryPipeline.cpp QueryPlan/AddingDelayedSourceStep.cpp QueryPlan/AggregatingStep.cpp @@ -151,6 +150,7 @@ SRCS( Transforms/RollupTransform.cpp Transforms/SortingTransform.cpp Transforms/TotalsHavingTransform.cpp + printPipeline.cpp ) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 784f88d82c7..6188bf13b3d 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -15,10 +15,8 @@ SRCS( ConstraintsDescription.cpp Distributed/DirectoryMonitor.cpp Distributed/DistributedBlockOutputStream.cpp - extractKeyExpressionList.cpp - getStructureOfRemoteTable.cpp - IndicesDescription.cpp IStorage.cpp + IndicesDescription.cpp JoinSettings.cpp KeyDescription.cpp LiveView/StorageLiveView.cpp @@ -27,19 +25,15 @@ SRCS( MergeTree/AllMergeSelector.cpp MergeTree/BackgroundProcessingPool.cpp MergeTree/BoolMask.cpp - MergeTree/checkDataPart.cpp MergeTree/DataPartsExchange.cpp MergeTree/EphemeralLockInZooKeeper.cpp - MergeTree/IMergedBlockOutputStream.cpp MergeTree/IMergeTreeDataPart.cpp MergeTree/IMergeTreeDataPartWriter.cpp MergeTree/IMergeTreeReader.cpp + MergeTree/IMergedBlockOutputStream.cpp MergeTree/KeyCondition.cpp MergeTree/LevelMergeSelector.cpp - MergeTree/localBackup.cpp MergeTree/MergeAlgorithm.cpp - MergeTree/MergedBlockOutputStream.cpp - MergeTree/MergedColumnOnlyOutputStream.cpp MergeTree/MergeList.cpp MergeTree/MergeTreeBaseSelectProcessor.cpp MergeTree/MergeTreeBlockOutputStream.cpp @@ -76,11 +70,11 @@ SRCS( MergeTree/MergeTreePartition.cpp MergeTree/MergeTreePartsMover.cpp MergeTree/MergeTreeRangeReader.cpp + MergeTree/MergeTreeReadPool.cpp MergeTree/MergeTreeReaderCompact.cpp MergeTree/MergeTreeReaderInMemory.cpp MergeTree/MergeTreeReaderStream.cpp MergeTree/MergeTreeReaderWide.cpp - MergeTree/MergeTreeReadPool.cpp MergeTree/MergeTreeReverseSelectProcessor.cpp MergeTree/MergeTreeSelectProcessor.cpp MergeTree/MergeTreeSequentialSource.cpp @@ -89,7 +83,8 @@ SRCS( MergeTree/MergeTreeWhereOptimizer.cpp MergeTree/MergeTreeWriteAheadLog.cpp MergeTree/MergeType.cpp - MergeTree/registerStorageMergeTree.cpp + MergeTree/MergedBlockOutputStream.cpp + MergeTree/MergedColumnOnlyOutputStream.cpp MergeTree/ReplicatedFetchList.cpp MergeTree/ReplicatedMergeTreeAddress.cpp MergeTree/ReplicatedMergeTreeAltersSequence.cpp @@ -104,10 +99,12 @@ SRCS( MergeTree/ReplicatedMergeTreeTableMetadata.cpp MergeTree/SimpleMergeSelector.cpp MergeTree/TTLMergeSelector.cpp + MergeTree/checkDataPart.cpp + MergeTree/localBackup.cpp + MergeTree/registerStorageMergeTree.cpp MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp - registerStorages.cpp SelectQueryDescription.cpp SetSettings.cpp StorageBuffer.cpp @@ -121,8 +118,8 @@ SRCS( StorageJoin.cpp StorageLog.cpp StorageLogSettings.cpp - StorageMaterializedView.cpp StorageMaterializeMySQL.cpp + StorageMaterializedView.cpp StorageMemory.cpp StorageMerge.cpp StorageMergeTree.cpp @@ -137,7 +134,6 @@ SRCS( StorageValues.cpp StorageView.cpp StorageXDBC.cpp - System/attachSystemTables.cpp System/StorageSystemAggregateFunctionCombinators.cpp System/StorageSystemAsynchronousMetrics.cpp System/StorageSystemBuildOptions.cpp @@ -147,8 +143,8 @@ SRCS( System/StorageSystemContributors.cpp System/StorageSystemContributors.generated.cpp System/StorageSystemCurrentRoles.cpp - System/StorageSystemDatabases.cpp System/StorageSystemDataTypeFamilies.cpp + System/StorageSystemDatabases.cpp System/StorageSystemDetachedParts.cpp System/StorageSystemDictionaries.cpp System/StorageSystemDisks.cpp @@ -161,22 +157,22 @@ SRCS( System/StorageSystemGrants.cpp System/StorageSystemGraphite.cpp System/StorageSystemMacros.cpp - System/StorageSystemMerges.cpp System/StorageSystemMergeTreeSettings.cpp + System/StorageSystemMerges.cpp System/StorageSystemMetrics.cpp System/StorageSystemModels.cpp System/StorageSystemMutations.cpp System/StorageSystemNumbers.cpp System/StorageSystemOne.cpp + System/StorageSystemParts.cpp System/StorageSystemPartsBase.cpp System/StorageSystemPartsColumns.cpp - System/StorageSystemParts.cpp System/StorageSystemPrivileges.cpp System/StorageSystemProcesses.cpp System/StorageSystemQuotaLimits.cpp + System/StorageSystemQuotaUsage.cpp System/StorageSystemQuotas.cpp System/StorageSystemQuotasUsage.cpp - System/StorageSystemQuotaUsage.cpp System/StorageSystemReplicas.cpp System/StorageSystemReplicatedFetches.cpp System/StorageSystemReplicationQueue.cpp @@ -195,9 +191,13 @@ SRCS( System/StorageSystemUsers.cpp System/StorageSystemZeros.cpp System/StorageSystemZooKeeper.cpp - transformQueryForExternalDatabase.cpp + System/attachSystemTables.cpp TTLDescription.cpp VirtualColumnUtils.cpp + extractKeyExpressionList.cpp + getStructureOfRemoteTable.cpp + registerStorages.cpp + transformQueryForExternalDatabase.cpp ) diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index 50f685fef3e..2bafb588fbb 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -10,8 +10,6 @@ SRCS( ITableFunction.cpp ITableFunctionFileLike.cpp ITableFunctionXDBC.cpp - parseColumnsListForTableFunction.cpp - registerTableFunctions.cpp TableFunctionFactory.cpp TableFunctionFile.cpp TableFunctionGenerateRandom.cpp @@ -25,6 +23,8 @@ SRCS( TableFunctionValues.cpp TableFunctionView.cpp TableFunctionZeros.cpp + parseColumnsListForTableFunction.cpp + registerTableFunctions.cpp ) diff --git a/utils/generate-ya-make/generate-ya-make.sh b/utils/generate-ya-make/generate-ya-make.sh index 84be2177386..d20f8d4fb8c 100755 --- a/utils/generate-ya-make/generate-ya-make.sh +++ b/utils/generate-ya-make/generate-ya-make.sh @@ -7,6 +7,9 @@ ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing' +# Otherwise 'sort' behaves differently on Mac OS and also depends on locale. +export LC_ALL=C + find "${ROOT_PATH}" -name 'ya.make.in' | while read path; do echo "# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it." > "${path/.in/}" (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" >> "${path/.in/}") From 159cd5ef72dd7e7473975adad22e43a87bc15cf1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 5 Nov 2020 15:32:03 +0300 Subject: [PATCH 103/149] add --database arg to CLICKHOUSE_BENCHMARK --- tests/queries/shell_config.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 9249fbc0411..0b5b0940cd7 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -7,6 +7,7 @@ export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL [ -v CLICKHOUSE_PORT_TCP ] && CLICKHOUSE_CLIENT_OPT0+=" --port=${CLICKHOUSE_PORT_TCP} " [ -v CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL ] && CLICKHOUSE_CLIENT_OPT0+=" --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL} " [ -v CLICKHOUSE_DATABASE ] && CLICKHOUSE_CLIENT_OPT0+=" --database=${CLICKHOUSE_DATABASE} " +[ -v CLICKHOUSE_DATABASE ] && CLICKHOUSE_BENCHMARK_OPT0+=" --database=${CLICKHOUSE_DATABASE} " export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} [ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client} @@ -17,7 +18,7 @@ export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY ${CLICK [ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY} local"} export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"} export CLICKHOUSE_OBFUSCATOR=${CLICKHOUSE_OBFUSCATOR:="${CLICKHOUSE_BINARY}-obfuscator"} -export CLICKHOUSE_BENCHMARK=${CLICKHOUSE_BENCHMARK:="${CLICKHOUSE_BINARY}-benchmark"} +export CLICKHOUSE_BENCHMARK=${CLICKHOUSE_BENCHMARK:="${CLICKHOUSE_BINARY}-benchmark ${CLICKHOUSE_BENCHMARK_OPT0:-}"} export CLICKHOUSE_CONFIG=${CLICKHOUSE_CONFIG:="/etc/clickhouse-server/config.xml"} export CLICKHOUSE_CONFIG_CLIENT=${CLICKHOUSE_CONFIG_CLIENT:="/etc/clickhouse-client/config.xml"} From 6f96cd04cca722b9cf7c04f01d1a30f85a23f54b Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 5 Nov 2020 15:53:40 +0300 Subject: [PATCH 104/149] Update aws submodule, use version with patch for non-empty http path in urls --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index 17e10c0fc77..a220591e335 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 17e10c0fc77f22afe890fa6d1b283760e5edaa56 +Subproject commit a220591e335923ce1c19bbf9eb925787f7ab6c13 From 9b8da816d223aad0b60c63d21bdd50466b446837 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 5 Nov 2020 11:05:56 -0400 Subject: [PATCH 105/149] Update mergetree.md tokenbf_v1 supports like --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index bb076d480cd..e4b6e0b1e59 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -332,8 +332,8 @@ INDEX b (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100) GRANULARIT |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| | [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗ | -| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✗ | ✗ | +| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | | [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | From 771814bb85781c10fed77b749ac805f5d8bc2fe6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 5 Nov 2020 11:07:24 -0400 Subject: [PATCH 106/149] Update mergetree.md bloom does not support like --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d48c05326c1..584bd31e276 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -343,8 +343,8 @@ The `set` index can be used with all functions. Function subsets for other index |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| | [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | | [notEquals(!=, \<\>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | | [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | | [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | | [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | From 9dd0c7708e7ab0e416bd31652e2ad411ec5a06b3 Mon Sep 17 00:00:00 2001 From: Alain BERRIER <24854656+aberrier@users.noreply.github.com> Date: Thu, 5 Nov 2020 16:23:23 +0100 Subject: [PATCH 107/149] fix: fix typo in array functions documentation --- docs/en/sql-reference/functions/array-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 50d1ae2aed4..abdf778c089 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -89,7 +89,7 @@ If the index falls outside of the bounds of an array, it returns some default va ## has(arr, elem) {#hasarr-elem} Checks whether the ‘arr’ array has the ‘elem’ element. -Returns 0 if the the element is not in the array, or 1 if it is. +Returns 0 if the element is not in the array, or 1 if it is. `NULL` is processed as a value. From aceb82ae6a4c812bf770167dbb0c0acaa4b51884 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 5 Nov 2020 19:00:05 +0300 Subject: [PATCH 108/149] better --- src/Interpreters/ExpressionActions.cpp | 3 - .../Transforms/TotalsHavingTransform.cpp | 11 ++++ .../01472_many_rows_in_totals.reference | 63 ------------------- .../0_stateless/01472_many_rows_in_totals.sql | 6 -- .../01548_with_totals_having.reference | 12 ---- .../0_stateless/01548_with_totals_having.sql | 4 +- 6 files changed, 13 insertions(+), 86 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index f5cd0ff409d..025c7d212ab 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -387,9 +387,6 @@ void ExpressionAction::execute(Block & block, bool dry_run) const source.type = assert_cast(*source.type).getNestedType(); source.name = result_name; - if (source.type->getTypeId() == TypeIndex::Nothing) - throw Exception("ARRAY JOIN of empty array is not allowed", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - block.insert(std::move(source)); break; diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 65043f65e1a..738b0062e6c 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int ILLEGAL_COLUMN; } void finalizeChunk(Chunk & chunk) @@ -156,6 +157,13 @@ void TotalsHavingTransform::transform(Chunk & chunk) /// Compute the expression in HAVING. const auto & cur_header = final ? finalized_header : getInputPort().getHeader(); auto finalized_block = cur_header.cloneWithColumns(finalized.detachColumns()); + + for (const ExpressionAction & action : expression->getActions()) + { + if (action.type == ExpressionAction::ARRAY_JOIN) + throw Exception("Having clause cannot contain arrayJoin", ErrorCodes::ILLEGAL_COLUMN); + } + expression->execute(finalized_block); auto columns = finalized_block.getColumns(); @@ -222,6 +230,9 @@ void TotalsHavingTransform::addToTotals(const Chunk & chunk, const IColumn::Filt const ColumnAggregateFunction::Container & vec = column->getData(); size_t size = vec.size(); + if (filter && filter->size() != size) + throw Exception("Filter has size which differs from column size", ErrorCodes::LOGICAL_ERROR); + if (filter) { for (size_t row = 0; row < size; ++row) diff --git a/tests/queries/0_stateless/01472_many_rows_in_totals.reference b/tests/queries/0_stateless/01472_many_rows_in_totals.reference index 6c91bf9f104..28f78731283 100644 --- a/tests/queries/0_stateless/01472_many_rows_in_totals.reference +++ b/tests/queries/0_stateless/01472_many_rows_in_totals.reference @@ -1,20 +1,3 @@ -┏━━━┳━━━━━━━━━━━━━┓ -┃ g ┃ sum(number) ┃ -┡━━━╇━━━━━━━━━━━━━┩ -│ 0 │ 2 │ -├───┼─────────────┤ -│ 0 │ 2 │ -├───┼─────────────┤ -│ 1 │ 4 │ -└───┴─────────────┘ - -Totals: -┏━━━┳━━━━━━━━━━━━━┓ -┃ g ┃ sum(number) ┃ -┡━━━╇━━━━━━━━━━━━━┩ -│ 0 │ 6 │ -└───┴─────────────┘ -- ┏━━━┳━━━┓ ┃ g ┃ s ┃ ┡━━━╇━━━┩ @@ -37,56 +20,10 @@ Totals: 0 2 0 2 1 4 - -0 6 -- -0 2 -0 2 -1 4 1 4 0 6 -- -{ - "meta": - [ - { - "name": "g", - "type": "UInt64" - }, - { - "name": "sum(number)", - "type": "UInt64" - } - ], - - "data": - [ - { - "g": "0", - "sum(number)": "2" - }, - { - "g": "0", - "sum(number)": "2" - }, - { - "g": "1", - "sum(number)": "4" - } - ], - - "totals": - { - "g": "0", - "sum(number)": "6" - }, - - "rows": 3, - - "rows_before_limit_at_least": 4 -} -- { "meta": [ diff --git a/tests/queries/0_stateless/01472_many_rows_in_totals.sql b/tests/queries/0_stateless/01472_many_rows_in_totals.sql index 0be4e2c5c37..4869e264d76 100644 --- a/tests/queries/0_stateless/01472_many_rows_in_totals.sql +++ b/tests/queries/0_stateless/01472_many_rows_in_totals.sql @@ -1,15 +1,9 @@ set output_format_write_statistics = 0; -select g, sum(number) from numbers(4) group by bitAnd(number, 1) as g with totals having sum(number) <= arrayJoin([2, 4]) format Pretty; -select '-'; select g, s from (select g, sum(number) as s from numbers(4) group by bitAnd(number, 1) as g with totals) array join [1, 2] as a format Pretty; select '--'; -select g, sum(number) from numbers(4) group by bitAnd(number, 1) as g with totals having sum(number) <= arrayJoin([2, 4]) format TSV; -select '-'; select g, s from (select g, sum(number) as s from numbers(4) group by bitAnd(number, 1) as g with totals) array join [1, 2] as a format TSV; select '--'; -select g, sum(number) from numbers(4) group by bitAnd(number, 1) as g with totals having sum(number) <= arrayJoin([2, 4]) format JSON; -select '-'; select g, s from (select g, sum(number) as s from numbers(4) group by bitAnd(number, 1) as g with totals) array join [1, 2] as a format JSON; select '--'; diff --git a/tests/queries/0_stateless/01548_with_totals_having.reference b/tests/queries/0_stateless/01548_with_totals_having.reference index c728434f5dd..e69de29bb2d 100644 --- a/tests/queries/0_stateless/01548_with_totals_having.reference +++ b/tests/queries/0_stateless/01548_with_totals_having.reference @@ -1,12 +0,0 @@ -0 -0 -0 -0 -1 -1 -1 -2 -2 -3 - -0 diff --git a/tests/queries/0_stateless/01548_with_totals_having.sql b/tests/queries/0_stateless/01548_with_totals_having.sql index 167d879bbeb..669d989aa5f 100644 --- a/tests/queries/0_stateless/01548_with_totals_having.sql +++ b/tests/queries/0_stateless/01548_with_totals_having.sql @@ -1,2 +1,2 @@ -SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 43 } -SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([3, 2, 1, 0]) ORDER BY number; +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([]); -- { serverError 44 } +SELECT * FROM numbers(4) GROUP BY number WITH TOTALS HAVING sum(number) <= arrayJoin([3, 2, 1, 0]) ORDER BY number; -- { serverError 44 } From 2593b37c0067117a722a3df0a05ba0e26d1fadda Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 5 Nov 2020 19:09:45 +0300 Subject: [PATCH 109/149] style --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 025c7d212ab..31bc51ec6fa 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -50,7 +50,6 @@ namespace ErrorCodes extern const int TOO_MANY_TEMPORARY_COLUMNS; extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS; extern const int TYPE_MISMATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; } /// Read comment near usage @@ -377,6 +376,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const block.erase(source_name); source.column = source.column->convertToFullColumnIfConst(); const ColumnArray * array = typeid_cast(source.column.get()); + if (!array) throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); From 488909208b748120e6aa4dddfff641d4e2eb3195 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 5 Nov 2020 19:10:44 +0300 Subject: [PATCH 110/149] style --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 31bc51ec6fa..18d21b84b42 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -375,8 +375,8 @@ void ExpressionAction::execute(Block & block, bool dry_run) const auto source = block.getByName(source_name); block.erase(source_name); source.column = source.column->convertToFullColumnIfConst(); + const ColumnArray * array = typeid_cast(source.column.get()); - if (!array) throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); From 83af1f9bbbfcc6ffb94353592bec6963453c528e Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 5 Nov 2020 19:11:32 +0300 Subject: [PATCH 111/149] style --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 18d21b84b42..762ad6ae575 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -375,7 +375,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const auto source = block.getByName(source_name); block.erase(source_name); source.column = source.column->convertToFullColumnIfConst(); - + const ColumnArray * array = typeid_cast(source.column.get()); if (!array) throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); From 90a4e0a56b61ce7e0e55b00fb1613bde04340edb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 19:14:17 +0300 Subject: [PATCH 112/149] Fix "format-changelog.py" for Python3 --- utils/simple-backport/format-changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index e0fe4912d5d..91547befed4 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -18,7 +18,7 @@ args = parser.parse_args() def parse_one_pull_request(item): description = item['body'] # Don't skip empty lines because they delimit parts of description - lines = [line for line in [x.strip() for x in description.split('\n') if description else []]] + lines = [line for line in [x.strip() for x in (description.split('\n') if description else [])]] lines = [re.sub(r'\s+', ' ', l) for l in lines] category = '' From 02a071e88fd8ade5b17ed06aeb34f83936a8ceb2 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Thu, 5 Nov 2020 17:51:17 +0100 Subject: [PATCH 113/149] RBAC Testflows - Instrumenting server log for debug and new ALTER tests --- docker/test/testflows/runner/Dockerfile | 2 +- tests/testflows/rbac/helper/common.py | 29 + .../rbac/requirements/requirements.md | 467 ++++--- .../rbac/requirements/requirements.py | 1154 ++++++++--------- tests/testflows/rbac/tests/errors.py | 90 -- .../tests/privileges/alter/alter_column.py | 320 +---- .../privileges/alter/alter_constraint.py | 328 +---- .../tests/privileges/alter/alter_delete.py | 95 ++ .../tests/privileges/alter/alter_fetch.py | 109 ++ .../tests/privileges/alter/alter_freeze.py | 95 ++ .../tests/privileges/alter/alter_index.py | 336 +---- .../rbac/tests/privileges/alter/alter_move.py | 157 +++ .../tests/privileges/alter/alter_settings.py | 293 +---- .../rbac/tests/privileges/alter/alter_ttl.py | 328 +---- .../tests/privileges/alter/alter_update.py | 96 ++ .../tests/privileges/distributed_table.py | 12 +- .../rbac/tests/privileges/grant_option.py | 139 ++ .../testflows/rbac/tests/privileges/insert.py | 227 +--- .../rbac/tests/privileges/public_tables.py | 2 +- .../testflows/rbac/tests/privileges/select.py | 214 +-- .../rbac/tests/privileges/show_tables.py | 2 +- tests/testflows/rbac/tests/views/live_view.py | 44 +- .../rbac/tests/views/materialized_view.py | 82 +- tests/testflows/rbac/tests/views/view.py | 44 +- 24 files changed, 1627 insertions(+), 3038 deletions(-) delete mode 100755 tests/testflows/rbac/tests/errors.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_delete.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_fetch.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_freeze.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_move.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_update.py create mode 100644 tests/testflows/rbac/tests/privileges/grant_option.py diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index 06241d6d497..e99af08afa5 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -35,7 +35,7 @@ RUN apt-get update \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -RUN pip3 install urllib3 testflows==1.6.59 docker-compose docker dicttoxml kazoo tzlocal +RUN pip3 install urllib3 testflows==1.6.62 docker-compose docker dicttoxml kazoo tzlocal ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce diff --git a/tests/testflows/rbac/helper/common.py b/tests/testflows/rbac/helper/common.py index 005d0cb373c..19bc396a0b8 100755 --- a/tests/testflows/rbac/helper/common.py +++ b/tests/testflows/rbac/helper/common.py @@ -9,6 +9,35 @@ from testflows.core import * from rbac.helper.tables import table_types +@TestStep(Given) +def instrument_clickhouse_server_log(self, node=None, clickhouse_server_log="/var/log/clickhouse-server/clickhouse-server.log"): + """Instrument clickhouse-server.log for the current test + by adding start and end messages that include + current test name to the clickhouse-server.log of the specified node and + if the test fails then dump the messages from + the clickhouse-server.log for this test. + """ + if node is None: + node = self.context.node + + with By("getting current log size"): + cmd = node.command(f"stat --format=%s {clickhouse_server_log}") + logsize = cmd.output.split(" ")[0].strip() + + try: + with And("adding test name start message to the clickhouse-server.log"): + node.command(f"echo -e \"\\n-- start: {current().name} --\\n\" >> {clickhouse_server_log}") + yield + + finally: + with Finally("adding test name end message to the clickhouse-server.log", flags=TE): + node.command(f"echo -e \"\\n-- end: {current().name} --\\n\" >> {clickhouse_server_log}") + + with And("checking if test has failing result"): + if not self.parent.result: + with Then("dumping clickhouse-server.log for this test"): + node.command(f"tail -c +{logsize} {clickhouse_server_log}") + def join(tasks): """Join all parallel tests. """ diff --git a/tests/testflows/rbac/requirements/requirements.md b/tests/testflows/rbac/requirements/requirements.md index 10b2d277523..4f2be6776c0 100644 --- a/tests/testflows/rbac/requirements/requirements.md +++ b/tests/testflows/rbac/requirements/requirements.md @@ -378,74 +378,74 @@ * 5.2.11.2.3 [RQ.SRS-006.RBAC.Privileges.Select.Revoke](#rqsrs-006rbacprivilegesselectrevoke) * 5.2.11.2.4 [RQ.SRS-006.RBAC.Privileges.Select.Column](#rqsrs-006rbacprivilegesselectcolumn) * 5.2.11.2.5 [RQ.SRS-006.RBAC.Privileges.Select.Cluster](#rqsrs-006rbacprivilegesselectcluster) - * 5.2.11.2.6 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption](#rqsrs-006rbacprivilegesselectgrantoption) - * 5.2.11.2.7 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant](#rqsrs-006rbacprivilegesselectgrantoptiongrant) - * 5.2.11.2.8 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke](#rqsrs-006rbacprivilegesselectgrantoptionrevoke) - * 5.2.11.2.9 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) + * 5.2.11.2.6 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) * 5.2.11.3 [Insert](#insert) * 5.2.11.3.1 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) * 5.2.11.3.2 [RQ.SRS-006.RBAC.Privileges.Insert.Grant](#rqsrs-006rbacprivilegesinsertgrant) * 5.2.11.3.3 [RQ.SRS-006.RBAC.Privileges.Insert.Revoke](#rqsrs-006rbacprivilegesinsertrevoke) * 5.2.11.3.4 [RQ.SRS-006.RBAC.Privileges.Insert.Column](#rqsrs-006rbacprivilegesinsertcolumn) * 5.2.11.3.5 [RQ.SRS-006.RBAC.Privileges.Insert.Cluster](#rqsrs-006rbacprivilegesinsertcluster) - * 5.2.11.3.6 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption](#rqsrs-006rbacprivilegesinsertgrantoption) - * 5.2.11.3.7 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant](#rqsrs-006rbacprivilegesinsertgrantoptiongrant) - * 5.2.11.3.8 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke](#rqsrs-006rbacprivilegesinsertgrantoptionrevoke) - * 5.2.11.3.9 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) + * 5.2.11.3.6 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) * 5.2.11.4 [AlterColumn](#altercolumn) * 5.2.11.4.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) * 5.2.11.4.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) * 5.2.11.4.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) * 5.2.11.4.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) * 5.2.11.4.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) - * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption](#rqsrs-006rbacprivilegesaltercolumngrantoption) - * 5.2.11.4.7 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Grant](#rqsrs-006rbacprivilegesaltercolumngrantoptiongrant) - * 5.2.11.4.8 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Revoke](#rqsrs-006rbacprivilegesaltercolumngrantoptionrevoke) - * 5.2.11.4.9 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) + * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) * 5.2.11.5 [AlterIndex](#alterindex) * 5.2.11.5.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) * 5.2.11.5.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) * 5.2.11.5.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) * 5.2.11.5.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) - * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption](#rqsrs-006rbacprivilegesalterindexgrantoption) - * 5.2.11.5.6 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Grant](#rqsrs-006rbacprivilegesalterindexgrantoptiongrant) - * 5.2.11.5.7 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterindexgrantoptionrevoke) - * 5.2.11.5.8 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) + * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) * 5.2.11.6 [AlterConstraint](#alterconstraint) * 5.2.11.6.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) * 5.2.11.6.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) * 5.2.11.6.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) * 5.2.11.6.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) - * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption](#rqsrs-006rbacprivilegesalterconstraintgrantoption) - * 5.2.11.6.6 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Grant](#rqsrs-006rbacprivilegesalterconstraintgrantoptiongrant) - * 5.2.11.6.7 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterconstraintgrantoptionrevoke) - * 5.2.11.6.8 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) + * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) * 5.2.11.7 [AlterTTL](#alterttl) * 5.2.11.7.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) * 5.2.11.7.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) * 5.2.11.7.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) * 5.2.11.7.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) - * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption](#rqsrs-006rbacprivilegesalterttlgrantoption) - * 5.2.11.7.6 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Grant](#rqsrs-006rbacprivilegesalterttlgrantoptiongrant) - * 5.2.11.7.7 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterttlgrantoptionrevoke) - * 5.2.11.7.8 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) + * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) * 5.2.11.8 [AlterSettings](#altersettings) * 5.2.11.8.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) * 5.2.11.8.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) * 5.2.11.8.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) * 5.2.11.8.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) - * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption](#rqsrs-006rbacprivilegesaltersettingsgrantoption) - * 5.2.11.8.6 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Grant](#rqsrs-006rbacprivilegesaltersettingsgrantoptiongrant) - * 5.2.11.8.7 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Revoke](#rqsrs-006rbacprivilegesaltersettingsgrantoptionrevoke) - * 5.2.11.8.8 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) - * 5.2.11.9 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) - * 5.2.11.10 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) - * 5.2.11.11 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) - * 5.2.11.12 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) - * 5.2.11.13 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.14 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) + * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) + * 5.2.11.9 [Alter Update](#alter-update) + * 5.2.11.9.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) + * 5.2.11.9.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Access](#rqsrs-006rbacprivilegesalterupdateaccess) + * 5.2.11.9.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) + * 5.2.11.10 [Alter Delete](#alter-delete) + * 5.2.11.10.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) + * 5.2.11.10.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Access](#rqsrs-006rbacprivilegesalterdeleteaccess) + * 5.2.11.10.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) + * 5.2.11.11 [Alter Freeze Partition](#alter-freeze-partition) + * 5.2.11.11.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) + * 5.2.11.11.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Access](#rqsrs-006rbacprivilegesalterfreezeaccess) + * 5.2.11.11.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) + * 5.2.11.12 [Alter Fetch Partition](#alter-fetch-partition) + * 5.2.11.12.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) + * 5.2.11.12.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Access](#rqsrs-006rbacprivilegesalterfetchaccess) + * 5.2.11.12.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) + * 5.2.11.13 [Alter Move Partition](#alter-move-partition) + * 5.2.11.13.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) + * 5.2.11.13.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Access](#rqsrs-006rbacprivilegesaltermoveaccess) + * 5.2.11.13.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) + * 5.2.11.14 [Grant Option](#grant-option) + * 5.2.11.14.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) + * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) + * 5.2.11.17 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) + * 5.2.11.18 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) + * 5.2.11.19 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.11.20 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) + * 5.2.11.21 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 5.2.12 [Required Privileges](#required-privileges) * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) @@ -3042,33 +3042,6 @@ on a specified cluster to one or more **users** or **roles**. Any `SELECT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **select** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **select** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **select** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. Any `SELECT INTO` statements SHALL succeed -when done by a user with privilege granted by a user with `GRANT OPTION`, -either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **select** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `SELECT INTO` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.Select.TableEngines version: 1.0 @@ -3130,33 +3103,6 @@ on a specified cluster to one or more **users** or **roles**. Any `INSERT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **insert** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **insert** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **insert** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. Any `INSERT INTO` statements SHALL succeed -when done by a user with privilege granted by a user with `GRANT OPTION`, -either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **insert** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `INSERT INTO` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.Insert.TableEngines version: 1.0 @@ -3219,34 +3165,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter column** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter column** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter column** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter column** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines version: 1.0 @@ -3300,35 +3218,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter index** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter index** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter index** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter index** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines version: 1.0 @@ -3382,35 +3271,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter constraint** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter constraint** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter constraint** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter constraint** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines version: 1.0 @@ -3464,35 +3324,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter ttl** or **alter materialize ttl** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter ttl** or **alter materialize ttl** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter ttl** or **alter materialize ttl** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter ttl** or **alter materialize ttl** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines version: 1.0 @@ -3534,35 +3365,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter settings** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter settings** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter settings** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter settings** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... MODIFY SETTING setting` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines version: 1.0 @@ -3584,6 +3386,197 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree +##### Alter Update + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER UPDATE` statement if and only if the user has **alter update** privilege for that column, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter update** privilege on a column level +to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter update** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Delete + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER DELETE` statement if and only if the user has **alter delete** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter delete** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter delete** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Freeze Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER FREEZE` statement if and only if the user has **alter freeze** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter freeze** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter freeze** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Fetch Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER FETCH` statement if and only if the user has **alter fetch** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter fetch** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter fetch** privilege +on tables created using the following engines + +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Move Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterMove +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER MOVE` statement if and only if the user has **alter move**, **select**, and **alter delete** privilege on the source table +and **insert** privilege on the target table, either directly or through a role. +For example, +```sql +ALTER TABLE source_table MOVE PARTITION 1 TO target_table +``` + +###### RQ.SRS-006.RBAC.Privileges.AlterMove.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter move** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter move** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Grant Option + +###### RQ.SRS-006.RBAC.Privileges.GrantOption +version: 1.0 + +[ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if +the user has that privilege with `GRANT OPTION`, either directly or through a role. + +`GRANT OPTION` is supported by the following privileges + +* `ALTER MOVE PARTITION` +* `ALTER FETCH PARTITION` +* `ALTER FREEZE PARTITION` +* `ALTER DELETE` +* `ALTER UPDATE` +* `ALTER SETTINGS` +* `ALTER TTL` +* `ALTER CONSTRAINT` +* `ALTER COLUMN` +* `ALTER INDEX` +* `INSERT` +* `SELECT` + ##### RQ.SRS-006.RBAC.Privileges.Delete version: 1.0 @@ -3621,12 +3614,6 @@ version: 1.0 [ClickHouse] SHALL support granting or revoking **all** privileges for a database or a specific table to one or more **users** or **roles**. -##### RQ.SRS-006.RBAC.Privileges.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **grant option** privilege -for a database or a specific table to one or more **users** or **roles**. - ##### RQ.SRS-006.RBAC.Privileges.AdminOption version: 1.0 diff --git a/tests/testflows/rbac/requirements/requirements.py b/tests/testflows/rbac/requirements/requirements.py index bce0b3a9fcb..5277b732066 100755 --- a/tests/testflows/rbac/requirements/requirements.py +++ b/tests/testflows/rbac/requirements/requirements.py @@ -403,74 +403,74 @@ SRS_006_ClickHouse_Role_Based_Access_Control = Specification( * 5.2.11.2.3 [RQ.SRS-006.RBAC.Privileges.Select.Revoke](#rqsrs-006rbacprivilegesselectrevoke) * 5.2.11.2.4 [RQ.SRS-006.RBAC.Privileges.Select.Column](#rqsrs-006rbacprivilegesselectcolumn) * 5.2.11.2.5 [RQ.SRS-006.RBAC.Privileges.Select.Cluster](#rqsrs-006rbacprivilegesselectcluster) - * 5.2.11.2.6 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption](#rqsrs-006rbacprivilegesselectgrantoption) - * 5.2.11.2.7 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant](#rqsrs-006rbacprivilegesselectgrantoptiongrant) - * 5.2.11.2.8 [RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke](#rqsrs-006rbacprivilegesselectgrantoptionrevoke) - * 5.2.11.2.9 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) + * 5.2.11.2.6 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) * 5.2.11.3 [Insert](#insert) * 5.2.11.3.1 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) * 5.2.11.3.2 [RQ.SRS-006.RBAC.Privileges.Insert.Grant](#rqsrs-006rbacprivilegesinsertgrant) * 5.2.11.3.3 [RQ.SRS-006.RBAC.Privileges.Insert.Revoke](#rqsrs-006rbacprivilegesinsertrevoke) * 5.2.11.3.4 [RQ.SRS-006.RBAC.Privileges.Insert.Column](#rqsrs-006rbacprivilegesinsertcolumn) * 5.2.11.3.5 [RQ.SRS-006.RBAC.Privileges.Insert.Cluster](#rqsrs-006rbacprivilegesinsertcluster) - * 5.2.11.3.6 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption](#rqsrs-006rbacprivilegesinsertgrantoption) - * 5.2.11.3.7 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant](#rqsrs-006rbacprivilegesinsertgrantoptiongrant) - * 5.2.11.3.8 [RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke](#rqsrs-006rbacprivilegesinsertgrantoptionrevoke) - * 5.2.11.3.9 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) + * 5.2.11.3.6 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) * 5.2.11.4 [AlterColumn](#altercolumn) * 5.2.11.4.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) * 5.2.11.4.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) * 5.2.11.4.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) * 5.2.11.4.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) * 5.2.11.4.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) - * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption](#rqsrs-006rbacprivilegesaltercolumngrantoption) - * 5.2.11.4.7 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Grant](#rqsrs-006rbacprivilegesaltercolumngrantoptiongrant) - * 5.2.11.4.8 [RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Revoke](#rqsrs-006rbacprivilegesaltercolumngrantoptionrevoke) - * 5.2.11.4.9 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) + * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) * 5.2.11.5 [AlterIndex](#alterindex) * 5.2.11.5.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) * 5.2.11.5.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) * 5.2.11.5.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) * 5.2.11.5.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) - * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption](#rqsrs-006rbacprivilegesalterindexgrantoption) - * 5.2.11.5.6 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Grant](#rqsrs-006rbacprivilegesalterindexgrantoptiongrant) - * 5.2.11.5.7 [RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterindexgrantoptionrevoke) - * 5.2.11.5.8 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) + * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) * 5.2.11.6 [AlterConstraint](#alterconstraint) * 5.2.11.6.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) * 5.2.11.6.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) * 5.2.11.6.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) * 5.2.11.6.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) - * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption](#rqsrs-006rbacprivilegesalterconstraintgrantoption) - * 5.2.11.6.6 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Grant](#rqsrs-006rbacprivilegesalterconstraintgrantoptiongrant) - * 5.2.11.6.7 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterconstraintgrantoptionrevoke) - * 5.2.11.6.8 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) + * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) * 5.2.11.7 [AlterTTL](#alterttl) * 5.2.11.7.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) * 5.2.11.7.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) * 5.2.11.7.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) * 5.2.11.7.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) - * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption](#rqsrs-006rbacprivilegesalterttlgrantoption) - * 5.2.11.7.6 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Grant](#rqsrs-006rbacprivilegesalterttlgrantoptiongrant) - * 5.2.11.7.7 [RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Revoke](#rqsrs-006rbacprivilegesalterttlgrantoptionrevoke) - * 5.2.11.7.8 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) + * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) * 5.2.11.8 [AlterSettings](#altersettings) * 5.2.11.8.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) * 5.2.11.8.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) * 5.2.11.8.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) * 5.2.11.8.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) - * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption](#rqsrs-006rbacprivilegesaltersettingsgrantoption) - * 5.2.11.8.6 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Grant](#rqsrs-006rbacprivilegesaltersettingsgrantoptiongrant) - * 5.2.11.8.7 [RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Revoke](#rqsrs-006rbacprivilegesaltersettingsgrantoptionrevoke) - * 5.2.11.8.8 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) - * 5.2.11.9 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) - * 5.2.11.10 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) - * 5.2.11.11 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) - * 5.2.11.12 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) - * 5.2.11.13 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.14 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) + * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) + * 5.2.11.9 [Alter Update](#alter-update) + * 5.2.11.9.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) + * 5.2.11.9.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Access](#rqsrs-006rbacprivilegesalterupdateaccess) + * 5.2.11.9.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) + * 5.2.11.10 [Alter Delete](#alter-delete) + * 5.2.11.10.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) + * 5.2.11.10.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Access](#rqsrs-006rbacprivilegesalterdeleteaccess) + * 5.2.11.10.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) + * 5.2.11.11 [Alter Freeze Partition](#alter-freeze-partition) + * 5.2.11.11.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) + * 5.2.11.11.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Access](#rqsrs-006rbacprivilegesalterfreezeaccess) + * 5.2.11.11.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) + * 5.2.11.12 [Alter Fetch Partition](#alter-fetch-partition) + * 5.2.11.12.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) + * 5.2.11.12.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Access](#rqsrs-006rbacprivilegesalterfetchaccess) + * 5.2.11.12.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) + * 5.2.11.13 [Alter Move Partition](#alter-move-partition) + * 5.2.11.13.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) + * 5.2.11.13.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Access](#rqsrs-006rbacprivilegesaltermoveaccess) + * 5.2.11.13.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) + * 5.2.11.14 [Grant Option](#grant-option) + * 5.2.11.14.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.Delete](#rqsrs-006rbacprivilegesdelete) + * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.Alter](#rqsrs-006rbacprivilegesalter) + * 5.2.11.17 [RQ.SRS-006.RBAC.Privileges.Create](#rqsrs-006rbacprivilegescreate) + * 5.2.11.18 [RQ.SRS-006.RBAC.Privileges.Drop](#rqsrs-006rbacprivilegesdrop) + * 5.2.11.19 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.11.20 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) + * 5.2.11.21 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 5.2.12 [Required Privileges](#required-privileges) * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) @@ -3067,33 +3067,6 @@ on a specified cluster to one or more **users** or **roles**. Any `SELECT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **select** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **select** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **select** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. Any `SELECT INTO` statements SHALL succeed -when done by a user with privilege granted by a user with `GRANT OPTION`, -either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **select** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `SELECT INTO` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.Select.TableEngines version: 1.0 @@ -3155,33 +3128,6 @@ on a specified cluster to one or more **users** or **roles**. Any `INSERT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **insert** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **insert** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **insert** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. Any `INSERT INTO` statements SHALL succeed -when done by a user with privilege granted by a user with `GRANT OPTION`, -either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **insert** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `INSERT INTO` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.Insert.TableEngines version: 1.0 @@ -3244,34 +3190,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter column** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter column** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter column** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter column** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL fail -when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines version: 1.0 @@ -3325,35 +3243,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter index** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter index** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter index** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter index** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines version: 1.0 @@ -3407,35 +3296,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter constraint** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter constraint** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter constraint** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter constraint** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines version: 1.0 @@ -3489,35 +3349,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter ttl** or **alter materialize ttl** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter ttl** or **alter materialize ttl** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter ttl** or **alter materialize ttl** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter ttl** or **alter materialize ttl** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines version: 1.0 @@ -3559,35 +3390,6 @@ on a specified cluster to one or more **users** or **roles**. Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting **alter settings** privilege -for a database or a specific table to one or more **users** or **roles** -with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to -change access to the **alter settings** privilege by another user or role -on the same or smaller scope that they have access to. - -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Grant -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -granting **alter settings** privilege to other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL -succeed when done by a user with privilege granted by a user with -`GRANT OPTION`, either directly or through an assigned role. - -###### RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Revoke -version: 1.0 - -[ClickHouse] SHALL support a user with **grant option** privilege -revoking **alter settings** privilege from other **users** or **roles** on the same -or smaller scope that they have access to. -Any `ALTER TABLE ... MODIFY SETTING setting` statements -SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`, -either directly or through an assigned role, unless they have access otherwise. - ###### RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines version: 1.0 @@ -3609,6 +3411,197 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree +##### Alter Update + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER UPDATE` statement if and only if the user has **alter update** privilege for that column, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter update** privilege on a column level +to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter update** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Delete + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER DELETE` statement if and only if the user has **alter delete** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter delete** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter delete** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Freeze Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER FREEZE` statement if and only if the user has **alter freeze** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter freeze** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter freeze** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Fetch Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER FETCH` statement if and only if the user has **alter fetch** privilege for that table, +either directly or through a role. + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter fetch** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter fetch** privilege +on tables created using the following engines + +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Alter Move Partition + +###### RQ.SRS-006.RBAC.Privileges.AlterMove +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER MOVE` statement if and only if the user has **alter move**, **select**, and **alter delete** privilege on the source table +and **insert** privilege on the target table, either directly or through a role. +For example, +```sql +ALTER TABLE source_table MOVE PARTITION 1 TO target_table +``` + +###### RQ.SRS-006.RBAC.Privileges.AlterMove.Access +version: 1.0 + +[ClickHouse] SHALL support granting or revoking **alter move** privilege to one or more **users** or **roles**. + +###### RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines +version: 1.0 + +[ClickHouse] SHALL support controlling access to the **alter move** privilege +on tables created using the following engines + +* MergeTree +* ReplacingMergeTree +* SummingMergeTree +* AggregatingMergeTree +* CollapsingMergeTree +* VersionedCollapsingMergeTree +* GraphiteMergeTree +* ReplicatedMergeTree +* ReplicatedSummingMergeTree +* ReplicatedReplacingMergeTree +* ReplicatedAggregatingMergeTree +* ReplicatedCollapsingMergeTree +* ReplicatedVersionedCollapsingMergeTree +* ReplicatedGraphiteMergeTree + +##### Grant Option + +###### RQ.SRS-006.RBAC.Privileges.GrantOption +version: 1.0 + +[ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if +the user has that privilege with `GRANT OPTION`, either directly or through a role. + +`GRANT OPTION` is supported by the following privileges + +* `ALTER MOVE PARTITION` +* `ALTER FETCH PARTITION` +* `ALTER FREEZE PARTITION` +* `ALTER DELETE` +* `ALTER UPDATE` +* `ALTER SETTINGS` +* `ALTER TTL` +* `ALTER CONSTRAINT` +* `ALTER COLUMN` +* `ALTER INDEX` +* `INSERT` +* `SELECT` + ##### RQ.SRS-006.RBAC.Privileges.Delete version: 1.0 @@ -3646,12 +3639,6 @@ version: 1.0 [ClickHouse] SHALL support granting or revoking **all** privileges for a database or a specific table to one or more **users** or **roles**. -##### RQ.SRS-006.RBAC.Privileges.GrantOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **grant option** privilege -for a database or a specific table to one or more **users** or **roles**. - ##### RQ.SRS-006.RBAC.Privileges.AdminOption version: 1.0 @@ -9055,57 +9042,6 @@ RQ_SRS_006_RBAC_Privileges_Select_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_Select_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **select** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **select** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **select** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to. Any `SELECT INTO` statements SHALL succeed\n' - 'when done by a user with privilege granted by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **select** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to. Any `SELECT INTO` statements SHALL fail\n' - 'when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_Select_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.Select.TableEngines', version='1.0', @@ -9213,57 +9149,6 @@ RQ_SRS_006_RBAC_Privileges_Insert_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_Insert_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **insert** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **insert** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **insert** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to. Any `INSERT INTO` statements SHALL succeed\n' - 'when done by a user with privilege granted by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **insert** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to. Any `INSERT INTO` statements SHALL fail\n' - 'when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_Insert_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.Insert.TableEngines', version='1.0', @@ -9372,58 +9257,6 @@ RQ_SRS_006_RBAC_Privileges_AlterColumn_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter column** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **alter column** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **alter column** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL\n' - 'succeed when done by a user with privilege granted by a user with\n' - '`GRANT OPTION`, either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **alter column** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to. Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL fail\n' - 'when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines', version='1.0', @@ -9515,59 +9348,6 @@ RQ_SRS_006_RBAC_Privileges_AlterIndex_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter index** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **alter index** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **alter index** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL\n' - 'succeed when done by a user with privilege granted by a user with\n' - '`GRANT OPTION`, either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **alter index** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements\n' - 'SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AlterIndex_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines', version='1.0', @@ -9659,59 +9439,6 @@ RQ_SRS_006_RBAC_Privileges_AlterConstraint_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter constraint** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **alter constraint** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **alter constraint** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements SHALL\n' - 'succeed when done by a user with privilege granted by a user with\n' - '`GRANT OPTION`, either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **alter constraint** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ADD|DROP CONSTRAINT` statements\n' - 'SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines', version='1.0', @@ -9803,59 +9530,6 @@ RQ_SRS_006_RBAC_Privileges_AlterTTL_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter ttl** or **alter materialize ttl** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **alter ttl** or **alter materialize ttl** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **alter ttl** or **alter materialize ttl** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL\n' - 'succeed when done by a user with privilege granted by a user with\n' - '`GRANT OPTION`, either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **alter ttl** or **alter materialize ttl** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements\n' - 'SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AlterTTL_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines', version='1.0', @@ -9935,59 +9609,6 @@ RQ_SRS_006_RBAC_Privileges_AlterSettings_Cluster = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter settings** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - 'with a `GRANT OPTION` clause. User with **grant option** privilege SHALL be able to\n' - 'change access to the **alter settings** privilege by another user or role\n' - 'on the same or smaller scope that they have access to.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'granting **alter settings** privilege to other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL\n' - 'succeed when done by a user with privilege granted by a user with\n' - '`GRANT OPTION`, either directly or through an assigned role.\n' - '\n' - ), - link=None) - -RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.GrantOption.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support a user with **grant option** privilege\n' - 'revoking **alter settings** privilege from other **users** or **roles** on the same\n' - 'or smaller scope that they have access to.\n' - 'Any `ALTER TABLE ... MODIFY SETTING setting` statements\n' - 'SHALL fail when done by a user with privilege revoke by a user with `GRANT OPTION`,\n' - 'either directly or through an assigned role, unless they have access otherwise.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines = Requirement( name='RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines', version='1.0', @@ -10017,6 +9638,313 @@ RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines = Requirement( ), link=None) +RQ_SRS_006_RBAC_Privileges_AlterUpdate = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER UPDATE` statement if and only if the user has **alter update** privilege for that column,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate_Access = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter update** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter update** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER DELETE` statement if and only if the user has **alter delete** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete_Access = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter delete** privilege to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter delete** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER FREEZE` statement if and only if the user has **alter freeze** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze_Access = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter freeze** privilege to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter freeze** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER FETCH` statement if and only if the user has **alter fetch** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch_Access = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter fetch** privilege to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter fetch** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER MOVE` statement if and only if the user has **alter move**, **select**, and **alter delete** privilege on the source table\n' + 'and **insert** privilege on the target table, either directly or through a role.\n' + 'For example,\n' + '```sql\n' + 'ALTER TABLE source_table MOVE PARTITION 1 TO target_table\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove_Access = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter move** privilege to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter move** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if\n' + 'the user has that privilege with `GRANT OPTION`, either directly or through a role.\n' + '\n' + '`GRANT OPTION` is supported by the following privileges\n' + '\n' + '* `ALTER MOVE PARTITION`\n' + '* `ALTER FETCH PARTITION`\n' + '* `ALTER FREEZE PARTITION`\n' + '* `ALTER DELETE`\n' + '* `ALTER UPDATE`\n' + '* `ALTER SETTINGS`\n' + '* `ALTER TTL`\n' + '* `ALTER CONSTRAINT`\n' + '* `ALTER COLUMN`\n' + '* `ALTER INDEX`\n' + '* `INSERT`\n' + '* `SELECT`\n' + '\n' + ), + link=None) + RQ_SRS_006_RBAC_Privileges_Delete = Requirement( name='RQ.SRS-006.RBAC.Privileges.Delete', version='1.0', @@ -10102,20 +10030,6 @@ RQ_SRS_006_RBAC_Privileges_All_GrantRevoke = Requirement( ), link=None) -RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **grant option** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None) - RQ_SRS_006_RBAC_Privileges_AdminOption = Requirement( name='RQ.SRS-006.RBAC.Privileges.AdminOption', version='1.0', diff --git a/tests/testflows/rbac/tests/errors.py b/tests/testflows/rbac/tests/errors.py deleted file mode 100755 index 50004f03476..00000000000 --- a/tests/testflows/rbac/tests/errors.py +++ /dev/null @@ -1,90 +0,0 @@ -## Syntax - -# Errors: not found - -not_found = "Exception: There is no {type} `{name}` in user directories" - -def user_not_found_in_disk(name): - return (192,not_found.format(type="user",name=name)) - -def role_not_found_in_disk(name): - return (255,not_found.format(type="role",name=name)) - -def settings_profile_not_found_in_disk(name): - return (180,not_found.format(type="settings profile",name=name)) - -def quota_not_found_in_disk(name): - return (199,not_found.format(type="quota",name=name)) - -def row_policy_not_found_in_disk(name): - return (11,not_found.format(type="row policy",name=name)) - -# Errors: cannot_rename - -cannot_rename = "Exception: {type} `{name}`: cannot rename to `{name_new}` because {type} `{name_new}` already exists in local directory" -cannot_rename_exitcode = 237 - -def cannot_rename_user(name,name_new): - return (cannot_rename_exitcode, cannot_rename.format(type="user", name=name, name_new=name_new)) - -def cannot_rename_role(name,name_new): - return (cannot_rename_exitcode, cannot_rename.format(type="role", name=name, name_new=name_new)) - -def cannot_rename_settings_profile(name,name_new): - return (cannot_rename_exitcode, cannot_rename.format(type="settings profile", name=name, name_new=name_new)) - -def cannot_rename_quota(name,name_new): - return (cannot_rename_exitcode, cannot_rename.format(type="quota", name=name, name_new=name_new)) - -def cannot_rename_row_policy(name,name_new): - return (cannot_rename_exitcode, cannot_rename.format(type="row policy", name=name, name_new=name_new)) - -# Errors: cannot insert - -cannot_insert = "Exception: {type} `{name}`: cannot insert because {type} `{name}` already exists in local directory" -cannot_insert_exitcode = 237 - -def cannot_insert_user(name): - return (cannot_insert_exitcode, cannot_insert.format(type="user",name=name)) - -def cannot_insert_role(name): - return (cannot_insert_exitcode, cannot_insert.format(type="role",name=name)) - -def cannot_insert_settings_profile(name): - return (cannot_insert_exitcode, cannot_insert.format(type="settings profile",name=name)) - -def cannot_insert_quota(name): - return (cannot_insert_exitcode, cannot_insert.format(type="quota",name=name)) - -def cannot_insert_row_policy(name): - return (cannot_insert_exitcode, cannot_insert.format(type="row policy",name=name)) - -# Error: default is readonly - -default_readonly_exitcode = 239 -cannot_remove_default = "Exception: Cannot remove {type} `default` from users.xml because this storage is readonly" - -def cannot_update_default(): - return (default_readonly_exitcode, "Exception: Cannot update user `default` in users.xml because this storage is readonly") - -def cannot_remove_user_default(): - return (default_readonly_exitcode, cannot_remove_default.format(type="user")) - -def cannot_remove_settings_profile_default(): - return (default_readonly_exitcode, cannot_remove_default.format(type="settings profile")) - -def cannot_remove_quota_default(): - return (default_readonly_exitcode, cannot_remove_default.format(type="quota")) - -# Other syntax errors - -def unknown_setting(setting): - return (115, f"Exception: Unknown setting {setting}") - -def cluster_not_found(cluster): - return (170, f"Exception: Requested cluster '{cluster}' not found") - -## Privileges - -def not_enough_privileges(name): - return (241, f"Exception: {name}: Not enough privileges.") diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_column.py b/tests/testflows/rbac/tests/privileges/alter/alter_column.py index 46b75949093..f32ca8ce9e2 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_column.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_column.py @@ -476,16 +476,7 @@ def user_with_privileges_on_columns(self, table_type, permutation, node=None): examples=Examples("grant_columns revoke_columns alter_columns_fail table_type permutation", [tuple(list(row)+[table_type, permutation]) for row in self.examples]) - pool = Pool(3) - try: - tasks = [] - try: - for example in examples: - run_scenario(pool, tasks, Scenario(test=user_column_privileges, examples=examples)) - finally: - join(tasks) - finally: - pool.close() + Scenario(test=user_column_privileges, examples=examples)() @TestOutline @Requirements( @@ -593,16 +584,7 @@ def role_with_privileges_on_columns(self, table_type, permutation, node=None): examples=Examples("grant_columns revoke_columns alter_columns_fail table_type permutation", [tuple(list(row)+[table_type, permutation]) for row in self.examples]) - pool = Pool(3) - try: - tasks = [] - try: - for example in examples: - run_scenario(pool, tasks, Scenario(test=role_column_privileges, examples=examples)) - finally: - join(tasks) - finally: - pool.close() + Scenario(test=user_column_privileges, examples=examples)() @TestOutline @Requirements( @@ -676,281 +658,19 @@ def user_with_privileges_on_cluster(self, permutation, table_type, node=None): with Finally("I drop the user on a cluster"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_user_with_grant_option(self, permutation, table_type, node=None): - """Check that user is able to alter column on a table when granted privilege - from another user with grant option. - """ - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with When("I grant privileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to another user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER COLUMN"): - alter_column_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_user_with_grant_option(self, permutation, table_type, node=None): - """Check that user is able to alter column on a table when granted a role with - alter column privilege that was granted by another user with grant option. - """ - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When("I grant subprivileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to a role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings = [("user", user0_name)]) - - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with Then(f"I try to ALTER COLUMN"): - alter_column_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_role_with_grant_option(self, permutation, table_type, node=None): - """Check that user is able to alter column on a table when granted privilege from - a role with grant option - """ - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option to a role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And("I grant privileges to a user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER COLUMN"): - alter_column_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_role_with_grant_option(self, permutation, table_type, node=None): - """Check that a user is able to alter column on a table with a role that was - granted privilege by another role with grant option - """ - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privilege with grant option to role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And("I grant privileges to another role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings = [("user", user0_name)]) - - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with Then(f"I try to ALTER COLUMN"): - alter_column_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_user_with_grant_option(self, permutation, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a user. - """ - # This test does not apply when no privileges are granted - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with When(f"I grant privileges with grant option to user"): - granted_privileges = on_columns(privileges, "d") - node.query(f"GRANT {granted_privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(user0_name) - revoked_privileges = on_columns(privileges, "b") - - if permutation is 0: - """Revoking NONE always succeeds, regardless of privileges. - """ - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)]) - else: - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_user_with_grant_option(self, permutation, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a role. - """ - # This test does not apply when no privileges are granted - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user_name = f"user_{getuid()}" - role_name = f"role_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user_name), role(node, role_name): - with When(f"I grant privileges with grant option to user"): - granted_privileges = on_columns(privileges, "d") - node.query(f"GRANT {granted_privileges} ON {table_name} TO {user_name} WITH GRANT OPTION") - - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(user_name) - revoked_privileges = on_columns(privileges, "b") - - if permutation is 0: - """Revoking NONE always succeeds, regardless of privileges. - """ - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role_name}", settings=[("user", user_name)]) - else: - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role_name}", settings=[("user", user_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_role_with_grant_option(self, permutation, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have access to from a user. - """ - # This test does not apply when no privileges are granted - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option on column d to a role"): - granted_privileges = on_columns(privileges, "d") - node.query(f"GRANT {granted_privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(user0_name) - revoked_privileges = on_columns(privileges, "b") - - if permutation is 0: - """Revoking NONE always succeeds, regardless of privileges. - """ - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)]) - else: - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {user1_name}", settings=[("user", user0_name)], - exitcode=exitcode, message=message) - - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterColumn_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_role_with_grant_option(self, permutation, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have access to from a role. - """ - # This test does not apply when no privileges are granted - privileges = alter_column_privileges(permutation) - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user_name = f"user_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privileges with grant option to a role"): - granted_privileges = on_columns(privileges, "d") - node.query(f"GRANT {granted_privileges} ON {table_name} TO {user_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user_name}") - - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(user_name) - revoked_privileges = on_columns(privileges, "b") - - if permutation is 0: - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role1_name}", settings=[("user", user_name)]) - else: - node.query(f"REVOKE {revoked_privileges} ON {table_name} FROM {role1_name}", settings=[("user", user_name)], - exitcode=exitcode, message=message) +@TestSuite +def scenario_parallelization(self, table_type, permutation): + pool = Pool(7) + try: + tasks = [] + try: + for scenario in loads(current_module(), Scenario): + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), + {"table_type": table_type, "permutation": permutation}) + finally: + join(tasks) + finally: + pool.close() @TestFeature @Requirements( @@ -978,16 +698,16 @@ def feature(self, node="clickhouse1", stress=None, parallel=None): continue with Example(str(example)): - pool = Pool(15) + pool = Pool(10) try: tasks = [] try: for permutation in permutations(table_type): privileges = alter_column_privileges(permutation) - for scenario in loads(current_module(), Scenario): - with Suite(name=f"{privileges}"): - run_scenario(pool, tasks, Scenario(test=scenario), {"table_type": table_type, "permutation": permutation}) + + run_scenario(pool, tasks, Suite(test=scenario_parallelization, name=privileges), + {"table_type": table_type, "permutation": permutation}) finally: join(tasks) finally: - pool.close() \ No newline at end of file + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py b/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py index 2216ed90088..ea040c74a97 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py @@ -271,328 +271,6 @@ def user_with_privileges_on_cluster(self, table_type, node=None): with Finally("I drop the user on a cluster"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER CONSTRAINT on a table when granted privilege - from another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name): - with When("I grant privileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to another user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER CONSTRAINT"): - alter_constraint_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER CONSTRAINT on a table when granted a role with - ALTER CONSTRAINT privilege that was granted by another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When("I grant subprivileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to a role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings = [("user", user0_name)]) - - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with Then(f"I try to ALTER CONSTRAINT"): - alter_constraint_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER CONSTRAINT on a table when granted privilege from - a role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option to a role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And("I grant privileges to a user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER CONSTRAINT"): - alter_constraint_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that a user is able to ALTER CONSTRAINT on a table with a role that was - granted privilege by another role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privileges"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And("I grant privileges to another role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings = [("user", user0_name)]) - - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with Then(f"I try to ALTER CONSTRAINT"): - alter_constraint_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from user0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they dont have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from role0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they dont have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with And("I revoke privilege from user0 using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user" ,user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterConstraint_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they dont have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(): - privileges = alter_constraint_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with And("I revoke privilege from role0(user0) using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterConstraint("1.0"), @@ -617,13 +295,13 @@ def feature(self, node="clickhouse1", parallel=None, stress=None): continue with Example(str(example)): - pool = Pool(13) + pool = Pool(5) try: tasks = [] try: for scenario in loads(current_module(), Scenario): - run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type}) + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), {"table_type" : table_type}) finally: join(tasks) finally: - pool.close() \ No newline at end of file + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_delete.py b/tests/testflows/rbac/tests/privileges/alter/alter_delete.py new file mode 100644 index 00000000000..a4ccbdb6e6b --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_delete.py @@ -0,0 +1,95 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +aliases = {"ALTER DELETE", "DELETE"} + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterDelete_Access("1.0"), +) +def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): + """Check that user is only able to execute ALTER DELETE when they have required privilege, either directly or via role. + """ + role_name = f"role_{getuid()}" + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with Suite("user with direct privilege", setup=instrument_clickhouse_server_log): + with user(node, user_name): + with When(f"I run checks that {user_name} is only able to execute ALTER DELETE with required privileges"): + privilege_check(grant_target_name=user_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role", setup=instrument_clickhouse_server_log): + with user(node, user_name), role(node, role_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + with And(f"I run checks that {user_name} with {role_name} is only able to execute ALTER DELETE with required privileges"): + privilege_check(grant_target_name=role_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + +def privilege_check(grant_target_name, user_name, table_type, privilege, node=None): + """Run scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I attempt to delete columns without privilege"): + node.query(f"ALTER TABLE {table_name} DELETE WHERE 1", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with Then("I attempt to delete columns"): + node.query(f"ALTER TABLE {table_name} DELETE WHERE 1", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with And("I revoke the delete privilege"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") + with Then("I attempt to delete columns"): + node.query(f"ALTER TABLE {table_name} DELETE WHERE 1", settings = [("user", user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterDelete("1.0"), +) +@Examples("table_type", [ + (key,) for key in table_types.keys() +]) +@Name("alter delete") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of ALTER DELETE. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + for example in self.examples: + table_type, = example + + if table_type != "MergeTree" and not self.context.stress: + continue + + with Example(str(example)): + for alias in aliases: + with Suite(alias, test=privilege_granted_directly_or_via_role): + privilege_granted_directly_or_via_role(table_type=table_type, privilege=alias) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py b/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py new file mode 100644 index 00000000000..e01ab244f38 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py @@ -0,0 +1,109 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +aliases = {"ALTER FETCH PARTITION", "FETCH PARTITION"} + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterFetch_Access("1.0"), +) +def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): + """Check that user is only able to execute ALTER FETCH PARTITION when they have required privilege, either directly or via role. + """ + role_name = f"role_{getuid()}" + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with Suite("user with direct privilege", setup=instrument_clickhouse_server_log): + with user(node, user_name): + with When(f"I run checks that {user_name} is only able to execute ALTER FETCH PARTITION with required privileges"): + privilege_check(grant_target_name=user_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role", setup=instrument_clickhouse_server_log): + with user(node, user_name), role(node, role_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + with And(f"I run checks that {user_name} with {role_name} is only able to execute ALTER FETCH PARTITION with required privileges"): + privilege_check(grant_target_name=role_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + +def privilege_check(grant_target_name, user_name, table_type, privilege, node=None): + """Run scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I attempt to fetch a partition without privilege"): + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the fetch privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with Then("I attempt to fetch a partition"): + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + exitcode=231, message="DB::Exception: No node") + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the fetch privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with And("I revoke the fetch privilege"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") + with Then("I attempt to fetch a partition"): + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterFetch("1.0"), +) +@Examples("table_type",[ + ("ReplicatedMergeTree-sharded_cluster",), + ("ReplicatedMergeTree-one_shard_cluster",), + ("ReplicatedReplacingMergeTree-sharded_cluster",), + ("ReplicatedReplacingMergeTree-one_shard_cluster",), + ("ReplicatedSummingMergeTree-sharded_cluster",), + ("ReplicatedSummingMergeTree-one_shard_cluster",), + ("ReplicatedAggregatingMergeTree-sharded_cluster",), + ("ReplicatedAggregatingMergeTree-one_shard_cluster",), + ("ReplicatedCollapsingMergeTree-sharded_cluster",), + ("ReplicatedCollapsingMergeTree-one_shard_cluster",), + ("ReplicatedVersionedCollapsingMergeTree-sharded_cluster",), + ("ReplicatedVersionedCollapsingMergeTree-one_shard_cluster",), + ("ReplicatedGraphiteMergeTree-sharded_cluster",), + ("ReplicatedGraphiteMergeTree-one_shard_cluster",) +]) +@Name("alter fetch") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of ALTER FETCH. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + for example in self.examples: + table_type, = example + + if table_type != "ReplicatedMergeTree-sharded_cluster" and not self.context.stress: + continue + + with Example(str(example)): + for alias in aliases: + with Suite(alias, test=privilege_granted_directly_or_via_role): + privilege_granted_directly_or_via_role(table_type=table_type, privilege=alias) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py b/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py new file mode 100644 index 00000000000..4f1f99467e6 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py @@ -0,0 +1,95 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +aliases = {"ALTER FREEZE PARTITION", "FREEZE PARTITION"} + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterFreeze_Access("1.0"), +) +def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): + """Check that user is only able to execute ALTER FREEZE PARTITION when they have required privilege, either directly or via role. + """ + role_name = f"role_{getuid()}" + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with Scenario("user with direct privilege", setup=instrument_clickhouse_server_log): + with user(node, user_name): + with When(f"I run checks that {user_name} is only able to execute ALTER FREEZE PARTITION with required privileges"): + privilege_check(grant_target_name=user_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + + with Scenario("user with privilege via role", setup=instrument_clickhouse_server_log): + with user(node, user_name), role(node, role_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + with And(f"I run checks that {user_name} with {role_name} is only able to execute ALTER FREEZE PARTITION with required privileges"): + privilege_check(grant_target_name=role_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + +def privilege_check(grant_target_name, user_name, table_type, privilege, node=None): + """Run scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I attempt to freeze partitions without privilege"): + node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the freeze privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with Then("I attempt to freeze partitions"): + node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the freeze privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with And("I revoke the freeze privilege"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") + with Then("I attempt to freeze partitions"): + node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterFreeze("1.0"), +) +@Examples("table_type", [ + (key,) for key in table_types.keys() +]) +@Name("alter freeze") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of ALTER FREEZE. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + for example in self.examples: + table_type, = example + + if table_type != "MergeTree" and not self.context.stress: + continue + + with Example(str(example)): + for alias in aliases: + with Suite(alias, test=privilege_granted_directly_or_via_role): + privilege_granted_directly_or_via_role(table_type=table_type, privilege=alias) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_index.py b/tests/testflows/rbac/tests/privileges/alter/alter_index.py index 89547269b9c..cd2729f5641 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_index.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_index.py @@ -448,336 +448,6 @@ def user_with_privileges_on_cluster(self, table_type, node=None): with Finally("I drop the user on cluster"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER INDEX on a table when granted privilege - from another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name): - with When("I grant privileges needed for iteration with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to another user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER INDEX with given privileges"): - alter_index_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER INDEX on a table when granted a role with - ALTER INDEX privilege that was granted by another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When("I grant needed subprivileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to a role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings = [("user", user0_name)]) - - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with Then(f"I try to ALTER INDEX with given privileges"): - alter_index_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER INDEX on a table when granted privilege from - a role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option to a role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And("I grant privileges to a user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER INDEX with given privileges"): - alter_index_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that a user is able to ALTER INDEX on a table with a role that was - granted privilege by another role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privilege with grant option to role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And("I grant privileges to another role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings = [("user", user0_name)]) - - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with Then(f"I try to ALTER INDEX with given privileges"): - alter_index_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from user0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from role0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with And("I revoke privilege from user0 using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user" ,user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Flags(TE) -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterIndex_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(table_type): - privileges = alter_index_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with And("I revoke privilege from role0(user0) using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterIndex("1.0"), @@ -803,13 +473,13 @@ def feature(self, node="clickhouse1", stress=None, parallel=None): continue with Example(str(example)): - pool = Pool(13) + pool = Pool(5) try: tasks = [] try: for scenario in loads(current_module(), Scenario): - run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type}) + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), {"table_type" : table_type}) finally: join(tasks) finally: - pool.close() \ No newline at end of file + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_move.py b/tests/testflows/rbac/tests/privileges/alter/alter_move.py new file mode 100644 index 00000000000..57d7a7182a9 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_move.py @@ -0,0 +1,157 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +aliases = {"ALTER MOVE PARTITION", "ALTER MOVE PART", "MOVE PARTITION", "MOVE PART"} + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterMove_Access("1.0"), +) +def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): + """Check that user is only able to execute ALTER MOVE PARTITION when they have required privilege, either directly or via role. + """ + role_name = f"role_{getuid()}" + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with Suite("user with direct privilege", setup=instrument_clickhouse_server_log): + with user(node, user_name): + with When(f"I run checks that {user_name} is only able to execute ALTER MOVE PARTITION with required privileges"): + privilege_check(grant_target_name=user_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role", setup=instrument_clickhouse_server_log): + with user(node, user_name), role(node, role_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + with And(f"I run checks that {user_name} with {role_name} is only able to execute ALTER MOVE PARTITION with required privileges"): + privilege_check(grant_target_name=role_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + +def privilege_check(grant_target_name, user_name, table_type, privilege, node=None): + """Run scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + target_table_name = f"target_merge_tree_{getuid()}" + + with table(node, f"{source_table_name},{target_table_name}", table_type): + with When("I attempt to move partition without privilege"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {target_table_name}", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user without ALTER MOVE PARTITION privilege", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + target_table_name = f"target_merge_tree_{getuid()}" + + with table(node, f"{source_table_name},{target_table_name}", table_type): + with When(f"I grant SELECT and ALTER DELETE privileges on {source_table_name} to {grant_target_name}"): + node.query(f"GRANT SELECT, ALTER DELETE ON {source_table_name} TO {grant_target_name}") + with And(f"I grant INSERT on {target_table_name} to {grant_target_name}"): + node.query(f"GRANT INSERT ON {target_table_name} TO {grant_target_name}") + + with Then("I attempt to move partitions without ALTER MOVE privilege"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {target_table_name}", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user with ALTER MOVE PARTITION privilege", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + target_table_name = f"target_merge_tree_{getuid()}" + + with table(node, f"{source_table_name},{target_table_name}", table_type): + with When(f"I grant SELECT, ALTER DELETE, and ALTER MOVE PARTITION privileges on {source_table_name} to {grant_target_name}"): + node.query(f"GRANT SELECT, ALTER DELETE, {privilege} ON {source_table_name} TO {grant_target_name}") + with And(f"I grant INSERT on {target_table_name} to {grant_target_name}"): + node.query(f"GRANT INSERT ON {target_table_name} TO {grant_target_name}") + + with Then("I attempt to move partitions with ALTER MOVE privilege"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {target_table_name}", settings = [("user", user_name)]) + + with Scenario("user with revoked ALTER MOVE PARTITION privilege", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + target_table_name = f"target_merge_tree_{getuid()}" + + with table(node, f"{source_table_name},{target_table_name}", table_type): + with When(f"I grant SELECT, ALTER DELETE, and ALTER MOVE PARTITION privileges on {source_table_name} to {grant_target_name}"): + node.query(f"GRANT SELECT, ALTER DELETE, {privilege} ON {source_table_name} TO {grant_target_name}") + with And(f"I grant INSERT on {target_table_name} to {grant_target_name}"): + node.query(f"GRANT INSERT ON {target_table_name} TO {grant_target_name}") + + with And("I revoke ALTER MOVE PARTITION privilege"): + node.query(f"REVOKE {privilege} ON {source_table_name} FROM {grant_target_name}") + + with Then("I attempt to move partition"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {target_table_name}", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("move partition to source table of a materialized view", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + mat_view_name = f"mat_view_{getuid()}" + mat_view_source_table_name = f"mat_view_source_merge_tree_{getuid()}" + + with table(node, f"{source_table_name},{mat_view_source_table_name}", table_type): + with Given("I have a materialized view"): + node.query(f"CREATE MATERIALIZED VIEW {mat_view_name} ENGINE = {table_type} PARTITION BY y ORDER BY d AS SELECT * FROM {mat_view_source_table_name}") + + with When(f"I grant SELECT, ALTER DELETE, and ALTER MOVE PARTITION privileges on {source_table_name} to {grant_target_name}"): + node.query(f"GRANT SELECT, ALTER DELETE, {privilege} ON {source_table_name} TO {grant_target_name}") + with And(f"I grant INSERT on {mat_view_source_table_name} to {grant_target_name}"): + node.query(f"GRANT INSERT ON {mat_view_source_table_name} TO {grant_target_name}") + + with Then("I attempt to move partitions with ALTER MOVE privilege"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {mat_view_source_table_name}", settings = [("user", user_name)]) + + with Scenario("move partition to implicit target table of a materialized view", setup=instrument_clickhouse_server_log): + source_table_name = f"source_merge_tree_{getuid()}" + mat_view_name = f"mat_view_{getuid()}" + mat_view_source_table_name = f"mat_view_source_merge_tree_{getuid()}" + implicit_table_name = f"\\\".inner.{mat_view_name}\\\"" + + with table(node, f"{source_table_name},{mat_view_source_table_name}", table_type): + with Given("I have a materialized view"): + node.query(f"CREATE MATERIALIZED VIEW {mat_view_name} ENGINE = {table_type} PARTITION BY y ORDER BY d AS SELECT * FROM {mat_view_source_table_name}") + + with When(f"I grant SELECT, ALTER DELETE, and ALTER MOVE PARTITION privileges on {source_table_name} to {grant_target_name}"): + node.query(f"GRANT SELECT, ALTER DELETE, {privilege} ON {source_table_name} TO {grant_target_name}") + with And(f"I grant INSERT on {implicit_table_name} to {grant_target_name}"): + node.query(f"GRANT INSERT ON {implicit_table_name} TO {grant_target_name}") + + with Then("I attempt to move partitions with ALTER MOVE privilege"): + node.query(f"ALTER TABLE {source_table_name} MOVE PARTITION 1 TO TABLE {implicit_table_name}", settings = [("user", user_name)]) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterMove("1.0"), +) +@Examples("table_type", [ + (key,) for key in table_types.keys() +]) +@Name("alter move") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of ALTER MOVE. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + for example in self.examples: + table_type, = example + + if table_type != "MergeTree" and not self.context.stress: + continue + + with Example(str(example)): + for alias in aliases: + with Suite(alias, test=privilege_granted_directly_or_via_role): + privilege_granted_directly_or_via_role(table_type=table_type, privilege=alias) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_settings.py b/tests/testflows/rbac/tests/privileges/alter/alter_settings.py index c4e71be4f81..6ac482fa33d 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_settings.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_settings.py @@ -162,279 +162,20 @@ def user_with_privileges_on_cluster(self, privilege, table_type, node=None): with Finally("I drop the user on a cluster"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_user_with_grant_option(self, privilege, table_type, node=None): - """Check that user is able to alter settings on a table when granted privilege - from another user with grant option. +@TestSuite +def scenario_parallelization(self, table_type, privilege): + """Runs all scenarios in parallel for a given privilege. """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with When("I grant privileges with grant option to user"): - node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to another user via grant option"): - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER SETTINGS"): - check_alter_settings_when_privilege_is_granted(table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_user_with_grant_option(self, privilege, table_type, node=None): - """Check that user is able to alter settings on a table when granted a role with - alter settings privilege that was granted by another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When("I grant subprivileges with grant option to user"): - node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to a role via grant option"): - node.query(f"GRANT {privilege} ON {table_name} TO {role_name}", - settings = [("user", user0_name)]) - - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with Then(f"I try to ALTER SETTINGS"): - check_alter_settings_when_privilege_is_granted(table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_role_with_grant_option(self, privilege, table_type, node=None): - """Check that user is able to alter settings on a table when granted privilege from - a role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option to a role"): - node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And("I grant privileges to a user via grant option"): - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER SETTINGS"): - check_alter_settings_when_privilege_is_granted(table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_role_with_grant_option(self, privilege, table_type, node=None): - """Check that a user is able to alter settings on a table with a role that was - granted privilege by another role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privilege with grant option to role"): - node.query(f"GRANT {privilege} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And("I grant privileges to another role via grant option"): - node.query(f"GRANT {privilege} ON {table_name} TO {role1_name}", - settings = [("user", user0_name)]) - - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with Then(f"I try to ALTER SETTINGS"): - check_alter_settings_when_privilege_is_granted(table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_user_with_grant_option(self, privilege, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from user0 using user1"): - node.query(f"REVOKE {privilege} ON {table_name} FROM {user0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privilege} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_user_with_grant_option(self, privilege, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user_{getuid()}" - user1_name = f"user_{getuid()}" - role_name = f"role_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from role0 using user1"): - node.query(f"REVOKE {privilege} ON {table_name} FROM {role_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privilege} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privilege} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_role_with_grant_option(self, privilege, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privilege} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privilege} ON {table_name} TO {role_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with And("I revoke privilege from user0 using role1(user1)"): - node.query(f"REVOKE {privilege} ON {table_name} FROM {user0_name}", - settings=[("user" ,user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privilege} ON {table_name} TO {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privilege} ON {table_name} FROM {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterSettings_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_role_with_grant_option(self, privilege, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they don't have acces to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user_{getuid()}" - user1_name = f"user_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privilege} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privilege} ON {table_name} TO {role1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with And("I revoke privilege from role0(user0) using role1(user1)"): - node.query(f"REVOKE {privilege} ON {table_name} FROM {role0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privilege} ON {table_name} TO {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privilege} ON {table_name} FROM {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) + pool = Pool(4) + try: + tasks = [] + try: + for scenario in loads(current_module(), Scenario): + run_scenario(pool, tasks, Scenario(test=scenario), {"table_type": table_type, "privilege": privilege}) + finally: + join(tasks) + finally: + pool.close() @TestFeature @Requirements( @@ -462,15 +203,13 @@ def feature(self, node="clickhouse1", stress=None, parallel=None): continue with Example(str(example)): - pool = Pool(13) + pool = Pool(4) try: tasks = [] try: for alias in aliases: - for scenario in loads(current_module(), Scenario): - with Suite(name=f"{alias}"): - run_scenario(pool, tasks, Scenario(test=scenario), {"table_type": table_type, "privilege": alias}) + run_scenario(pool, tasks, Suite(test=scenario_parallelization, name=alias, setup=instrument_clickhouse_server_log), {"table_type": table_type, "privilege": alias}) finally: join(tasks) finally: - pool.close() \ No newline at end of file + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_ttl.py b/tests/testflows/rbac/tests/privileges/alter/alter_ttl.py index 7b6217375a0..00240f19bb4 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_ttl.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_ttl.py @@ -247,328 +247,6 @@ def user_with_privileges_on_cluster(self, table_type, node=None): with Finally("I drop the user on a cluster"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER TTL on a table when granted privilege - from another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type),user(node, user0_name), user(node, user1_name): - with When("I grant privileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to another user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER TTL"): - alter_ttl_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER TTL on a table when granted a role with - ALTER TTL privilege that was granted by another user with grant option. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When("I grant subprivileges with grant option to user"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And("I grant privileges to a role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings = [("user", user0_name)]) - - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with Then(f"I try to ALTER TTL"): - alter_ttl_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"), -) -def user_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that user is able to ALTER TTL on a table when granted privilege from - a role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with When(f"I grant privileges with grant option to a role"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And("I grant privileges to a user via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings = [("user", user0_name)]) - - with Then(f"I try to ALTER TTL"): - alter_ttl_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Grant("1.0"), -) -def role_with_privileges_from_role_with_grant_option(self, table_type, node=None): - """Check that a user is able to ALTER TTL on a table with a role that was - granted privilege by another role with grant option - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with When(f"I grant privileges"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And("I grant privileges to another role via grant option"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings = [("user", user0_name)]) - - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with Then(f"I try to ALTER TTL"): - alter_ttl_privilege_handler(permutation, table_name, user1_name, node) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they don't have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from user0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a privilege they dont have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to user1"): - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I revoke privilege from role0 using user1"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {user1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_user_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they dont have access to from a user. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role_name): - with Given(f"I grant privileges with grant option to user0"): - node.query(f"GRANT {privileges} ON {table_name} TO {user0_name} WITH GRANT OPTION") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role_name} TO {user1_name}") - - with And("I revoke privilege from user0 using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {user0_name}", - settings=[("user" ,user1_name)]) - - with Then("I verify that user0 has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_AlterTTL_GrantOption_Revoke("1.0"), -) -def revoke_privileges_from_role_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a privilege they dont have access to from a role. - """ - if node is None: - node = self.context.node - - table_name = f"merge_tree_{getuid()}" - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - - for permutation in permutations(): - privileges = alter_ttl_privileges(permutation) - - with When(f"granted={privileges}"): - # This test does not apply when no privileges are granted (permutation 0) - if permutation == 0: - continue - - with table(node, table_name, table_type), user(node, user0_name), user(node, user1_name): - with role(node, role0_name), role(node, role1_name): - with Given(f"I grant privileges with grant option to role0"): - node.query(f"GRANT {privileges} ON {table_name} TO {role0_name} WITH GRANT OPTION") - - with And("I grant role0 to user0"): - node.query(f"GRANT {role0_name} TO {user0_name}") - - with And(f"I grant privileges with grant option to role1"): - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name} WITH GRANT OPTION", - settings=[("user", user0_name)]) - - with When("I grant role1 to user1"): - node.query(f"GRANT {role1_name} TO {user1_name}") - - with And("I revoke privilege from role0(user0) using role1(user1)"): - node.query(f"REVOKE {privileges} ON {table_name} FROM {role0_name}", - settings=[("user", user1_name)]) - - with Then("I verify that role0(user0) has privileges revoked"): - exitcode, message = errors.not_enough_privileges(user0_name) - node.query(f"GRANT {privileges} ON {table_name} TO {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - node.query(f"REVOKE {privileges} ON {table_name} FROM {role1_name}", - settings=[("user", user0_name)], exitcode=exitcode, message=message) - @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterTTL("1.0"), @@ -593,13 +271,13 @@ def feature(self, node="clickhouse1", stress=None, parallel=None): continue with Example(str(example)): - pool = Pool(13) + pool = Pool(5) try: tasks = [] try: for scenario in loads(current_module(), Scenario): - run_scenario(pool, tasks, Scenario(test=scenario), {"table_type" : table_type}) + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), {"table_type" : table_type}) finally: join(tasks) finally: - pool.close() \ No newline at end of file + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_update.py b/tests/testflows/rbac/tests/privileges/alter/alter_update.py new file mode 100644 index 00000000000..4cb4bc48c95 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_update.py @@ -0,0 +1,96 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +aliases = {"ALTER UPDATE", "UPDATE"} + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterUpdate_Access("1.0"), +) +def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): + """Check that user is only able to execute ALTER UPDATE when they have required privilege, either directly or via role. + """ + role_name = f"role_{getuid()}" + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with Suite("user with direct privilege", setup=instrument_clickhouse_server_log): + with user(node, user_name): + with When(f"I run checks that {user_name} is only able to execute ALTER UPDATE with required privileges"): + privilege_check(grant_target_name=user_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role", setup=instrument_clickhouse_server_log): + with user(node, user_name), role(node, role_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + with And(f"I run checks that {user_name} with {role_name} is only able to execute ALTER UPDATE with required privileges"): + privilege_check(grant_target_name=role_name, user_name=user_name, table_type=table_type, privilege=privilege, node=node) + +def privilege_check(grant_target_name, user_name, table_type, privilege, node=None): + """Run scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I attempt to update a column without privilege"): + node.query(f"ALTER TABLE {table_name} UPDATE a = x WHERE 1", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the update privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with Then("I attempt to update a column"): + node.query(f"ALTER TABLE {table_name} UPDATE a = x WHERE 1", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, table_name, table_type): + with When("I grant the update privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with And("I revoke the update privilege"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") + with Then("I attempt to update a column"): + node.query(f"ALTER TABLE {table_name} UPDATE a = x WHERE 1", settings = [("user", user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterUpdate("1.0"), + RQ_SRS_006_RBAC_Privileges_AlterUpdate_TableEngines("1.0") +) +@Examples("table_type", [ + (key,) for key in table_types.keys() +]) +@Name("alter update") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of ALTER UPDATE. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + for example in self.examples: + table_type, = example + + if table_type != "MergeTree" and not self.context.stress: + continue + + with Example(str(example)): + for alias in aliases: + with Suite(alias, test=privilege_granted_directly_or_via_role): + privilege_granted_directly_or_via_role(table_type=table_type, privilege=alias) diff --git a/tests/testflows/rbac/tests/privileges/distributed_table.py b/tests/testflows/rbac/tests/privileges/distributed_table.py index 7a065fbcd34..94b27465191 100755 --- a/tests/testflows/rbac/tests/privileges/distributed_table.py +++ b/tests/testflows/rbac/tests/privileges/distributed_table.py @@ -72,7 +72,7 @@ def create(self): ] for scenario in create_scenarios: - Scenario(run=scenario) + Scenario(run=scenario, setup=instrument_clickhouse_server_log) @TestScenario def create_without_privilege(self, node=None): @@ -172,7 +172,7 @@ def select(self): ] for scenario in select_scenarios: - Scenario(run=scenario) + Scenario(run=scenario, setup=instrument_clickhouse_server_log) @TestScenario def select_without_privilege(self, node=None): @@ -286,7 +286,7 @@ def insert(self): ] for scenario in insert_scenarios: - Scenario(run=scenario) + Scenario(run=scenario, setup=instrument_clickhouse_server_log) @TestScenario def insert_without_privilege(self, node=None): @@ -405,7 +405,7 @@ def special_cases(self): ] for scenario in special_case_scenarios: - Scenario(run=scenario) + Scenario(run=scenario, setup=instrument_clickhouse_server_log) @TestScenario def select_with_table_on_materialized_view_privilege_granted_directly_or_via_role(self, node=None): @@ -989,7 +989,7 @@ def feature(self, node="clickhouse1"): self.context.node3 = self.context.cluster.node("clickhouse3") tasks = [] - pool = Pool(5) + pool = Pool(3) try: run_scenario(pool, tasks, Feature(test=cluster_tests)) @@ -997,4 +997,4 @@ def feature(self, node="clickhouse1"): run_scenario(pool, tasks, Scenario(test=multiple_node_user)) finally: - join(tasks) \ No newline at end of file + join(tasks) diff --git a/tests/testflows/rbac/tests/privileges/grant_option.py b/tests/testflows/rbac/tests/privileges/grant_option.py new file mode 100644 index 00000000000..f337aec2619 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/grant_option.py @@ -0,0 +1,139 @@ +from multiprocessing.dummy import Pool + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def grant_option(self, table_type, privilege, node=None): + """Check that user is able to execute GRANT and REVOKE privilege statements if and only if they have the privilege WITH GRANT OPTION, + either directly or through a role. + """ + user0_name = f"user0_{getuid()}" + user1_name = f"user1_{getuid()}" + role0_name = f"role0_{getuid()}" + role1_name = f"role1_{getuid()}" + if node is None: + node = self.context.node + + with Suite("user with direct privilege granting to user"): + with user(node, f"{user0_name},{user1_name}"): + with When(f"I run checks that grant and revoke privilege from {user0_name} to {user1_name}"): + grant_option_check(grant_option_target=user0_name, grant_target=user1_name, user_name=user0_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with direct privilege granting to role"): + with user(node, user0_name), role(node, role1_name): + with When(f"I run checks that grant and revoke privilege from {user0_name} to {role1_name}"): + grant_option_check(grant_option_target=user0_name, grant_target=role1_name, user_name=user0_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role granting to user"): + with user(node, f"{user0_name},{user1_name}"), role(node, role0_name): + with When("I grant the role to the user"): + node.query(f"GRANT {role0_name} TO {user0_name}") + with When(f"I run checks that grant and revoke privilege from {user0_name} with {role0_name} to {user1_name}"): + grant_option_check(grant_option_target=role0_name, grant_target=user1_name, user_name=user0_name, table_type=table_type, privilege=privilege, node=node) + + with Suite("user with privilege via role granting to role"): + with user(node, user0_name), role(node, f"{role0_name},{role1_name}"): + with When("I grant the role to the user"): + node.query(f"GRANT {role0_name} TO {user0_name}") + with When(f"I run checks that grant and revoke privilege from {user0_name} with {role0_name} to {role1_name}"): + grant_option_check(grant_option_target=role0_name, grant_target=role1_name, user_name=user0_name, table_type=table_type, privilege=privilege, node=node) + +def grant_option_check(grant_option_target, grant_target, user_name, table_type, privilege, node=None): + """Run different scenarios to check the user's access with different privileges. + """ + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("grant by user without privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, name=table_name, table_type_name=table_type): + with Then("I attempt to grant delete privilege without privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target}", settings = [("user", user_name)], + exitcode=exitcode, message=message) + + with Scenario("grant by user with grant option privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, name=table_name, table_type_name=table_type): + with When("I grant delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_option_target} WITH GRANT OPTION") + with Then("I attempt to grant delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target}", settings = [("user", user_name)]) + + with Scenario("revoke by user with grant option privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, name=table_name, table_type_name=table_type): + with When("I grant delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_option_target} WITH GRANT OPTION") + with Then("I attempt to revoke delete privilege"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target}", settings = [("user", user_name)]) + + with Scenario("grant by user with revoked grant option privilege", setup=instrument_clickhouse_server_log): + table_name = f"merge_tree_{getuid()}" + with table(node, name=table_name, table_type_name=table_type): + with When(f"I grant delete privilege with grant option to {grant_option_target}"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_option_target} WITH GRANT OPTION") + with And(f"I revoke delete privilege with grant option from {grant_option_target}"): + node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_option_target}") + with Then("I attempt to grant delete privilege"): + node.query(f"GRANT {privilege} ON {table_name} TO {grant_target}", settings = [("user", user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_Privileges_GrantOption("1.0"), +) +@Examples("privilege", [ + ("ALTER MOVE PARTITION",), ("ALTER MOVE PART",), ("MOVE PARTITION",), ("MOVE PART",), + ("ALTER DELETE",), ("DELETE",), + ("ALTER FETCH PARTITION",), ("FETCH PARTITION",), + ("ALTER FREEZE PARTITION",), ("FREEZE PARTITION",), + ("ALTER UPDATE",), ("UPDATE",), + ("ALTER ADD COLUMN",), ("ADD COLUMN",), + ("ALTER CLEAR COLUMN",), ("CLEAR COLUMN",), + ("ALTER MODIFY COLUMN",), ("MODIFY COLUMN",), + ("ALTER RENAME COLUMN",), ("RENAME COLUMN",), + ("ALTER COMMENT COLUMN",), ("COMMENT COLUMN",), + ("ALTER DROP COLUMN",), ("DROP COLUMN",), + ("ALTER COLUMN",), + ("ALTER SETTINGS",), ("ALTER SETTING",), ("ALTER MODIFY SETTING",), ("MODIFY SETTING",), + ("ALTER ORDER BY",), ("ALTER MODIFY ORDER BY",), ("MODIFY ORDER BY",), + ("ALTER SAMPLE BY",), ("ALTER MODIFY SAMPLE BY",), ("MODIFY SAMPLE BY",), + ("ALTER ADD INDEX",), ("ADD INDEX",), + ("ALTER MATERIALIZE INDEX",), ("MATERIALIZE INDEX",), + ("ALTER CLEAR INDEX",), ("CLEAR INDEX",), + ("ALTER DROP INDEX",), ("DROP INDEX",), + ("ALTER INDEX",), ("INDEX",), + ("ALTER TTL",), ("ALTER MODIFY TTL",), ("MODIFY TTL",), + ("ALTER MATERIALIZE TTL",), ("MATERIALIZE TTL",), + ("ALTER ADD CONSTRAINT",), ("ADD CONSTRAINT",), + ("ALTER DROP CONSTRAINT",), ("DROP CONSTRAINT",), + ("ALTER CONSTRAINT",), ("CONSTRAINT",), + ("INSERT",), + ("SELECT",), +]) +@Name("grant option") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of privileges with GRANT OPTION. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + pool = Pool(12) + try: + tasks = [] + try: + for example in self.examples: + privilege, = example + run_scenario(pool, tasks, Suite(test=grant_option, name=privilege, setup=instrument_clickhouse_server_log), {"table_type": "MergeTree", "privilege": privilege}) + finally: + join(tasks) + finally: + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/insert.py b/tests/testflows/rbac/tests/privileges/insert.py index 92620584c23..0ac953a6585 100755 --- a/tests/testflows/rbac/tests/privileges/insert.py +++ b/tests/testflows/rbac/tests/privileges/insert.py @@ -335,227 +335,6 @@ def role_with_privilege_on_cluster(self, table_type, node=None): with Finally("I drop the user"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), -) -def user_with_privilege_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to insert into a table when granted privilege - from another user with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION") - with And("I grant privilege on a column I don't have permission on"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"GRANT INSERT(b) ON {table_name} TO {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - with And("I grant privilege to another user via grant option"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user1_name}", settings=[("user",user0_name)]) - with And("I insert into a table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)]) - with Then("I check that I can read inserted data"): - output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output - assert output == '{"d":"2020-01-01"}', error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), -) -def role_with_privilege_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to insert into a table when granted a role with - insert privilege that was granted by another user with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION") - with And("I grant privilege on a column I don't have permission on"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"GRANT INSERT(b) ON {table_name} TO {role_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - with And("I grant privilege to a role via grant option"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name}", settings=[("user",user0_name)]) - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - with And("I insert into a table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)]) - with Then("I check that I can read inserted data"): - output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output - assert output == '{"d":"2020-01-01"}', error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), -) -def user_with_privilege_from_role_with_grant_option(self, table_type, node=None): - """Check that user is able to insert into a table when granted privilege from a role with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name} WITH GRANT OPTION") - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - with And("I grant privilege on a column I don't have permission on"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"GRANT INSERT(b) ON {table_name} TO {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - with And("I grant privilege to a user via grant option"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user1_name}", settings=[("user",user0_name)]) - with And("I insert into a table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)]) - with Then("I check that I can read inserted data"): - output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output - assert output == '{"d":"2020-01-01"}', error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Grant("1.0"), -) -def role_with_privilege_from_role_with_grant_option(self, table_type, node=None): - """Check that a user is able to insert into a table with a role that was granted privilege - by another role with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"), role(node, f"{role0_name},{role1_name}"): - with When("I grant privilege with grant option to role"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {role0_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - with And("I grant privilege on a column I don't have permission on"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"GRANT INSERT(b) ON {table_name} TO {role1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - with And("I grant privilege to another role via grant option"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {role1_name}", settings=[("user",user0_name)]) - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - with And("I insert into a table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')", settings=[("user",user1_name)]) - with Then("I check that I can read inserted data"): - output = node.query(f"SELECT d FROM {table_name} FORMAT JSONEachRow").output - assert output == '{"d":"2020-01-01"}', error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke insert privilege from another user - on a column they the original user doesn't have access to. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"REVOKE INSERT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke insert privilege from a role - on a column the original user doesn't have access to. - """ - user_name = f"user_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, user_name), role(node, role_name): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user_name} WITH GRANT OPTION") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user_name) - node.query(f"REVOKE INSERT(b) ON {table_name} FROM {role_name}", settings=[("user",user_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke insert privilege from a user - on a column the original user doesn't have access to. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {role_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"REVOKE INSERT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke insert privilege from a role - on a column the original user doesn't have access to. - """ - user_name = f"user_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, user_name), role(node, f"{role0_name},{role1_name}"): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT INSERT(d) ON {table_name} TO {user_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user_name}") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user_name) - node.query(f"REVOKE INSERT(b) ON {table_name} FROM {role1_name}", settings=[("user",user_name)], - exitcode=exitcode, message=message) - @TestOutline(Feature) @Requirements( RQ_SRS_006_RBAC_Privileges_Insert("1.0"), @@ -579,10 +358,10 @@ def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"): self.context.stress = parallel tasks = [] - pool = Pool(3) + pool = Pool(10) try: for scenario in loads(current_module(), Scenario): - run_scenario(pool, tasks, scenario, {"table_type" : table_type}) + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), {"table_type" : table_type}) finally: - join(tasks) \ No newline at end of file + join(tasks) diff --git a/tests/testflows/rbac/tests/privileges/public_tables.py b/tests/testflows/rbac/tests/privileges/public_tables.py index 489c4d21778..bd3cc55f812 100755 --- a/tests/testflows/rbac/tests/privileges/public_tables.py +++ b/tests/testflows/rbac/tests/privileges/public_tables.py @@ -35,4 +35,4 @@ def public_tables(self, node=None): def feature(self, node="clickhouse1"): self.context.node = self.context.cluster.node(node) - Scenario(run=public_tables, flags=TE) \ No newline at end of file + Scenario(run=public_tables, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file diff --git a/tests/testflows/rbac/tests/privileges/select.py b/tests/testflows/rbac/tests/privileges/select.py index 485fe97a8a3..eb6702238b6 100755 --- a/tests/testflows/rbac/tests/privileges/select.py +++ b/tests/testflows/rbac/tests/privileges/select.py @@ -241,7 +241,7 @@ def role_column_privileges(self, grant_columns, select_columns_pass, data_pass, @TestScenario @Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), + RQ_SRS_006_RBAC_Privileges_Select_Cluster("1.0"), ) def user_with_privilege_on_cluster(self, table_type, node=None): """Check that user is able to select from a table with @@ -268,212 +268,6 @@ def user_with_privilege_on_cluster(self, table_type, node=None): with Finally("I drop the user"): node.query(f"DROP USER {user_name} ON CLUSTER sharded_cluster") -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), -) -def user_with_privilege_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to select from a table when granted privilege - from another user with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with Given("I have some data inserted into table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')") - with user(node, f"{user0_name},{user1_name}"): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT SELECT ON {table_name} TO {user0_name} WITH GRANT OPTION") - with And("I grant privilege to another user via grant option"): - node.query(f"GRANT SELECT ON {table_name} TO {user1_name}", settings = [("user",user0_name)]) - with Then("I verify SELECT command"): - user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)]) - default = node.query(f"SELECT d FROM {table_name}") - assert user_select.output == default.output, error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), -) -def role_with_privilege_from_user_with_grant_option(self, table_type, node=None): - """Check that user is able to select from a table when granted a role with - select privilege that was granted by another user with grant option. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with Given("I have some data inserted into table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')") - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT SELECT ON {table_name} TO {user0_name} WITH GRANT OPTION") - with And("I grant privilege to a role via grant option"): - node.query(f"GRANT SELECT ON {table_name} TO {role_name}", settings = [("user",user0_name)]) - with And("I grant the role to another user"): - node.query(f"GRANT {role_name} TO {user1_name}") - with Then("I verify SELECT command"): - user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)]) - default = node.query(f"SELECT d FROM {table_name}") - assert user_select.output == default.output, error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), -) -def user_with_privilege_from_role_with_grant_option(self, table_type, node=None): - """Check that user is able to select from a table when granted privilege from - a role with grant option - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with Given("I have some data inserted into table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')") - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT SELECT ON {table_name} TO {role_name} WITH GRANT OPTION") - with When("I grant role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - with And("I grant privilege to a user via grant option"): - node.query(f"GRANT SELECT ON {table_name} TO {user1_name}", settings = [("user",user0_name)]) - with Then("I verify SELECT command"): - user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)]) - default = node.query(f"SELECT d FROM {table_name}") - assert user_select.output == default.output, error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Grant("1.0"), -) -def role_with_privilege_from_role_with_grant_option(self, table_type, node=None): - """Check that a user is able to select from a table with a role that was - granted privilege by another role with grant option - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with Given("I have some data inserted into table"): - node.query(f"INSERT INTO {table_name} (d) VALUES ('2020-01-01')") - with user(node, f"{user0_name},{user1_name}"), role(node, f"{role0_name},{role1_name}"): - with When("I grant privilege with grant option to role"): - node.query(f"GRANT SELECT ON {table_name} TO {role0_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user0_name}") - with And("I grant privilege to another role via grant option"): - node.query(f"GRANT SELECT ON {table_name} TO {role1_name}", settings = [("user",user0_name)]) - with And("I grant the second role to another user"): - node.query(f"GRANT {role1_name} TO {user1_name}") - with Then("I verify SELECT command"): - user_select = node.query(f"SELECT d FROM {table_name}", settings = [("user",user1_name)]) - default = node.query(f"SELECT d FROM {table_name}") - assert user_select.output == default.output, error() - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_user_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a column they don't have access to from a user. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT SELECT(d) ON {table_name} TO {user0_name} WITH GRANT OPTION") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"REVOKE SELECT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_role_via_user_with_grant_option(self, table_type, node=None): - """Check that user is unable to revoke a column they dont have acces to from a role. - """ - user_name = f"user_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, user_name), role(node, role_name): - with When("I grant privilege with grant option to user"): - node.query(f"GRANT SELECT(d) ON {table_name} TO {user_name} WITH GRANT OPTION") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user_name) - node.query(f"REVOKE SELECT(b) ON {table_name} FROM {role_name}", settings=[("user",user_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_user_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a column they dont have acces to from a user. - """ - user0_name = f"user0_{getuid()}" - user1_name = f"user1_{getuid()}" - role_name = f"role_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, f"{user0_name},{user1_name}"), role(node, role_name): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT SELECT(d) ON {table_name} TO {role_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role_name} TO {user0_name}") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user0_name) - node.query(f"REVOKE SELECT(b) ON {table_name} FROM {user1_name}", settings=[("user",user0_name)], - exitcode=exitcode, message=message) - -@TestScenario -@Requirements( - RQ_SRS_006_RBAC_Privileges_Select_GrantOption_Revoke("1.0"), -) -def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node=None): - """Check that user with a role is unable to revoke a column they dont have acces to from a role. - """ - user_name = f"user_{getuid()}" - role0_name = f"role0_{getuid()}" - role1_name = f"role1_{getuid()}" - table_name = f"table_{getuid()}" - if node is None: - node = self.context.node - with table(node, table_name, table_type): - with user(node, user_name), role(node, f"{role0_name},{role1_name}"): - with When("I grant privilege with grant option to a role"): - node.query(f"GRANT SELECT(d) ON {table_name} TO {role0_name} WITH GRANT OPTION") - with And("I grant the role to a user"): - node.query(f"GRANT {role0_name} TO {user_name}") - with Then("I revoke privilege on a column the user with grant option does not have access to"): - exitcode, message = errors.not_enough_privileges(name=user_name) - node.query(f"REVOKE SELECT(b) ON {table_name} FROM {role1_name}", settings=[("user",user_name)], - exitcode=exitcode, message=message) - @TestOutline(Feature) @Requirements( RQ_SRS_006_RBAC_Privileges_Select("1.0"), @@ -492,10 +286,10 @@ def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"): self.context.stress = parallel tasks = [] - pool = Pool(3) + pool = Pool(10) try: for scenario in loads(current_module(), Scenario): - run_scenario(pool, tasks, scenario, {"table_type" : table_type}) + run_scenario(pool, tasks, Scenario(test=scenario, setup=instrument_clickhouse_server_log), {"table_type" : table_type}) finally: - join(tasks) \ No newline at end of file + join(tasks) diff --git a/tests/testflows/rbac/tests/privileges/show_tables.py b/tests/testflows/rbac/tests/privileges/show_tables.py index 5625cd146dd..b8cbc7ddb2b 100755 --- a/tests/testflows/rbac/tests/privileges/show_tables.py +++ b/tests/testflows/rbac/tests/privileges/show_tables.py @@ -59,4 +59,4 @@ def show_tables_general(self, grant_target_name, user_name, node=None): def feature(self, node="clickhouse1"): self.context.node = self.context.cluster.node(node) - Scenario(run=show_tables, flags=TE) \ No newline at end of file + Scenario(run=show_tables, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file diff --git a/tests/testflows/rbac/tests/views/live_view.py b/tests/testflows/rbac/tests/views/live_view.py index 0ffbb123197..a510877a95b 100755 --- a/tests/testflows/rbac/tests/views/live_view.py +++ b/tests/testflows/rbac/tests/views/live_view.py @@ -30,15 +30,15 @@ def allow_experimental_live_view(node): def create(self, node=None): """Test the RBAC functionality of the `CREATE LIVE VIEW` command. """ - Scenario(run=create_without_create_view_privilege) - Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role) - Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role) - Scenario(run=create_without_source_table_privilege) - Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role) + Scenario(run=create_without_create_view_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def create_without_create_view_privilege(self, node=None): @@ -473,15 +473,15 @@ def create_with_nested_views(self, grant_target_name, user_name, node=None): def select(self, node=None): """Test the RBAC functionality of the `SELECT FROM live view` command. """ - Scenario(run=select_without_select_privilege) - Scenario(run=select_with_select_privilege_granted_directly_or_via_role) - Scenario(run=select_with_select_privilege_revoked_directly_or_from_role) - Scenario(run=select_without_source_table_privilege) - Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role) + Scenario(run=select_without_select_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def select_without_select_privilege(self, node=None): @@ -923,8 +923,8 @@ def select_with_nested_views(self, grant_target_name, user_name, node=None): def drop(self, node=None): """Test the RBAC functionality of the `DROP VIEW` command. """ - Scenario(run=drop_with_privilege_granted_directly_or_via_role) - Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role) + Scenario(run=drop_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) @TestScenario def drop_with_privilege_granted_directly_or_via_role(self, node=None): @@ -1025,8 +1025,8 @@ def drop_with_revoked_privilege(self, grant_target_name, user_name, node=None): def refresh(self, node=None): """Test the RBAC functionality of the `ALTER LIVE VIEW REFRESH` command. """ - Scenario(run=refresh_with_privilege_granted_directly_or_via_role) - Scenario(run=refresh_with_privilege_revoked_directly_or_from_role) + Scenario(run=refresh_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=refresh_with_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) @TestScenario def refresh_with_privilege_granted_directly_or_via_role(self, node=None): diff --git a/tests/testflows/rbac/tests/views/materialized_view.py b/tests/testflows/rbac/tests/views/materialized_view.py index 97d41b9c15e..fc3b393c114 100755 --- a/tests/testflows/rbac/tests/views/materialized_view.py +++ b/tests/testflows/rbac/tests/views/materialized_view.py @@ -30,19 +30,19 @@ def allow_experimental_alter_materialized_view_structure(node): def create(self, node=None): """Test the RBAC functionality of the `CREATE MATERIALIZED VIEW` command. """ - Scenario(run=create_without_create_view_privilege) - Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role) - Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role) - Scenario(run=create_without_source_table_privilege) - Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=create_with_union_query_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_union_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role) - Scenario(run=create_with_target_table_privilege_directly_or_via_role) - Scenario(run=create_with_populate_privilege_granted_directly_or_via_role) - Scenario(run=create_with_populate_source_table_privilege_granted_directly_or_via_role) + Scenario(run=create_without_create_view_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_union_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_union_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_target_table_privilege_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_populate_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_populate_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def create_without_create_view_privilege(self, node=None): @@ -714,17 +714,17 @@ def create_with_populate_source_table(self, user_name, grant_target_name, node=N def select(self, node=None): """Test the RBAC functionality of the `SELECT FROM materialized view` command """ - Scenario(run=select_without_select_privilege) - Scenario(run=select_with_select_privilege_granted_directly_or_via_role) - Scenario(run=select_with_select_privilege_revoked_directly_or_from_role) - Scenario(run=select_without_source_table_privilege) - Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=select_with_union_query_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_union_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role) - Scenario(run=select_with_privilege_granted_directly_or_via_role_without_target_table_privilege) + Scenario(run=select_without_select_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_union_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_union_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_privilege_granted_directly_or_via_role_without_target_table_privilege, setup=instrument_clickhouse_server_log) @TestScenario def select_without_select_privilege(self, node=None): @@ -1266,9 +1266,9 @@ def select_without_target_table_privilege(self, grant_target_name, user_name, no def select_from_tables(self, node=None): """Testing RBAC functionality of SELECT for tables related to materialized views - target tables, source tables. """ - Scenario(run=select_from_implicit_target_table_privilege_granted_directly_or_via_role) - Scenario(run=select_from_explicit_target_table_privilege_granted_directly_or_via_role) - Scenario(run=select_from_source_table_privilege_granted_directly_or_via_role) + Scenario(run=select_from_implicit_target_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_from_explicit_target_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_from_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario @Requirements( @@ -1447,8 +1447,8 @@ def select_from_source_table(self, grant_target_name, user_name, node=None): def drop(self, node=None): """Test the RBAC functionality of the `DROP VIEW` command. """ - Scenario(run=drop_with_privilege_granted_directly_or_via_role) - Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role) + Scenario(run=drop_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) @TestScenario def drop_with_privilege_granted_directly_or_via_role(self, node=None): @@ -1554,15 +1554,15 @@ def modify_query(self, node=None): node = self.context.node with allow_experimental_alter_materialized_view_structure(node): - Scenario(run=modify_query_with_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_privilege_revoked_directly_or_from_role) - Scenario(run=modify_query_without_source_table_privilege) - Scenario(run=modify_query_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_union_query_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_join_union_subquery_privilege_granted_directly_or_via_role) - Scenario(run=modify_query_with_nested_views_privilege_granted_directly_or_via_role) + Scenario(run=modify_query_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_union_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_join_union_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=modify_query_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def modify_query_with_privilege_granted_directly_or_via_role(self, node=None): @@ -2049,9 +2049,9 @@ def modify_query_with_nested_views(self, grant_target_name, user_name, node=None def insert(self, node=None): """Check RBAC functionality of INSERT with materialized views. """ - Scenario(run=insert_on_source_table_privilege_granted_directly_or_via_role) - Scenario(run=insert_with_privilege_granted_directly_or_via_role) - Scenario(run=insert_on_target_table_privilege_granted_directly_or_via_role) + Scenario(run=insert_on_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=insert_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=insert_on_target_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario @Requirements( diff --git a/tests/testflows/rbac/tests/views/view.py b/tests/testflows/rbac/tests/views/view.py index 60b006324cf..524eda91297 100755 --- a/tests/testflows/rbac/tests/views/view.py +++ b/tests/testflows/rbac/tests/views/view.py @@ -12,16 +12,16 @@ import rbac.helper.errors as errors def create(self, node=None): """Test the RBAC functionality of the `CREATE VIEW` command. """ - Scenario(run=create_without_create_view_privilege) - Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role) - Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role) - Scenario(run=create_without_source_table_privilege) - Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=create_with_union_query_privilege_granted_directly_or_via_role) - Scenario(run=create_with_join_union_subquery_privilege_granted_directly_or_via_role) - Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role) + Scenario(run=create_without_create_view_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_create_view_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_revoked_create_view_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_union_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_join_union_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=create_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def create_without_create_view_privilege(self, node=None): @@ -525,16 +525,16 @@ def create_with_nested_views(self, grant_target_name, user_name, node=None): def select(self, node=None): """Test the RBAC functionality of the `SELECT FROM view` command. """ - Scenario(run=select_without_select_privilege) - Scenario(run=select_with_select_privilege_granted_directly_or_via_role) - Scenario(run=select_with_select_privilege_revoked_directly_or_from_role) - Scenario(run=select_without_source_table_privilege) - Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role) - Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role) - Scenario(run=select_with_union_query_privilege_granted_directly_or_via_role) - Scenario(run=select_with_join_union_subquery_privilege_granted_directly_or_via_role) - Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role) + Scenario(run=select_without_select_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_select_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_without_source_table_privilege, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_source_table_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_union_query_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_join_union_subquery_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=select_with_nested_views_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) @TestScenario def select_without_select_privilege(self, node=None): @@ -1039,8 +1039,8 @@ def select_with_nested_views(self, grant_target_name, user_name, node=None): def drop(self, node=None): """Test the RBAC functionality of the `DROP VIEW` command. """ - Scenario(run=drop_with_privilege_granted_directly_or_via_role) - Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role) + Scenario(run=drop_with_privilege_granted_directly_or_via_role, setup=instrument_clickhouse_server_log) + Scenario(run=drop_with_revoked_privilege_revoked_directly_or_from_role, setup=instrument_clickhouse_server_log) @TestScenario From 6fcd7eee3990dda4dc79ddb6e4755aa407e0212d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 21:06:40 +0300 Subject: [PATCH 114/149] Changelog for 20.10 --- CHANGELOG.md | 216 ++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 214 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 09ce72d20ed..457346aff9a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,218 @@ +## ClickHouse release 20.10 + +### ClickHouse release v20.10.3.30, 2020-10-28 + +#### Backward Incompatible Change + +* Make `multiple_joins_rewriter_version` obsolete. Remove first version of joins rewriter. [#15472](https://github.com/ClickHouse/ClickHouse/pull/15472) ([Artem Zuikov](https://github.com/4ertus2)). +* Change default value of `format_regexp_escaping_rule` setting (it's related to `Regexp` format) to `Raw` (it means - read whole subpattern as a value) to make the behaviour more like to what users expect. [#15426](https://github.com/ClickHouse/ClickHouse/pull/15426) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for nested multiline comments `/* comment /* comment */ */` in SQL. This conforms to the SQL standard. [#14655](https://github.com/ClickHouse/ClickHouse/pull/14655) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added MergeTree settings (`max_replicated_merges_with_ttl_in_queue` and `max_number_of_merges_with_ttl_in_pool`) to control the number of merges with TTL in the background pool and replicated queue. This change breaks compatibility with older versions only if you use delete TTL. Otherwise, replication will stay compatible. You can avoid incompatibility issues if you update all shard replicas at once or execute `SYSTEM STOP TTL MERGES` until you finish the update of all replicas. If you'll get an incompatible entry in the replication queue, first of all, execute `SYSTEM STOP TTL MERGES` and after `ALTER TABLE ... DETACH PARTITION ...` the partition where incompatible TTL merge was assigned. Attach it back on a single replica. [#14490](https://github.com/ClickHouse/ClickHouse/pull/14490) ([alesapin](https://github.com/alesapin)). + +#### New Feature + +* Background data recompression. Add the ability to specify `TTL ... RECOMPRESS codec_name` for MergeTree table engines family. [#14494](https://github.com/ClickHouse/ClickHouse/pull/14494) ([alesapin](https://github.com/alesapin)). +* Add parallel quorum inserts. This closes [#15601](https://github.com/ClickHouse/ClickHouse/issues/15601). [#15601](https://github.com/ClickHouse/ClickHouse/pull/15601) ([Latysheva Alexandra](https://github.com/alexelex)). +* Settings for additional enforcement of data durability. Useful for non-replicated setups. [#11948](https://github.com/ClickHouse/ClickHouse/pull/11948) ([Anton Popov](https://github.com/CurtizJ)). +* When duplicate block is written to replica where it does not exist locally (has not been fetched from replicas), don't ignore it and write locally to achieve the same effect as if it was successfully replicated. [#11684](https://github.com/ClickHouse/ClickHouse/pull/11684) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now we support `WITH AS (subquery) ... ` to introduce named subqueries in the query context. This closes [#2416](https://github.com/ClickHouse/ClickHouse/issues/2416). This closes [#4967](https://github.com/ClickHouse/ClickHouse/issues/4967). [#14771](https://github.com/ClickHouse/ClickHouse/pull/14771) ([Amos Bird](https://github.com/amosbird)). +* Introduce `enable_global_with_statement` setting which propagates the first select's `WITH` statements to other select queries at the same level, and makes aliases in `WITH` statements visible to subqueries. [#15451](https://github.com/ClickHouse/ClickHouse/pull/15451) ([Amos Bird](https://github.com/amosbird)). +* Secure inter-cluster query execution (with initial_user as current query user). [#13156](https://github.com/ClickHouse/ClickHouse/pull/13156) ([Azat Khuzhin](https://github.com/azat)). [#15551](https://github.com/ClickHouse/ClickHouse/pull/15551) ([Azat Khuzhin](https://github.com/azat)). +* Add the ability to remove column properties and table TTLs. Introduced queries `ALTER TABLE MODIFY COLUMN col_name REMOVE what_to_remove` and `ALTER TABLE REMOVE TTL`. Both operations are lightweight and executed at the metadata level. [#14742](https://github.com/ClickHouse/ClickHouse/pull/14742) ([alesapin](https://github.com/alesapin)). +* Added format `RawBLOB`. It is intended for input or output a single value without any escaping and delimiters. This closes [#15349](https://github.com/ClickHouse/ClickHouse/issues/15349). [#15364](https://github.com/ClickHouse/ClickHouse/pull/15364) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the `reinterpretAsUUID` function that allows to convert a big-endian byte string to UUID. [#15480](https://github.com/ClickHouse/ClickHouse/pull/15480) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Implement `force_data_skipping_indices` setting. [#15642](https://github.com/ClickHouse/ClickHouse/pull/15642) ([Azat Khuzhin](https://github.com/azat)). +* Add a setting `output_format_pretty_row_numbers` to numerate the result in Pretty formats. This closes [#15350](https://github.com/ClickHouse/ClickHouse/issues/15350). [#15443](https://github.com/ClickHouse/ClickHouse/pull/15443) ([flynn](https://github.com/ucasFL)). +* Added query obfuscation tool. It allows to share more queries for better testing. This closes [#15268](https://github.com/ClickHouse/ClickHouse/issues/15268). [#15321](https://github.com/ClickHouse/ClickHouse/pull/15321) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add table function `null('structure')`. [#14797](https://github.com/ClickHouse/ClickHouse/pull/14797) ([vxider](https://github.com/Vxider)). +* Added `formatReadableQuantity` function. It is useful for reading big numbers by human. [#14725](https://github.com/ClickHouse/ClickHouse/pull/14725) ([Artem Hnilov](https://github.com/BooBSD)). +* Add format `LineAsString` that accepts a sequence of lines separated by newlines, every line is parsed as a whole as a single String field. [#14703](https://github.com/ClickHouse/ClickHouse/pull/14703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)), [#13846](https://github.com/ClickHouse/ClickHouse/pull/13846) ([hexiaoting](https://github.com/hexiaoting)). +* Add `JSONStrings` format which output data in arrays of strings. [#14333](https://github.com/ClickHouse/ClickHouse/pull/14333) ([hcz](https://github.com/hczhcz)). +* Add support for "Raw" column format for `Regexp` format. It allows to simply extract subpatterns as a whole without any escaping rules. [#15363](https://github.com/ClickHouse/ClickHouse/pull/15363) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow configurable `NULL` representation for `TSV` output format. It is controlled by the setting `output_format_tsv_null_representation` which is `\N` by default. This closes [#9375](https://github.com/ClickHouse/ClickHouse/issues/9375). Note that the setting only controls output format and `\N` is the only supported `NULL` representation for `TSV` input format. [#14586](https://github.com/ClickHouse/ClickHouse/pull/14586) ([Kruglov Pavel](https://github.com/Avogar)). +* Support Decimal data type for `MaterializedMySQL`. `MaterializedMySQL` is an experimental feature. [#14535](https://github.com/ClickHouse/ClickHouse/pull/14535) ([Winter Zhang](https://github.com/zhang2014)). +* Add new feature: `SHOW DATABASES LIKE 'xxx'`. [#14521](https://github.com/ClickHouse/ClickHouse/pull/14521) ([hexiaoting](https://github.com/hexiaoting)). +* Added a script to import (arbitrary) git repository to ClickHouse as a sample dataset. [#14471](https://github.com/ClickHouse/ClickHouse/pull/14471) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now insert statements can have asterisk (or variants) with column transformers in the column list. [#14453](https://github.com/ClickHouse/ClickHouse/pull/14453) ([Amos Bird](https://github.com/amosbird)). +* New query complexity limit settings `max_rows_to_read_leaf`, `max_bytes_to_read_leaf` for distributed queries to limit max rows/bytes read on the leaf nodes. Limit is applied for local reads only, *excluding* the final merge stage on the root node. [#14221](https://github.com/ClickHouse/ClickHouse/pull/14221) ([Roman Khavronenko](https://github.com/hagen1778)). +* Allow user to specify settings for `ReplicatedMergeTree*` storage in `` section of config file. It works similarly to `` section. For `ReplicatedMergeTree*` storages settings from `` and `` are applied together, but settings from `` has higher priority. Added `system.replicated_merge_tree_settings` table. [#13573](https://github.com/ClickHouse/ClickHouse/pull/13573) ([Amos Bird](https://github.com/amosbird)). +* Add `mapPopulateSeries` function. [#13166](https://github.com/ClickHouse/ClickHouse/pull/13166) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Supporting MySQL types: `decimal` (as ClickHouse `Decimal`) and `datetime` with sub-second precision (as `DateTime64`). [#11512](https://github.com/ClickHouse/ClickHouse/pull/11512) ([Vasily Nemkov](https://github.com/Enmk)). +* Introduce `event_time_microseconds` field to `system.text_log`, `system.trace_log`, `system.query_log` and `system.query_thread_log` tables. [#14760](https://github.com/ClickHouse/ClickHouse/pull/14760) ([Bharat Nallan](https://github.com/bharatnc)). +* Add `event_time_microseconds` to `system.asynchronous_metric_log` & `system.metric_log` tables. [#14514](https://github.com/ClickHouse/ClickHouse/pull/14514) ([Bharat Nallan](https://github.com/bharatnc)). +* Add `query_start_time_microseconds` field to `system.query_log` & `system.query_thread_log` tables. [#14252](https://github.com/ClickHouse/ClickHouse/pull/14252) ([Bharat Nallan](https://github.com/bharatnc)). + +#### Bug Fix + +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `executable` dictionary source hang. In previous versions, when using some formats (e.g. `JSONEachRow`) data was not feed to a child process before it outputs at least something. This closes [#1697](https://github.com/ClickHouse/ClickHouse/issues/1697). This closes [#2455](https://github.com/ClickHouse/ClickHouse/issues/2455). [#14525](https://github.com/ClickHouse/ClickHouse/pull/14525) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT with prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix segfault in some cases of wrong aggregation in lambdas. [#16082](https://github.com/ClickHouse/ClickHouse/pull/16082) ([Anton Popov](https://github.com/CurtizJ)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* `MaterializedMySQL` (experimental feature): Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use `direct` layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* `MaterializedMySQL` (experimental feature): Fix crash on create database failure. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) - Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixes [#12513](https://github.com/ClickHouse/ClickHouse/issues/12513): difference expressions with same alias when query is reanalyzed. [#15886](https://github.com/ClickHouse/ClickHouse/pull/15886) ([Winter Zhang](https://github.com/zhang2014)). +* Fix possible very rare deadlocks in RBAC implementation. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* `MaterializedMySQL` (experimental feature): Fix `select count()` inaccuracy. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fix drop of materialized view with inner table in Atomic database (hangs all subsequent DROP TABLE due to hang of the worker thread, due to recursive DROP TABLE for inner table of MV). [#15743](https://github.com/ClickHouse/ClickHouse/pull/15743) ([Azat Khuzhin](https://github.com/azat)). +* Possibility to move part to another disk/volume if the first attempt was failed. [#15723](https://github.com/ClickHouse/ClickHouse/pull/15723) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix rare race condition in dictionaries and tables from MySQL. [#15686](https://github.com/ClickHouse/ClickHouse/pull/15686) ([alesapin](https://github.com/alesapin)). +* Fix (benign) race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Proper error handling during insert into MergeTree with S3. MergeTree over S3 is an experimental feature. [#15657](https://github.com/ClickHouse/ClickHouse/pull/15657) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed bug with S3 table function: region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Subtract `ReadonlyReplica` metric when detach readonly tables. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Query will finish faster in case of exception. Cancel execution on remote replicas if exception happens. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `Database doesn't exist.` in queries with IN and Distributed table when there's no database on initiator. [#15538](https://github.com/ClickHouse/ClickHouse/pull/15538) ([Artem Zuikov](https://github.com/4ertus2)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fixes [#15365](https://github.com/ClickHouse/ClickHouse/issues/15365): attach a database with MySQL engine throws exception (no query context). [#15384](https://github.com/ClickHouse/ClickHouse/pull/15384) ([Winter Zhang](https://github.com/zhang2014)). +* Fix the case of multiple occurrences of column transformers in a select query. [#15378](https://github.com/ClickHouse/ClickHouse/pull/15378) ([Amos Bird](https://github.com/amosbird)). +* Fixed compression in `S3` storage. [#15376](https://github.com/ClickHouse/ClickHouse/pull/15376) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix bug where queries like `SELECT toStartOfDay(today())` fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix instance crash when using `joinGet` with `LowCardinality` types. This fixes https://github.com/ClickHouse/ClickHouse/issues/15214. [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust Decimal field size in MySQL column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* Fixes `Data compressed with different methods` in `join_algorithm='auto'`. Keep LowCardinality as type for left table join key in `join_algorithm='partial_merge'`. [#15088](https://github.com/ClickHouse/ClickHouse/pull/15088) ([Artem Zuikov](https://github.com/4ertus2)). +* Update `jemalloc` to fix `percpu_arena` with affinity mask. [#15035](https://github.com/ClickHouse/ClickHouse/pull/15035) ([Azat Khuzhin](https://github.com/azat)). [#14957](https://github.com/ClickHouse/ClickHouse/pull/14957) ([Azat Khuzhin](https://github.com/azat)). +* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes https://github.com/ClickHouse/ClickHouse/issues/14908. [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in Docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Fix crash in RIGHT or FULL JOIN with join_algorith='auto' when memory limit exceeded and we should change HashJoin with MergeJoin. [#15002](https://github.com/ClickHouse/ClickHouse/pull/15002) ([Artem Zuikov](https://github.com/4ertus2)). +* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). +* Fix to make predicate push down work when subquery contains `finalizeAggregation` function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes https://github.com/ClickHouse/ClickHouse/issues/14923. [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* `MaterializedMySQL` (experimental feature): Fixed `.metadata.tmp File exists` error. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile with file descriptor. [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fixed segfault in `cache` dictionary [#14837](https://github.com/ClickHouse/ClickHouse/issues/14837). [#14879](https://github.com/ClickHouse/ClickHouse/pull/14879) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* `MaterializedMySQL` (experimental feature): Fixed bug in parsing MySQL binlog events, which causes `Attempt to read after eof` and `Packet payload is not fully read` in `MaterializeMySQL` database engine. [#14852](https://github.com/ClickHouse/ClickHouse/pull/14852) ([Winter Zhang](https://github.com/zhang2014)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* `Replace` column transformer should replace identifiers with cloned ASTs. This fixes https://github.com/ClickHouse/ClickHouse/issues/14695 . [#14734](https://github.com/ClickHouse/ClickHouse/pull/14734) ([Amos Bird](https://github.com/amosbird)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ALTER UPDATE` mutation with `Nullable` column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix function `has` with `LowCardinality` of `Nullable`. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator `-Resample`, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). +* Fix a bug when converting `Nullable(String)` to Enum. Introduced by https://github.com/ClickHouse/ClickHouse/pull/12745. This fixes https://github.com/ClickHouse/ClickHouse/issues/14435. [#14530](https://github.com/ClickHouse/ClickHouse/pull/14530) ([Amos Bird](https://github.com/amosbird)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `currentDatabase()` function cannot be used in `ON CLUSTER` ddl query. [#14211](https://github.com/ClickHouse/ClickHouse/pull/14211) ([Winter Zhang](https://github.com/zhang2014)). +* `MaterializedMySQL` (experimental feature): Fixed `Packet payload is not fully read` error in `MaterializeMySQL` database engine. [#14696](https://github.com/ClickHouse/ClickHouse/pull/14696) ([BohuTANG](https://github.com/BohuTANG)). + +#### Improvement + +* Enable `Atomic` database engine by default for newly created databases. [#15003](https://github.com/ClickHouse/ClickHouse/pull/15003) ([tavplubix](https://github.com/tavplubix)). +* Add the ability to specify specialized codecs like `Delta`, `T64`, etc. for columns with subtypes. Implements [#12551](https://github.com/ClickHouse/ClickHouse/issues/12551), fixes [#11397](https://github.com/ClickHouse/ClickHouse/issues/11397), fixes [#4609](https://github.com/ClickHouse/ClickHouse/issues/4609). [#15089](https://github.com/ClickHouse/ClickHouse/pull/15089) ([alesapin](https://github.com/alesapin)). +* Dynamic reload of zookeeper config. [#14678](https://github.com/ClickHouse/ClickHouse/pull/14678) ([sundyli](https://github.com/sundy-li)). +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Now `joinGet` supports multi-key lookup. Continuation of [#12418](https://github.com/ClickHouse/ClickHouse/issues/12418). [#13015](https://github.com/ClickHouse/ClickHouse/pull/13015) ([Amos Bird](https://github.com/amosbird)). +* Wait for `DROP/DETACH TABLE` to actually finish if `NO DELAY` or `SYNC` is specified for `Atomic` database. [#15448](https://github.com/ClickHouse/ClickHouse/pull/15448) ([tavplubix](https://github.com/tavplubix)). +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). +* The function `now` allows an argument with timezone. This closes [15264](https://github.com/ClickHouse/ClickHouse/issues/15264). [#15285](https://github.com/ClickHouse/ClickHouse/pull/15285) ([flynn](https://github.com/ucasFL)). +* Do not allow connections to ClickHouse server until all scripts in `/docker-entrypoint-initdb.d/` are executed. [#15244](https://github.com/ClickHouse/ClickHouse/pull/15244) ([Aleksei Kozharin](https://github.com/alekseik1)). +* Added `optimize` setting to `EXPLAIN PLAN` query. If enabled, query plan level optimisations are applied. Enabled by default. [#15201](https://github.com/ClickHouse/ClickHouse/pull/15201) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Proper exception message for wrong number of arguments of CAST. This closes [#13992](https://github.com/ClickHouse/ClickHouse/issues/13992). [#15029](https://github.com/ClickHouse/ClickHouse/pull/15029) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add option to disable TTL move on data part insert. [#15000](https://github.com/ClickHouse/ClickHouse/pull/15000) ([Pavel Kovalenko](https://github.com/Jokser)). +* Ignore key constraints when doing mutations. Without this pull request, it's not possible to do mutations when `force_index_by_date = 1` or `force_primary_key = 1`. [#14973](https://github.com/ClickHouse/ClickHouse/pull/14973) ([Amos Bird](https://github.com/amosbird)). +* Allow to drop Replicated table if previous drop attempt was failed due to ZooKeeper session expiration. This fixes [#11891](https://github.com/ClickHouse/ClickHouse/issues/11891). [#14926](https://github.com/ClickHouse/ClickHouse/pull/14926) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed excessive settings constraint violation when running SELECT with SETTINGS from a distributed table. [#14876](https://github.com/ClickHouse/ClickHouse/pull/14876) ([Amos Bird](https://github.com/amosbird)). +* Provide a `load_balancing_first_offset` query setting to explicitly state what the first replica is. It's used together with `FIRST_OR_RANDOM` load balancing strategy, which allows to control replicas workload. [#14867](https://github.com/ClickHouse/ClickHouse/pull/14867) ([Amos Bird](https://github.com/amosbird)). +* Show subqueries for `SET` and `JOIN` in `EXPLAIN` result. [#14856](https://github.com/ClickHouse/ClickHouse/pull/14856) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow using multi-volume storage configuration in storage `Distributed`. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Construct `query_start_time` and `query_start_time_microseconds` from the same timespec. [#14831](https://github.com/ClickHouse/ClickHouse/pull/14831) ([Bharat Nallan](https://github.com/bharatnc)). +* Support for disabling persistency for `StorageJoin` and `StorageSet`, this feature is controlled by setting `disable_set_and_join_persistency`. And this PR solved issue [#6318](https://github.com/ClickHouse/ClickHouse/issues/6318). [#14776](https://github.com/ClickHouse/ClickHouse/pull/14776) ([vxider](https://github.com/Vxider)). +* Now `COLUMNS` can be used to wrap over a list of columns and apply column transformers afterwards. [#14775](https://github.com/ClickHouse/ClickHouse/pull/14775) ([Amos Bird](https://github.com/amosbird)). +* Add `merge_algorithm` to `system.merges` table to improve merging inspections. [#14705](https://github.com/ClickHouse/ClickHouse/pull/14705) ([Amos Bird](https://github.com/amosbird)). +* Fix potential memory leak caused by zookeeper exists watch. [#14693](https://github.com/ClickHouse/ClickHouse/pull/14693) ([hustnn](https://github.com/hustnn)). +* Allow parallel execution of distributed DDL. [#14684](https://github.com/ClickHouse/ClickHouse/pull/14684) ([Azat Khuzhin](https://github.com/azat)). +* Add `QueryMemoryLimitExceeded` event counter. This closes [#14589](https://github.com/ClickHouse/ClickHouse/issues/14589). [#14647](https://github.com/ClickHouse/ClickHouse/pull/14647) ([fastio](https://github.com/fastio)). +* Fix some trailing whitespaces in query formatting. [#14595](https://github.com/ClickHouse/ClickHouse/pull/14595) ([Azat Khuzhin](https://github.com/azat)). +* ClickHouse treats partition expr and key expr differently. Partition expr is used to construct an minmax index containing related columns, while primary key expr is stored as an expr. Sometimes user might partition a table at coarser levels, such as `partition by i / 1000`. However, binary operators are not monotonic and this PR tries to fix that. It might also benifit other use cases. [#14513](https://github.com/ClickHouse/ClickHouse/pull/14513) ([Amos Bird](https://github.com/amosbird)). +* Add an option to skip access checks for `DiskS3`. `s3` disk is an experimental feature. [#14497](https://github.com/ClickHouse/ClickHouse/pull/14497) ([Pavel Kovalenko](https://github.com/Jokser)). +* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). +* `SYSTEM RELOAD CONFIG` now throws an exception if failed to reload and continues using the previous users.xml. The background periodic reloading also continues using the previous users.xml if failed to reload. [#14492](https://github.com/ClickHouse/ClickHouse/pull/14492) ([Vitaly Baranov](https://github.com/vitlibar)). +* For INSERTs with inline data in VALUES format in the script mode of `clickhouse-client`, support semicolon as the data terminator, in addition to the new line. Closes https://github.com/ClickHouse/ClickHouse/issues/12288. [#13192](https://github.com/ClickHouse/ClickHouse/pull/13192) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + +#### Performance Improvement + +* Enable compact parts by default for small parts. This will allow to process frequent inserts slightly more efficiently (4..100 times). [#11913](https://github.com/ClickHouse/ClickHouse/pull/11913) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve `quantileTDigest` performance. This fixes [#2668](https://github.com/ClickHouse/ClickHouse/issues/2668). [#15542](https://github.com/ClickHouse/ClickHouse/pull/15542) ([Kruglov Pavel](https://github.com/Avogar)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Faster 256-bit multiplication. [#15418](https://github.com/ClickHouse/ClickHouse/pull/15418) ([Artem Zuikov](https://github.com/4ertus2)). +* Improve performance of 256-bit types using (u)int64_t as base type for wide integers. Original wide integers use 8-bit types as base. [#14859](https://github.com/ClickHouse/ClickHouse/pull/14859) ([Artem Zuikov](https://github.com/4ertus2)). +* Explicitly use a temporary disk to store vertical merge temporary data. [#15639](https://github.com/ClickHouse/ClickHouse/pull/15639) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). +* Use one S3 DeleteObjects request instead of multiple DeleteObject in a loop. No any functionality changes, so covered by existing tests like integration/test_log_family_s3. [#15238](https://github.com/ClickHouse/ClickHouse/pull/15238) ([ianton-ru](https://github.com/ianton-ru)). +* Fix `DateTime DateTime` mistakenly choosing the slow generic implementation. This fixes https://github.com/ClickHouse/ClickHouse/issues/15153. [#15178](https://github.com/ClickHouse/ClickHouse/pull/15178) ([Amos Bird](https://github.com/amosbird)). +* Improve performance of GROUP BY key of type `FixedString`. [#15034](https://github.com/ClickHouse/ClickHouse/pull/15034) ([Amos Bird](https://github.com/amosbird)). +* Only `mlock` code segment when starting clickhouse-server. In previous versions, all mapped regions were locked in memory, including debug info. Debug info is usually splitted to a separate file but if it isn't, it led to +2..3 GiB memory usage. [#14929](https://github.com/ClickHouse/ClickHouse/pull/14929) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* ClickHouse binary become smaller due to link time optimization. + +#### Build/Testing/Packaging Improvement + +* Now we use clang-11 for production ClickHouse build. [#15239](https://github.com/ClickHouse/ClickHouse/pull/15239) ([alesapin](https://github.com/alesapin)). +* Now we use clang-11 to build ClickHouse in CI. [#14846](https://github.com/ClickHouse/ClickHouse/pull/14846) ([alesapin](https://github.com/alesapin)). +* Switch binary builds (Linux, Darwin, AArch64, FreeDSD) to clang-11. [#15622](https://github.com/ClickHouse/ClickHouse/pull/15622) ([Ilya Yatsishin](https://github.com/qoega)). +* Now all test images use `llvm-symbolizer-11`. [#15069](https://github.com/ClickHouse/ClickHouse/pull/15069) ([alesapin](https://github.com/alesapin)). +* Allow to build with llvm-11. [#15366](https://github.com/ClickHouse/ClickHouse/pull/15366) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Switch from `clang-tidy-10` to `clang-tidy-11`. [#14922](https://github.com/ClickHouse/ClickHouse/pull/14922) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Use LLVM's experimental pass manager by default. [#15608](https://github.com/ClickHouse/ClickHouse/pull/15608) ([Danila Kutenin](https://github.com/danlark1)). +* Don't allow any C++ translation unit to build more than 10 minutes or to use more than 10 GB or memory. This fixes [#14925](https://github.com/ClickHouse/ClickHouse/issues/14925). [#15060](https://github.com/ClickHouse/ClickHouse/pull/15060) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make performance test more stable and representative by splitting test runs and profile runs. [#15027](https://github.com/ClickHouse/ClickHouse/pull/15027) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Attempt to make performance test more reliable. It is done by remapping the executable memory of the process on the fly with `madvise` to use transparent huge pages - it can lower the number of iTLB misses which is the main source of instabilities in performance tests. [#14685](https://github.com/ClickHouse/ClickHouse/pull/14685) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Convert to python3. This closes [#14886](https://github.com/ClickHouse/ClickHouse/issues/14886). [#15007](https://github.com/ClickHouse/ClickHouse/pull/15007) ([Azat Khuzhin](https://github.com/azat)). +* Fail early in functional tests if server failed to respond. This closes [#15262](https://github.com/ClickHouse/ClickHouse/issues/15262). [#15267](https://github.com/ClickHouse/ClickHouse/pull/15267) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to run AArch64 version of clickhouse-server without configs. This facilitates [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174). [#15266](https://github.com/ClickHouse/ClickHouse/pull/15266) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improvements in CI docker images: get rid of ZooKeeper and single script for test configs installation. [#15215](https://github.com/ClickHouse/ClickHouse/pull/15215) ([alesapin](https://github.com/alesapin)). +* Fix CMake options forwarding in fast test script. Fixes error in [#14711](https://github.com/ClickHouse/ClickHouse/issues/14711). [#15155](https://github.com/ClickHouse/ClickHouse/pull/15155) ([alesapin](https://github.com/alesapin)). +* Added a script to perform hardware benchmark in a single command. [#15115](https://github.com/ClickHouse/ClickHouse/pull/15115) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Splitted huge test `test_dictionaries_all_layouts_and_sources` into smaller ones. [#15110](https://github.com/ClickHouse/ClickHouse/pull/15110) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Maybe fix MSan report in base64 (on servers with AVX-512). This fixes [#14006](https://github.com/ClickHouse/ClickHouse/issues/14006). [#15030](https://github.com/ClickHouse/ClickHouse/pull/15030) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Reformat and cleanup code in all integration test *.py files. [#14864](https://github.com/ClickHouse/ClickHouse/pull/14864) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix MaterializeMySQL empty transaction unstable test case found in CI. [#14854](https://github.com/ClickHouse/ClickHouse/pull/14854) ([Winter Zhang](https://github.com/zhang2014)). +* Attempt to speed up build a little. [#14808](https://github.com/ClickHouse/ClickHouse/pull/14808) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Speed up build a little by removing unused headers. [#14714](https://github.com/ClickHouse/ClickHouse/pull/14714) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix build failure in OSX. [#14761](https://github.com/ClickHouse/ClickHouse/pull/14761) ([Winter Zhang](https://github.com/zhang2014)). +* Enable ccache by default in cmake if it's found in OS. [#14575](https://github.com/ClickHouse/ClickHouse/pull/14575) ([alesapin](https://github.com/alesapin)). +* Control CI builds configuration from the ClickHouse repository. [#14547](https://github.com/ClickHouse/ClickHouse/pull/14547) ([alesapin](https://github.com/alesapin)). +* In CMake files: - Moved some options' descriptions' parts to comments above. - Replace 0 -> `OFF`, 1 -> `ON` in `option`s default values. - Added some descriptions and links to docs to the options. - Replaced `FUZZER` option (there is another option `ENABLE_FUZZING` which also enables same functionality). - Removed `ENABLE_GTEST_LIBRARY` option as there is `ENABLE_TESTS`. See the full description in PR: [#14711](https://github.com/ClickHouse/ClickHouse/pull/14711) ([Mike](https://github.com/myrrc)). +* Make binary a bit smaller (~50 Mb for debug version). [#14555](https://github.com/ClickHouse/ClickHouse/pull/14555) ([Artem Zuikov](https://github.com/4ertus2)). +* Use std::filesystem::path in ConfigProcessor for concatenating file paths. [#14558](https://github.com/ClickHouse/ClickHouse/pull/14558) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix debug assertion in `bitShiftLeft()` when called with negative big integer. [#14697](https://github.com/ClickHouse/ClickHouse/pull/14697) ([Artem Zuikov](https://github.com/4ertus2)). + + ## ClickHouse release 20.9 -### ClickHouse release v20.9.2.20-stable, 2020-09-22 +### ClickHouse release v20.9.2.20, 2020-09-22 #### New Feature @@ -84,7 +296,6 @@ #### New Feature -* ClickHouse can work as MySQL replica - it is implemented by `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). * Add the ability to specify `Default` compression codec for columns that correspond to settings specified in `config.xml`. Implements: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). * Support Kerberos authentication in Kafka, using `krb5` and `cyrus-sasl` libraries. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). * Add function `normalizeQuery` that replaces literals, sequences of literals and complex aliases with placeholders. Add function `normalizedQueryHash` that returns identical 64bit hash values for similar queries. It helps to analyze query log. This closes [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -184,6 +395,7 @@ #### Experimental Feature +* ClickHouse can work as MySQL replica - it is implemented by `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). * Add types `Int128`, `Int256`, `UInt256` and related functions for them. Extend Decimals with Decimal256 (precision up to 76 digits). New types are under the setting `allow_experimental_bigint_types`. It is working extremely slow and bad. The implementation is incomplete. Please don't use this feature. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). #### Build/Testing/Packaging Improvement From 8d42305184778a65ad25f33dcb0e37a6e2f87cf1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 5 Nov 2020 21:07:44 +0300 Subject: [PATCH 115/149] fix transform_null_in with sets of tuples --- src/Interpreters/NullableUtils.cpp | 9 ++---- src/Interpreters/NullableUtils.h | 2 +- src/Interpreters/Set.cpp | 20 ++++++------ src/Interpreters/Set.h | 3 +- .../01558_transform_null_in.reference | 17 ++++++++++ .../0_stateless/01558_transform_null_in.sql | 31 +++++++++++++++++++ 6 files changed, 62 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/01558_transform_null_in.reference create mode 100644 tests/queries/0_stateless/01558_transform_null_in.sql diff --git a/src/Interpreters/NullableUtils.cpp b/src/Interpreters/NullableUtils.cpp index d1586bec06b..ce681b1d569 100644 --- a/src/Interpreters/NullableUtils.cpp +++ b/src/Interpreters/NullableUtils.cpp @@ -5,7 +5,7 @@ namespace DB { -ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map, bool exact_null) +ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map) { ColumnPtr null_map_holder; @@ -38,12 +38,7 @@ ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullM PaddedPODArray & mutable_null_map = assert_cast(*mutable_null_map_holder).getData(); const PaddedPODArray & other_null_map = column_nullable->getNullMapData(); for (size_t i = 0, size = mutable_null_map.size(); i < size; ++i) - { - if (exact_null) - mutable_null_map[i] &= other_null_map[i]; - else - mutable_null_map[i] |= other_null_map[i]; - } + mutable_null_map[i] |= other_null_map[i]; null_map_holder = std::move(mutable_null_map_holder); } diff --git a/src/Interpreters/NullableUtils.h b/src/Interpreters/NullableUtils.h index f8294832c1d..8750a60f247 100644 --- a/src/Interpreters/NullableUtils.h +++ b/src/Interpreters/NullableUtils.h @@ -9,6 +9,6 @@ namespace DB * In 'null_map' return a map of positions where at least one column was NULL. * @returns ownership column of null_map. */ -ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map, bool exact_null = false); +ColumnPtr extractNestedColumnsAndNullMap(ColumnRawPtrs & key_columns, ConstNullMapPtr & null_map); } diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 0f976066165..13606e1d54c 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -87,8 +87,6 @@ void NO_INLINE Set::insertFromBlockImplCase( { if ((*null_map)[i]) { - has_null = true; - if constexpr (build_filter) { (*out_filter)[i] = false; @@ -140,7 +138,9 @@ void Set::setHeader(const Block & header) /// We will insert to the Set only keys, where all components are not NULL. ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map, transform_null_in); + ColumnPtr null_map_holder; + if (!transform_null_in) + extractNestedColumnsAndNullMap(key_columns, null_map); if (fill_set_elements) { @@ -180,7 +180,9 @@ bool Set::insertFromBlock(const Block & block) /// We will insert to the Set only keys, where all components are not NULL. ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map, transform_null_in); + ColumnPtr null_map_holder; + if (!transform_null_in) + null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); /// Filter to extract distinct values from the block. ColumnUInt8::MutablePtr filter; @@ -259,8 +261,9 @@ ColumnPtr Set::execute(const Block & block, bool negative) const /// We will check existence in Set only for keys, where all components are not NULL. ConstNullMapPtr null_map{}; - - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map, transform_null_in); + ColumnPtr null_map_holder; + if (!transform_null_in) + null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); executeOrdinary(key_columns, vec_res, negative, null_map); @@ -303,10 +306,7 @@ void NO_INLINE Set::executeImplCase( { if (has_null_map && (*null_map)[i]) { - if (transform_null_in && has_null) - vec_res[i] = !negative; - else - vec_res[i] = negative; + vec_res[i] = negative; } else { diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index faae9c7318e..d3c9b8f684b 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -108,10 +108,9 @@ private: /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements; + /// If true, insert NULL values to set. bool transform_null_in; - bool has_null = false; - /// Check if set contains all the data. bool is_created = false; diff --git a/tests/queries/0_stateless/01558_transform_null_in.reference b/tests/queries/0_stateless/01558_transform_null_in.reference new file mode 100644 index 00000000000..7555eedd5d0 --- /dev/null +++ b/tests/queries/0_stateless/01558_transform_null_in.reference @@ -0,0 +1,17 @@ +4 +2 +2 +1 +0 +3 +============== +1 +1 +0 +1 +1 +0 +1 +1 +1 +0 diff --git a/tests/queries/0_stateless/01558_transform_null_in.sql b/tests/queries/0_stateless/01558_transform_null_in.sql new file mode 100644 index 00000000000..0036c08a649 --- /dev/null +++ b/tests/queries/0_stateless/01558_transform_null_in.sql @@ -0,0 +1,31 @@ +SET transform_null_in = 1; + +DROP TABLE IF EXISTS null_in_1; +CREATE TABLE null_in_1 (u UInt32, n Nullable(UInt32)) ENGINE = Memory; +INSERT INTO null_in_1 VALUES (1, NULL), (2, 2), (3, NULL), (4, 4), (5, NULL); + +SELECT count() FROM null_in_1 WHERE n IN (1, 2, NULL); +SELECT count() FROM null_in_1 WHERE u IN (1, 2, NULL); +SELECT count() FROM null_in_1 WHERE (u, n) IN ((1, 2), (1, NULL), (2, 2)); +SELECT count() FROM null_in_1 WHERE (u, n) IN ((NULL, NULL), (2, 2), (NULL, 2)); +SELECT count() FROM null_in_1 WHERE (u, n) IN (42, NULL); +SELECT count() FROM null_in_1 WHERE (u, n) NOT IN ((3, NULL), (5, NULL)); + +SELECT '=============='; +DROP TABLE IF EXISTS null_in_1; + +CREATE TABLE null_in_1 (a Nullable(UInt32), b Nullable(UInt32)) ENGINE = Memory; +INSERT INTO null_in_1 VALUES (1, NULL) (0, NULL) (NULL, NULL) (NULL, 1) (NULL, 0) (0, 0) (1, 1); + +SELECT count() FROM null_in_1 WHERE (a, b) IN (1, NULL); +SELECT count() FROM null_in_1 WHERE (a, b) IN (0, NULL); +SELECT count() FROM null_in_1 WHERE (a, b) IN (42, NULL); +SELECT count() FROM null_in_1 WHERE (a, b) IN (NULL, 0); +SELECT count() FROM null_in_1 WHERE (a, b) IN (NULL, 1); +SELECT count() FROM null_in_1 WHERE (a, b) IN (NULL, 42); +SELECT count() FROM null_in_1 WHERE (a, b) IN (NULL, NULL); +SELECT count() FROM null_in_1 WHERE (a, b) IN (0, 0); +SELECT count() FROM null_in_1 WHERE (a, b) IN (1, 1); +SELECT count() FROM null_in_1 WHERE (a, b) IN (1, 42); + +DROP TABLE IF EXISTS null_in_1; From e53f604924deaca2d9930471b417be6eb5ea1503 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 5 Nov 2020 21:43:18 +0300 Subject: [PATCH 116/149] Update 01304_direct_io.sh --- tests/queries/0_stateless/01304_direct_io.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01304_direct_io.sh b/tests/queries/0_stateless/01304_direct_io.sh index dcf2adbd64f..244e4c6e02d 100755 --- a/tests/queries/0_stateless/01304_direct_io.sh +++ b/tests/queries/0_stateless/01304_direct_io.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT --multiquery --query " INSERT INTO bug SELECT rand64(), '2020-06-07' FROM numbers(50000000); OPTIMIZE TABLE bug FINAL;" -$CLICKHOUSE_BENCHMARK --database "$CLICKHOUSE_DATABASE" --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err +$CLICKHOUSE_BENCHMARK --iterations 10 --max_threads 100 --min_bytes_to_use_direct_io 1 <<< "SELECT sum(UserID) FROM bug PREWHERE NOT ignore(Date)" 1>/dev/null 2>"$CLICKHOUSE_TMP"/err cat "$CLICKHOUSE_TMP"/err | grep Exception cat "$CLICKHOUSE_TMP"/err | grep Loaded From 19ea4f1a95cb502565f9ee227606cf7f62c3505d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 5 Nov 2020 21:50:15 +0300 Subject: [PATCH 117/149] Auto version update to [20.11.1.5109] [54442] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index fadf3cadfa1..89187c37767 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -3,7 +3,7 @@ SET(VERSION_REVISION 54442) SET(VERSION_MAJOR 20) SET(VERSION_MINOR 11) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 76a04fb4b4f6cd27ad999baf6dc9a25e88851c42) -SET(VERSION_DESCRIBE v20.11.1.1-prestable) -SET(VERSION_STRING 20.11.1.1) +SET(VERSION_GITHASH c53725fb1f846fda074347607ab582fbb9c6f7a1) +SET(VERSION_DESCRIBE v20.11.1.5109-prestable) +SET(VERSION_STRING 20.11.1.5109) # end of autochange From 990f8a23ab7292a1e144b4d1541be78222ae1c9c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 5 Nov 2020 21:51:36 +0300 Subject: [PATCH 118/149] Translate comments to english --- tests/performance/cpu_synthetic.xml | 71 +++++++++++++++-------------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/tests/performance/cpu_synthetic.xml b/tests/performance/cpu_synthetic.xml index 85a4e54c752..26a88d1f097 100644 --- a/tests/performance/cpu_synthetic.xml +++ b/tests/performance/cpu_synthetic.xml @@ -4,74 +4,75 @@ hits_10m_single - + + + SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(SearchPhrase)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(SearchPhrase)) - SELECT count() FROM hits_10m_single WHERE NOT ignore(farmHash64(SearchPhrase)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(farmHash64(SearchPhrase)) - SELECT count() FROM hits_10m_single WHERE NOT ignore(metroHash64(SearchPhrase)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(metroHash64(SearchPhrase)) - -SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(SearchPhrase)) SETTINGS max_threads = 1 -SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(SearchPhrase)) - - -SELECT count() FROM hits_100m_single WHERE NOT ignore(MD5(SearchPhrase)) - - - SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(URL)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(URL)) - -SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(URL)) SETTINGS max_threads = 1 -SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(URL)) - SELECT count() FROM hits_10m_single WHERE NOT ignore(cityHash64(PageCharset)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(cityHash64(PageCharset)) - + + +SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(SearchPhrase)) SETTINGS max_threads = 1 +SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(SearchPhrase)) +SELECT count() FROM hits_100m_single WHERE NOT ignore(MD5(SearchPhrase)) +SELECT count() FROM hits_10m_single WHERE NOT ignore(sipHash64(URL)) SETTINGS max_threads = 1 +SELECT count() FROM hits_100m_single WHERE NOT ignore(sipHash64(URL)) + + SELECT count() FROM hits_10m_single WHERE URL LIKE '%metrika%' SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE URL LIKE '%metrika%' - + + SELECT count() FROM hits_10m_single WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0 SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE positionCaseInsensitiveUTF8(URL, 'новости') != 0 - + + SELECT count() FROM hits_10m_single WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014') SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE match(URL, '^https?://(?:www\\.)?metri[kc]a\\.yandex\\.(?:ru|com|com\\.tr|ua|by|kz)/.+?2014') - + + SELECT SearchEngineID, SearchPhrase, RegionID FROM hits_10m_single GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10 SETTINGS max_threads = 1 SELECT SearchEngineID, SearchPhrase, RegionID FROM hits_100m_single GROUP BY SearchEngineID, SearchPhrase, RegionID ORDER BY count() DESC LIMIT 10 - + + SELECT count() FROM hits_10m_single WHERE NOT ignore(toMonday(EventTime)) SETTINGS max_threads = 1 - + + SELECT count() FROM hits_10m_single WHERE NOT ignore(cutQueryString(URL)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(cutQueryString(URL)) - + + SELECT quantilesIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single SETTINGS max_threads = 1 - SELECT quantilesTimingIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single SETTINGS max_threads = 1 - SELECT quantilesExactIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single SETTINGS max_threads = 1 - SELECT quantilesTDigestIf(0.5, 0.9)(SendTiming, SendTiming > 0) FROM hits_100m_single SETTINGS max_threads = 1 - + + SELECT uniq(UserID) FROM hits_10m_single SETTINGS max_threads = 1 SELECT uniq(UserID) FROM hits_100m_single - SELECT uniqCombined(17)(UserID) FROM hits_10m_single SETTINGS max_threads = 1 SELECT uniqCombined(17)(UserID) FROM hits_100m_single - SELECT uniqExact(UserID) FROM hits_10m_single SETTINGS max_threads = 1 SELECT uniqExact(UserID) FROM hits_100m_single - + + SELECT RegionID, uniq(UserID) FROM hits_10m_single GROUP BY RegionID SETTINGS max_threads = 1 SELECT RegionID, uniq(UserID) FROM hits_100m_single GROUP BY RegionID - + + SELECT count() FROM hits_10m_single WHERE NOT ignore(Referer, URL, Title) SETTINGS max_threads = 1 SELECT count() FROM hits_10m_single WHERE NOT ignore(Referer, URL, Title) From f7b7854ace769c6ec4e4774dbcf29656149fc69a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 5 Nov 2020 21:52:50 +0300 Subject: [PATCH 119/149] Auto version update to [20.12.1.1] [54443] --- cmake/autogenerated_versions.txt | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../System/StorageSystemContributors.generated.cpp | 12 ++++++++++++ 6 files changed, 21 insertions(+), 9 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 89187c37767..0e65568f185 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54442) +SET(VERSION_REVISION 54443) SET(VERSION_MAJOR 20) -SET(VERSION_MINOR 11) +SET(VERSION_MINOR 12) SET(VERSION_PATCH 1) SET(VERSION_GITHASH c53725fb1f846fda074347607ab582fbb9c6f7a1) -SET(VERSION_DESCRIBE v20.11.1.5109-prestable) -SET(VERSION_STRING 20.11.1.5109) +SET(VERSION_DESCRIBE v20.12.1.1-prestable) +SET(VERSION_STRING 20.12.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index 7629346a5e4..3da82efd47e 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (20.11.1.1) unstable; urgency=low +clickhouse (20.12.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Sat, 10 Oct 2020 18:39:55 +0300 + -- clickhouse-release Thu, 05 Nov 2020 21:52:47 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index a5dcf0cbdef..2223b942429 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.11.1.* +ARG version=20.12.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index c552f90e635..1ce6e427409 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.11.1.* +ARG version=20.12.1.* ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index a3c4aa0f2e9..cd2bead5616 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=20.11.1.* +ARG version=20.12.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index f251fe5ad37..1540a43716f 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -6,8 +6,10 @@ const char * auto_contributors[] { "243f6a8885a308d313198a2e037", "821008736@qq.com", "Akazz", + "Alain BERRIER", "Albert Kidrachev", "Alberto", + "Aleksandr Karo", "Aleksandra (Ася)", "Aleksei Levushkin", "Aleksey", @@ -77,6 +79,7 @@ const char * auto_contributors[] { "Andrey Urusov", "Andy Yang", "Anna", + "Anna Shakhova", "Anton Ivashkin", "Anton Kobzev", "Anton Kvasha", @@ -151,6 +154,7 @@ const char * auto_contributors[] { "Dmitriev Mikhail", "Dmitrii Kovalkov", "Dmitrii Raev", + "Dmitriy", "Dmitry", "Dmitry Belyavtsev", "Dmitry Bilunov", @@ -183,6 +187,7 @@ const char * auto_contributors[] { "Evgeny Konkov", "Ewout", "Fabian Stäber", + "Fabiano Francesconi", "Fadi Hadzh", "Fan()", "FawnD2", @@ -205,6 +210,7 @@ const char * auto_contributors[] { "Gleb-Tretyakov", "Gregory", "Grigory", + "Grigory Buteyko", "Grigory Pervakov", "Guillaume Tassery", "Hamoon", @@ -242,6 +248,7 @@ const char * auto_contributors[] { "Ivan Remen", "Ivan Starkov", "Ivan Zhukov", + "Jacob Hayes", "JaosnHsieh", "Jason", "Javi Santana", @@ -397,6 +404,7 @@ const char * auto_contributors[] { "Reilee", "Reto Kromer", "Ri", + "Robert Hodges", "Roman Bug", "Roman Lipovsky", "Roman Nikolaev", @@ -499,11 +507,13 @@ const char * auto_contributors[] { "Weiqing Xu", "William Shallum", "Winter Zhang", + "Xianda Ke", "Y Lu", "Yangkuan Liu", "Yatsishin Ilya", "Yağızcan Değirmenci", "Yegor Andreenko", + "Yingchun Lai", "Yiğit Konur", "Yohann Jardin", "Yuntao Wu", @@ -533,6 +543,7 @@ const char * auto_contributors[] { "alexey-milovidov", "amoschen", "amudong", + "ana-uvarova", "andrei-karpliuk", "andrewsg", "anrodigina", @@ -710,6 +721,7 @@ const char * auto_contributors[] { "sundyli", "svladykin", "tai", + "taichong", "taiyang-li", "tao jiang", "tavplubix", From 59cdc964a17c6f55e7476e1d469dfb1269a69910 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Nov 2020 21:58:42 +0300 Subject: [PATCH 120/149] Do not store reference to BackgroundSchedulePool in DirectoryMonitor (useless) --- src/Storages/Distributed/DirectoryMonitor.cpp | 3 +-- src/Storages/Distributed/DirectoryMonitor.h | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f40ce1e06fc..43a957a2fd9 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -79,7 +79,7 @@ namespace StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool_) + StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool) : storage(storage_) , pool(std::move(pool_)) , path{path_ + '/'} @@ -92,7 +92,6 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , max_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()} , log{&Poco::Logger::get(getLoggerName())} , monitor_blocker(monitor_blocker_) - , bg_pool(bg_pool_) , metric_pending_files(CurrentMetrics::DistributedFilesToInsert, 0) { task_handle = bg_pool.createTask(getLoggerName() + "/Bg", [this]{ run(); }); diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index a6fb78c8db3..1d34357b3b1 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -25,7 +25,7 @@ class StorageDistributedDirectoryMonitor { public: StorageDistributedDirectoryMonitor( - StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool_); + StorageDistributed & storage_, std::string path_, ConnectionPoolPtr pool_, ActionBlocker & monitor_blocker_, BackgroundSchedulePool & bg_pool); ~StorageDistributedDirectoryMonitor(); @@ -95,7 +95,6 @@ private: Poco::Logger * log; ActionBlocker & monitor_blocker; - BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder task_handle; CurrentMetrics::Increment metric_pending_files; From 16c048cacd85e754254317a5d6e3af5c7f80edf5 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 5 Nov 2020 23:44:36 +0300 Subject: [PATCH 121/149] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык описания настройки max_concurrent_queries_for_all_users. --- .../settings.md | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 17966ef0547..2a23a27f4c0 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -467,6 +467,26 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 100 ``` +## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} + +Если значение этой настройки меньше или равно текущему количеству одновременно обрабатываемых запросов, то будет сгенерировано исключение. + +Пример: `max_concurrent_queries_for_all_users` установлен на 99 для всех пользователей. Чтобы выполнять запросы даже когда сервер перегружен, администратор баз данных устанавливает для себя значение настройки на 100. + +Изменение настройки для одного запроса или пользователя не влияет на другие запросы. + +Значение по умолчанию: `0` — отсутствие ограничений. + +**Пример** + +``` xml +99 +``` + +**Смотрите также** + +- [max_concurrent_queries](#max-concurrent-queries) + ## max_connections {#max-connections} Максимальное количество входящих соединений. From ac487ab26a15b530ae3b8c543b6249da545b8ce1 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 5 Nov 2020 23:45:58 +0300 Subject: [PATCH 122/149] Update compare.sh --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 60d226f025d..7b436833225 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1096,7 +1096,7 @@ function upload_results --database perftest \ --date_time_input_format=best_effort \ --query " - insert into query_metrics_tmp + insert into query_metrics_v1 select toDate(event_time) event_date, toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, From f23995d290dc38c3eb2cbfd585b17ae1c4253d80 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Nov 2020 21:58:43 +0300 Subject: [PATCH 123/149] Remove empty directories for async INSERT at start of Distributed engine Will be created by DistributedBlockOutputStream on demand. --- src/Storages/StorageDistributed.cpp | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0c1561fca9b..4409be73e52 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -670,8 +670,28 @@ void StorageDistributed::createDirectoryMonitors(const std::string & disk) std::filesystem::directory_iterator begin(path); std::filesystem::directory_iterator end; for (auto it = begin; it != end; ++it) - if (std::filesystem::is_directory(*it)) - requireDirectoryMonitor(disk, it->path().filename().string()); + { + const auto & dir_path = it->path(); + if (std::filesystem::is_directory(dir_path)) + { + const auto & tmp_path = dir_path / "tmp"; + + /// "tmp" created by DistributedBlockOutputStream + if (std::filesystem::is_directory(tmp_path) && std::filesystem::is_empty(tmp_path)) + std::filesystem::remove(tmp_path); + + if (std::filesystem::is_empty(dir_path)) + { + LOG_DEBUG(log, "Removing {} (used for async INSERT into Distributed)", dir_path); + /// Will be created by DistributedBlockOutputStream on demand. + std::filesystem::remove(dir_path); + } + else + { + requireDirectoryMonitor(disk, dir_path.filename().string()); + } + } + } } From b2a872b035778cd91ec21d10f94520f9f572eb84 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 5 Nov 2020 23:54:37 +0300 Subject: [PATCH 124/149] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Удалил изменения в своем репозитории в английской версии. --- .../server-configuration-parameters/settings.md | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ad7f92a76f4..31a8e896438 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -471,7 +471,7 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa ## max_concurrent_queries {#max-concurrent-queries} -The maximum number of simultaneously processed queries. +The maximum number of simultaneously processed requests. **Example** @@ -483,18 +483,11 @@ The maximum number of simultaneously processed queries. Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. +Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + Modifying the setting for one query or user does not affect other queries. -Possible values: - -- Positive integer. -- 0 — Unlimited. - -Default value: `0`. - -**Usage** - -`max_concurrent_queries_for_all_users` can be set to 99 for all users, and database administrator can set it to 100 or even 0 for himself to run queries for investigation even when the ClickHouse server is overloaded. +Default value: `0` that means no limit. **Example** From b2e17916e43688aca6f347787d044c48960f78d4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Nov 2020 23:04:34 +0300 Subject: [PATCH 125/149] Mask password in data_path in the system.distribution_queue --- .../test_cluster_with_incorrect_pw.xml | 1 + .../System/StorageSystemDistributionQueue.cpp | 71 ++++++++++++++++++- .../test_cluster_with_incorrect_pw.xml | 14 ++++ tests/config/install.sh | 1 + ...5_system_distribution_queue_mask.reference | 4 ++ .../01555_system_distribution_queue_mask.sql | 36 ++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + 7 files changed, 127 insertions(+), 1 deletion(-) create mode 120000 programs/server/config.d/test_cluster_with_incorrect_pw.xml create mode 100644 tests/config/config.d/test_cluster_with_incorrect_pw.xml create mode 100644 tests/queries/0_stateless/01555_system_distribution_queue_mask.reference create mode 100644 tests/queries/0_stateless/01555_system_distribution_queue_mask.sql diff --git a/programs/server/config.d/test_cluster_with_incorrect_pw.xml b/programs/server/config.d/test_cluster_with_incorrect_pw.xml new file mode 120000 index 00000000000..4e4b334c6d1 --- /dev/null +++ b/programs/server/config.d/test_cluster_with_incorrect_pw.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/test_cluster_with_incorrect_pw.xml \ No newline at end of file diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 39ccea64e26..786bd8bf967 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -10,6 +10,75 @@ #include #include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +} + + +namespace +{ + +using namespace DB; + +/// Drop "password" from the path. +/// +/// In case of use_compact_format_in_distributed_parts_names=0 the path format is: +/// +/// user[:password]@host:port#default_database format +/// +/// And password should be masked out. +/// +/// See: +/// - Cluster::Address::fromFullString() +/// - Cluster::Address::toFullString() +std::string maskDataPath(const std::string & path) +{ + std::string masked_path = path; + + if (!masked_path.ends_with('/')) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid path format"); + + masked_path.pop_back(); + + size_t dir_name_pos = masked_path.rfind('/'); + if (dir_name_pos == std::string::npos) + { + /// Do not include full path into the exception message since it may include password. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid path format"); + } + ++dir_name_pos; + + size_t user_pw_end = masked_path.find('@', dir_name_pos); + if (user_pw_end == std::string::npos) + { + /// Likey new format (use_compact_format_in_distributed_parts_names=1) + return path; + } + + size_t pw_start = masked_path.find(':', dir_name_pos); + if (pw_start > user_pw_end) + { + /// No password in path + return path; + } + ++pw_start; + + size_t pw_length = user_pw_end - pw_start; + /// Replace with a single '*' to hide even the password length. + masked_path.replace(pw_start, pw_length, 1, '*'); + + masked_path.push_back('/'); + + return masked_path; +} + +} namespace DB { @@ -103,7 +172,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, cons size_t col_num = 0; res_columns[col_num++]->insert(database); res_columns[col_num++]->insert(table); - res_columns[col_num++]->insert(status.path); + res_columns[col_num++]->insert(maskDataPath(status.path)); res_columns[col_num++]->insert(status.is_blocked); res_columns[col_num++]->insert(status.error_count); res_columns[col_num++]->insert(status.files_count); diff --git a/tests/config/config.d/test_cluster_with_incorrect_pw.xml b/tests/config/config.d/test_cluster_with_incorrect_pw.xml new file mode 100644 index 00000000000..79d122f7b9b --- /dev/null +++ b/tests/config/config.d/test_cluster_with_incorrect_pw.xml @@ -0,0 +1,14 @@ + + + + + + localhost + 9000 + + foo + + + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index ff96e46c947..f6fae181ac8 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -27,6 +27,7 @@ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference new file mode 100644 index 00000000000..fda87e4d99e --- /dev/null +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference @@ -0,0 +1,4 @@ +masked +2,"default:*@localhost:9000" +no masking +1,"default@localhost:9000" diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql new file mode 100644 index 00000000000..0143b8e46ed --- /dev/null +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.sql @@ -0,0 +1,36 @@ +-- force data path with the user/pass in it +set use_compact_format_in_distributed_parts_names=0; +-- use async send even for localhost +set prefer_localhost_replica=0; + +drop table if exists dist_01555; +drop table if exists data_01555; +create table data_01555 (key Int) Engine=Null(); + +-- +-- masked +-- +SELECT 'masked'; +create table dist_01555 (key Int) Engine=Distributed(test_cluster_with_incorrect_pw, currentDatabase(), data_01555, key); + +insert into dist_01555 values (1)(2); +-- since test_cluster_with_incorrect_pw contains incorrect password ignore error +system flush distributed dist_01555; -- { serverError 516; } +select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1') from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; + +drop table dist_01555; + +-- +-- no masking +-- +SELECT 'no masking'; +create table dist_01555 (key Int) Engine=Distributed(test_shard_localhost, currentDatabase(), data_01555, key); + +insert into dist_01555 values (1)(2); +-- since test_cluster_with_incorrect_pw contains incorrect password ignore error +system flush distributed dist_01555; +select length(splitByChar('*', data_path)), replaceRegexpOne(data_path, '^.*/([^/]*)/' , '\\1') from system.distribution_queue where database = currentDatabase() and table = 'dist_01555' format CSV; + +-- cleanup +drop table dist_01555; +drop table data_01555; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 900cc82b33f..f35f5bde6a9 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -159,4 +159,5 @@ 01547_query_log_current_database 01548_query_log_query_execution_ms 01552_dict_fixedstring +01555_system_distribution_queue_mask 01557_max_parallel_replicas_no_sample.sql From e01b220cd898ca63d5d779de15dfe7596fd1859b Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 6 Nov 2020 00:44:56 +0300 Subject: [PATCH 126/149] Update index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- docs/ru/sql-reference/operators/index.md | 28 +++++++++++++++++++++--- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 3befb18687d..1eddfc4dcaf 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -152,21 +152,43 @@ FROM test.Orders; - `QUARTER` - `YEAR` +В качестве значения оператора `INTERVAL` вы можете также использовать строковый литерал. Например, выражение `INTERVAL 1 HOUR` идентично выражению `INTERVAL '1 hour'` или `INTERVAL '1' hour`. + !!! warning "Внимание" Интервалы различных типов нельзя объединять. Нельзя использовать выражения вида `INTERVAL 4 DAY 1 HOUR`. Вместо этого интервалы можно выразить в единицах меньших или равных наименьшей единице интервала, Например, `INTERVAL 25 HOUR`. Также можно выполнять последовательные операции как показано в примере ниже. -Пример: +Примеры: ``` sql -SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR +SELECT now() AS current_date_time, current_date_time + INTERVAL 4 DAY + INTERVAL 3 HOUR; ``` ``` text ┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ -│ 2019-10-23 11:16:28 │ 2019-10-27 14:16:28 │ +│ 2020-11-03 22:09:50 │ 2020-11-08 01:09:50 │ └─────────────────────┴────────────────────────────────────────────────────────┘ ``` +``` sql +SELECT now() AS current_date_time, current_date_time + INTERVAL '4 day' + INTERVAL '3 hour'; +``` + +``` text +┌───current_date_time─┬─plus(plus(now(), toIntervalDay(4)), toIntervalHour(3))─┐ +│ 2020-11-03 22:12:10 │ 2020-11-08 01:12:10 │ +└─────────────────────┴────────────────────────────────────────────────────────┘ +``` + +``` sql +SELECT now() AS current_date_time, current_date_time + INTERVAL '4' day + INTERVAL '3' hour; +``` + +``` text +┌───current_date_time─┬─plus(plus(now(), toIntervalDay('4')), toIntervalHour('3'))─┐ +│ 2020-11-03 22:33:19 │ 2020-11-08 01:33:19 │ +└─────────────────────┴────────────────────────────────────────────────────────────┘ +``` + **Смотрите также** - Тип данных [Interval](../../sql-reference/operators/index.md) From dc665d7c43f138367cc89c7ce2817c582b4f596f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 6 Nov 2020 13:45:42 +0300 Subject: [PATCH 127/149] Update compare.sh --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7b436833225..92f134011ab 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1096,7 +1096,7 @@ function upload_results --database perftest \ --date_time_input_format=best_effort \ --query " - insert into query_metrics_v1 + insert into query_metrics_v2 select toDate(event_time) event_date, toDateTime('$(cd right/ch && git show -s --format=%ci "$SHA_TO_TEST" | cut -d' ' -f-2)') event_time, From 0d1fcd2f96a86723f5a140e5e43cdfebffcc268b Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Fri, 6 Nov 2020 14:23:04 +0300 Subject: [PATCH 128/149] DOCSUP-3227: Documented the persistent setting (#16579) * added description of disable_set_and_join_persistency setting. * Changed setting name to persistent, added references about persistent to join table engine and set table engine * Resolved conflict. * Added reviewers suggestions * Added translation for Russian. * Fixed typos * Update docs/en/operations/settings/settings.md Fixed minor stuff. Co-authored-by: BayoNet * Update settings.md Updated translation. * Fixed description and translation * Fixed letter cases Co-authored-by: George Co-authored-by: BayoNet --- docs/en/engines/table-engines/special/join.md | 1 + docs/en/engines/table-engines/special/set.md | 6 ++++++ docs/en/operations/settings/settings.md | 13 +++++++++++++ docs/ru/engines/table-engines/special/join.md | 1 + docs/ru/engines/table-engines/special/set.md | 6 ++++++ docs/ru/operations/settings/settings.md | 13 +++++++++++++ 6 files changed, 40 insertions(+) diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 2066abec6a8..30dbec73939 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -98,6 +98,7 @@ When creating a table, the following settings are applied: - [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) - [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) - [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [persistent](../../../operations/settings/settings.md#persistent) The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations. diff --git a/docs/en/engines/table-engines/special/set.md b/docs/en/engines/table-engines/special/set.md index 65fd7376532..a5a7e46af13 100644 --- a/docs/en/engines/table-engines/special/set.md +++ b/docs/en/engines/table-engines/special/set.md @@ -14,4 +14,10 @@ Data is always located in RAM. For `INSERT`, the blocks of inserted data are als For a rough server restart, the block of data on the disk might be lost or damaged. In the latter case, you may need to manually delete the file with damaged data. +### Limitations and Settings {#join-limitations-and-settings} + +When creating a table, the following settings are applied: + +- [persistent](../../../operations/settings/settings.md#persistent) + [Original article](https://clickhouse.tech/docs/en/operations/table_engines/set/) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 359a3579e46..63ff688e51a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2203,4 +2203,17 @@ Possible values: Default value: `0`. +## persistent {#persistent} + +Disables persistency for the [Set](../../engines/table-engines/special/set.md#set) and [Join](../../engines/table-engines/special/join.md#join) table engines. + +Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence. + +Possible values: + +- 1 — Enabled. +- 0 — Disabled. + +Default value: `1`. + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index 65bece4724c..8cb7acd91e1 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -95,6 +95,7 @@ SELECT joinGet('id_val_join', 'val', toUInt32(1)) - [max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) - [join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) - [join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +- [persistent](../../../operations/settings/settings.md#persistent) Таблицы с движком `Join` нельзя использовать в операциях `GLOBAL JOIN`. diff --git a/docs/ru/engines/table-engines/special/set.md b/docs/ru/engines/table-engines/special/set.md index 73fe3c3cf50..14b7f123a34 100644 --- a/docs/ru/engines/table-engines/special/set.md +++ b/docs/ru/engines/table-engines/special/set.md @@ -14,4 +14,10 @@ toc_title: Set При грубом перезапуске сервера, блок данных на диске может быть потерян или повреждён. В последнем случае, может потребоваться вручную удалить файл с повреждёнными данными. +### Ограничения и настройки {#join-limitations-and-settings} + +При создании таблицы, применяются следующие параметры: + +- [persistent](../../../operations/settings/settings.md#persistent) + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/table_engines/set/) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 700203e36e1..39a996cb44e 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2082,4 +2082,17 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); - Функция [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) +## persistent {#persistent} + +Отключает перманентность для табличных движков [Set](../../engines/table-engines/special/set.md#set) и [Join](../../engines/table-engines/special/join.md#join). + +Уменьшает расходы на ввод/вывод. Может быть полезно, когда требуется высокая производительность, а перманентность не обязательна. + +Возможные значения: + +- 1 — включено. +- 0 — отключено. + +Значение по умолчанию: `1`. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From 044f14346354334f93e97705d555c102d7f99b76 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 6 Nov 2020 18:53:33 +0300 Subject: [PATCH 129/149] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index e527e97d608..7b2c2108629 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -216,7 +216,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ if (!jobs.empty()) { - job = jobs.top().job; + job = std::move(jobs.top().job); jobs.pop(); } else From b072bc7d2bfe390c835ff69184e00bf041146470 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 6 Nov 2020 17:54:44 +0100 Subject: [PATCH 130/149] Test for the issue #12615 --- .../format_schemas/social.proto | 6 ++ .../test_storage_kafka/social_pb2.py | 75 +++++++++++++++ tests/integration/test_storage_kafka/test.py | 96 ++++++++++++++++++- 3 files changed, 176 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_kafka/clickhouse_path/format_schemas/social.proto create mode 100644 tests/integration/test_storage_kafka/social_pb2.py diff --git a/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/social.proto b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/social.proto new file mode 100644 index 00000000000..3bf82737fa5 --- /dev/null +++ b/tests/integration/test_storage_kafka/clickhouse_path/format_schemas/social.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message User { + string username = 1; + int32 timestamp = 2; +} \ No newline at end of file diff --git a/tests/integration/test_storage_kafka/social_pb2.py b/tests/integration/test_storage_kafka/social_pb2.py new file mode 100644 index 00000000000..eeba5efc8b1 --- /dev/null +++ b/tests/integration/test_storage_kafka/social_pb2.py @@ -0,0 +1,75 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: social.proto + +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='social.proto', + package='', + syntax='proto3', + serialized_options=None, + serialized_pb=b'\n\x0csocial.proto\"+\n\x04User\x12\x10\n\x08username\x18\x01 \x01(\t\x12\x11\n\ttimestamp\x18\x02 \x01(\x05\x62\x06proto3' +) + + + + +_USER = _descriptor.Descriptor( + name='User', + full_name='User', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='username', full_name='User.username', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=b"".decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='timestamp', full_name='User.timestamp', index=1, + number=2, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=16, + serialized_end=59, +) + +DESCRIPTOR.message_types_by_name['User'] = _USER +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +User = _reflection.GeneratedProtocolMessageType('User', (_message.Message,), { + 'DESCRIPTOR' : _USER, + '__module__' : 'social_pb2' + # @@protoc_insertion_point(class_scope:User) + }) +_sym_db.RegisterMessage(User) + + +# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 6ef37c1e231..5d943361414 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -30,6 +30,8 @@ libprotoc 3.0.0 protoc --python_out=. kafka.proto """ from . import kafka_pb2 +from . import social_pb2 + # TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. # TODO: add test for SELECT LIMIT is working. @@ -115,6 +117,20 @@ def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messag producer.flush() print("Produced {} messages for topic {}".format(num_messages, topic)) +def kafka_produce_protobuf_social(topic, start_index, num_messages): + data = b'' + for i in range(start_index, start_index + num_messages): + msg = social_pb2.User() + msg.username='John Doe {}'.format(i) + msg.timestamp=1000000+i + serialized_msg = msg.SerializeToString() + data = data + _VarintBytes(len(serialized_msg)) + serialized_msg + producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer.send(topic=topic, value=data) + producer.flush() + print(("Produced {} messages for topic {}".format(num_messages, topic))) + + def avro_confluent_message(schema_registry_client, value): # type: (CachedSchemaRegistryClient, dict) -> str @@ -982,6 +998,84 @@ def test_kafka_protobuf(kafka_cluster): kafka_check_result(result, True) +@pytest.mark.timeout(180) +def test_kafka_string_field_on_first_position_in_protobuf(kafka_cluster): +# https://github.com/ClickHouse/ClickHouse/issues/12615 + + instance.query(''' +CREATE TABLE test.kafka ( + username String, + timestamp Int32 + ) ENGINE = Kafka() +SETTINGS + kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'string_field_on_first_position_in_protobuf', + kafka_group_name = 'string_field_on_first_position_in_protobuf', + kafka_format = 'Protobuf', + kafka_schema = 'social:User'; + + SELECT * FROM test.kafka; + ''') + + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + + result = instance.query('SELECT * FROM test.kafka', ignore_error=True) + expected = '''\ +John Doe 0 1000000 +John Doe 1 1000001 +John Doe 2 1000002 +John Doe 3 1000003 +John Doe 4 1000004 +John Doe 5 1000005 +John Doe 6 1000006 +John Doe 7 1000007 +John Doe 8 1000008 +John Doe 9 1000009 +John Doe 10 1000010 +John Doe 11 1000011 +John Doe 12 1000012 +John Doe 13 1000013 +John Doe 14 1000014 +John Doe 15 1000015 +John Doe 16 1000016 +John Doe 17 1000017 +John Doe 18 1000018 +John Doe 19 1000019 +John Doe 20 1000020 +John Doe 21 1000021 +John Doe 22 1000022 +John Doe 23 1000023 +John Doe 24 1000024 +John Doe 25 1000025 +John Doe 26 1000026 +John Doe 27 1000027 +John Doe 28 1000028 +John Doe 29 1000029 +John Doe 30 1000030 +John Doe 31 1000031 +John Doe 32 1000032 +John Doe 33 1000033 +John Doe 34 1000034 +John Doe 35 1000035 +John Doe 36 1000036 +John Doe 37 1000037 +John Doe 38 1000038 +John Doe 39 1000039 +John Doe 40 1000040 +John Doe 41 1000041 +John Doe 42 1000042 +John Doe 43 1000043 +John Doe 44 1000044 +John Doe 45 1000045 +John Doe 46 1000046 +John Doe 47 1000047 +John Doe 48 1000048 +John Doe 49 1000049 +''' + assert TSV(result) == TSV(expected) + @pytest.mark.timeout(30) def test_kafka_protobuf_no_delimiter(kafka_cluster): instance.query(''' @@ -2117,7 +2211,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): kafka_format = 'JSONEachRow', kafka_max_block_size = 20, kafka_flush_interval_ms = 1000; - + SELECT * FROM test.kafka LIMIT 1; /* do subscription & assignment in advance (it can take different time, test rely on timing, so can flap otherwise) */ ''') From ad52fceb83440c977333f3d12aa1a08e7cdb5fdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:34:14 +0300 Subject: [PATCH 131/149] Fix bad code in SmallObjectPool --- src/Common/SmallObjectPool.h | 56 ++++++++++++------------------------ 1 file changed, 19 insertions(+), 37 deletions(-) diff --git a/src/Common/SmallObjectPool.h b/src/Common/SmallObjectPool.h index 469c102bdae..b17b509f14c 100644 --- a/src/Common/SmallObjectPool.h +++ b/src/Common/SmallObjectPool.h @@ -1,76 +1,59 @@ #pragma once #include -#include -#include -#include -#include -#include +#include namespace DB { - /** Can allocate memory objects of fixed size with deletion support. - * For small `object_size`s allocated no less than getMinAllocationSize() bytes. */ + * For small `object_size`s allocated no less than pointer size. + */ class SmallObjectPool { private: - struct Block { Block * next; }; - static constexpr auto getMinAllocationSize() { return sizeof(Block); } - const size_t object_size; Arena pool; - Block * free_list{}; + char * free_list{}; public: - SmallObjectPool( - const size_t object_size_, const size_t initial_size = 4096, const size_t growth_factor = 2, - const size_t linear_growth_threshold = 128 * 1024 * 1024) - : object_size{std::max(object_size_, getMinAllocationSize())}, - pool{initial_size, growth_factor, linear_growth_threshold} + SmallObjectPool(size_t object_size_) + : object_size{std::max(object_size_, sizeof(char *))} { if (pool.size() < object_size) return; - const auto num_objects = pool.size() / object_size; - auto head = free_list = ext::bit_cast(pool.alloc(num_objects * object_size)); + const size_t num_objects = pool.size() / object_size; + free_list = pool.alloc(num_objects * object_size); + char * head = free_list; - for (const auto i : ext::range(0, num_objects - 1)) + for (size_t i = 0; i < num_objects - 1; ++i) { - (void) i; - head->next = ext::bit_cast(ext::bit_cast(head) + object_size); - head = head->next; + char * next = head + object_size; + unalignedStore(head, next); + head = next; } - head->next = nullptr; + unalignedStore(head, nullptr); } char * alloc() { if (free_list) { - const auto res = reinterpret_cast(free_list); - free_list = free_list->next; + char * res = free_list; + free_list = unalignedLoad(free_list); return res; } return pool.alloc(object_size); } - void free(const void * ptr) + void free(char * ptr) { - union - { - const void * p_v; - Block * block; - }; - - p_v = ptr; - block->next = free_list; - - free_list = block; + unalignedStore(ptr, free_list); + free_list = ptr; } /// The size of the allocated pool in bytes @@ -81,5 +64,4 @@ public: }; - } From c4398952da66d15eb7086bb2419605d6f9ef9e1e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:40:11 +0300 Subject: [PATCH 132/149] Addition to prev. revision --- src/Dictionaries/ComplexKeyCacheDictionary.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Dictionaries/ComplexKeyCacheDictionary.h b/src/Dictionaries/ComplexKeyCacheDictionary.h index e055d1a6b33..2663fee266d 100644 --- a/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -16,6 +16,8 @@ #include #include #include +#include +#include #include #include "DictionaryStructure.h" #include "IDictionary.h" From 57de99bcd994a317b53f4e2c5598bc9c2134a8f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:46:43 +0300 Subject: [PATCH 133/149] More simple --- src/Common/SmallObjectPool.h | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/src/Common/SmallObjectPool.h b/src/Common/SmallObjectPool.h index b17b509f14c..0eda8931946 100644 --- a/src/Common/SmallObjectPool.h +++ b/src/Common/SmallObjectPool.h @@ -15,27 +15,12 @@ class SmallObjectPool private: const size_t object_size; Arena pool; - char * free_list{}; + char * free_list = nullptr; public: SmallObjectPool(size_t object_size_) : object_size{std::max(object_size_, sizeof(char *))} { - if (pool.size() < object_size) - return; - - const size_t num_objects = pool.size() / object_size; - free_list = pool.alloc(num_objects * object_size); - char * head = free_list; - - for (size_t i = 0; i < num_objects - 1; ++i) - { - char * next = head + object_size; - unalignedStore(head, next); - head = next; - } - - unalignedStore(head, nullptr); } char * alloc() From 584f1a83be5288d0f474013633a03de9d1903f44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:52:28 +0300 Subject: [PATCH 134/149] Do not instrument 3rd-party libraries with UBSan --- contrib/CMakeLists.txt | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7d6b9c0e374..d15715c7b8f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -14,6 +14,11 @@ unset (_current_dir_name) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") +if (SANITIZE STREQUAL "undefined") + # 3rd-party libraries usually not intended to work with UBSan. + add_compile_options(-fno-sanitize=undefined) +endif() + set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1) add_subdirectory (boost-cmake) @@ -157,9 +162,6 @@ if(USE_INTERNAL_SNAPPY_LIBRARY) add_subdirectory(snappy) set (SNAPPY_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/snappy") - if(SANITIZE STREQUAL "undefined") - target_compile_options(${SNAPPY_LIBRARY} PRIVATE -fno-sanitize=undefined) - endif() endif() if (USE_INTERNAL_PARQUET_LIBRARY) From 8ec6653fb13762639cf8862f2f5505b7ac7ef81b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:56:21 +0300 Subject: [PATCH 135/149] Fix UBSan report in Poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 757d947235b..f49c6ab8d3a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 757d947235b307675cff964f29b19d388140a9eb +Subproject commit f49c6ab8d3aa71828bd1b411485c21722e8c9d82 From 17b3dff0c2745e788019143bf7fd134c54555bb1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 20:58:04 +0300 Subject: [PATCH 136/149] Whitespaces --- src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h | 2 ++ src/Processors/Sources/SinkToOutputStream.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h index 054aec94464..531b2636747 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.h @@ -1,9 +1,11 @@ #pragma once + #include #include #include #include + namespace DB { diff --git a/src/Processors/Sources/SinkToOutputStream.cpp b/src/Processors/Sources/SinkToOutputStream.cpp index 9727b637d8b..7612ba10fb3 100644 --- a/src/Processors/Sources/SinkToOutputStream.cpp +++ b/src/Processors/Sources/SinkToOutputStream.cpp @@ -1,6 +1,7 @@ #include #include + namespace DB { From 28c6dea1a20d9c96ecb7ead1a09b30fd43d8719b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 21:02:16 +0300 Subject: [PATCH 137/149] LOGICAL_ERROR was mistakenly used for some ErrorCodes #12735 --- src/Interpreters/Context.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index befb097faf7..b909ce383a3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -65,6 +65,7 @@ #include #include + namespace ProfileEvents { extern const Event ContextLock; @@ -153,7 +154,7 @@ public: } else if (it->second->key.first != context.client_info.current_user) { - throw Exception("Session belongs to a different user", ErrorCodes::LOGICAL_ERROR); + throw Exception("Session belongs to a different user", ErrorCodes::SESSION_IS_LOCKED); } /// Use existing session. @@ -596,7 +597,8 @@ VolumePtr Context::setTemporaryStorage(const String & path, const String & polic { StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name); if (tmp_policy->getVolumes().size() != 1) - throw Exception("Policy " + policy_name + " is used temporary files, such policy should have exactly one volume", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + throw Exception("Policy " + policy_name + " is used temporary files, such policy should have exactly one volume", + ErrorCodes::NO_ELEMENTS_IN_CONFIG); shared->tmp_volume = tmp_policy->getVolume(0); } @@ -1083,11 +1085,13 @@ String Context::getInitialQueryId() const void Context::setCurrentDatabaseNameInGlobalContext(const String & name) { if (global_context != this) - throw Exception("Cannot set current database for non global context, this method should be used during server initialization", ErrorCodes::LOGICAL_ERROR); + throw Exception("Cannot set current database for non global context, this method should be used during server initialization", + ErrorCodes::LOGICAL_ERROR); auto lock = getLock(); if (!current_database.empty()) - throw Exception("Default database name cannot be changed in global context without server restart", ErrorCodes::LOGICAL_ERROR); + throw Exception("Default database name cannot be changed in global context without server restart", + ErrorCodes::LOGICAL_ERROR); current_database = name; } @@ -1470,7 +1474,7 @@ DDLWorker & Context::getDDLWorker() const { auto lock = getLock(); if (!shared->ddl_worker) - throw Exception("DDL background thread is not initialized.", ErrorCodes::LOGICAL_ERROR); + throw Exception("DDL background thread is not initialized.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); return *shared->ddl_worker; } From ee46993359b2a9fb62443b6f35951c86e73f726a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Nov 2020 23:37:17 +0300 Subject: [PATCH 138/149] Enable use_compact_format_in_distributed_parts_names by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4d4712dcba7..b099b9c6ab5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -376,7 +376,7 @@ class IColumn; M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ - M(Bool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ + M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ From f379f9837d05d475433d8d9e634cec243ff4613f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Nov 2020 23:58:06 +0300 Subject: [PATCH 139/149] Document use_compact_format_in_distributed_parts_names setting Since it has been enabled by default, at least it should be documented. --- docs/en/operations/settings/settings.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 63ff688e51a..d83f7d6c219 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1765,6 +1765,23 @@ Default value: `0`. - [Distributed Table Engine](../../engines/table-engines/special/distributed.md#distributed) - [Managing Distributed Tables](../../sql-reference/statements/system.md#query-language-system-distributed) + + +## use_compact_format_in_distributed_parts_names {#use_compact_format_in_distributed_parts_names} + +Uses compact format for storing blocks for async (`insert_distributed_sync`) INSERT into tables with `Distributed` engine. + +Possible values: + +- 0 — Uses `user[:password]@host:port#default_database` directory format. +- 1 — Uses `[shard{shard_index}[_replica{replica_index}]]` directory format. + +Default value: `1`. + +!!! note "Note" + - with `use_compact_format_in_distributed_parts_names=0` changes from cluster definition will not be applied for async INSERT. + - with `use_compact_format_in_distributed_parts_names=1` changing the order of the nodes in the cluster definition, will change the `shard_index`/`replica_index` so be aware. + ## background_buffer_flush_schedule_pool_size {#background_buffer_flush_schedule_pool_size} Sets the number of threads performing background flush in [Buffer](../../engines/table-engines/special/buffer.md)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. From ac8df08186ac940d97cd0413649872ef6abecac3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Nov 2020 08:34:37 +0300 Subject: [PATCH 140/149] Set use_compact_format_in_distributed_parts_names=0 for some integration tests To avoid depends from default. --- tests/integration/test_distributed_format/test.py | 14 +++++++++----- .../test_distributed_storage_configuration/test.py | 8 ++++++-- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 607154e37f1..22054077544 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -47,10 +47,12 @@ def test_single_file(started_cluster, cluster): def test_two_files(started_cluster, cluster): node.query( "create table test.distr_2 (x UInt64, s String) engine = Distributed('{}', database, table)".format(cluster)) - node.query("insert into test.distr_2 values (0, '_'), (1, 'a')", - settings={"use_compact_format_in_distributed_parts_names": "1"}) - node.query("insert into test.distr_2 values (2, 'bb'), (3, 'ccc')", - settings={"use_compact_format_in_distributed_parts_names": "1"}) + node.query("insert into test.distr_2 values (0, '_'), (1, 'a')", settings={ + "use_compact_format_in_distributed_parts_names": "1", + }) + node.query("insert into test.distr_2 values (2, 'bb'), (3, 'ccc')", settings={ + "use_compact_format_in_distributed_parts_names": "1", + }) query = "select * from file('/var/lib/clickhouse/data/test/distr_2/shard1_replica1/{1,2,3,4}.bin', 'Distributed') order by x" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) @@ -70,7 +72,9 @@ def test_two_files(started_cluster, cluster): def test_single_file_old(started_cluster, cluster): node.query( "create table test.distr_3 (x UInt64, s String) engine = Distributed('{}', database, table)".format(cluster)) - node.query("insert into test.distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')") + node.query("insert into test.distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')", settings={ + "use_compact_format_in_distributed_parts_names": "0", + }) query = "select * from file('/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin', 'Distributed')" out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query]) diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index d293b96399d..976fc5211a7 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -48,7 +48,9 @@ def test_insert(start_cluster): # manual only (but only for remote node) node.query('SYSTEM STOP DISTRIBUTED SENDS test.dist_foo') - node.query('INSERT INTO test.dist_foo SELECT * FROM numbers(100)') + node.query('INSERT INTO test.dist_foo SELECT * FROM numbers(100)', settings={ + 'use_compact_format_in_distributed_parts_names': '0', + }) assert _files_in_dist_mon(node, 'disk1', 'dist_foo') == 1 assert _files_in_dist_mon(node, 'disk2', 'dist_foo') == 0 @@ -61,7 +63,9 @@ def test_insert(start_cluster): # node.query('RENAME TABLE test.dist_foo TO test.dist2_foo') - node.query('INSERT INTO test.dist2_foo SELECT * FROM numbers(100)') + node.query('INSERT INTO test.dist2_foo SELECT * FROM numbers(100)', settings={ + 'use_compact_format_in_distributed_parts_names': '0', + }) assert _files_in_dist_mon(node, 'disk1', 'dist2_foo') == 0 assert _files_in_dist_mon(node, 'disk2', 'dist2_foo') == 1 From c057e86040f9ffc4f972e29558eff6c2c10ae5ea Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 6 Nov 2020 21:04:50 +0300 Subject: [PATCH 141/149] Fix path masking in system.distribution_queue for multiple nodes --- .../System/StorageSystemDistributionQueue.cpp | 46 ++++++++++--------- .../test_cluster_with_incorrect_pw.xml | 9 +++- ...5_system_distribution_queue_mask.reference | 2 +- 3 files changed, 33 insertions(+), 24 deletions(-) diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 786bd8bf967..c8d8c88ec08 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -46,32 +46,34 @@ std::string maskDataPath(const std::string & path) masked_path.pop_back(); - size_t dir_name_pos = masked_path.rfind('/'); - if (dir_name_pos == std::string::npos) + size_t node_pos = masked_path.rfind('/'); + /// Loop through each node, that separated with a comma + while (node_pos != std::string::npos) { - /// Do not include full path into the exception message since it may include password. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid path format"); - } - ++dir_name_pos; + ++node_pos; - size_t user_pw_end = masked_path.find('@', dir_name_pos); - if (user_pw_end == std::string::npos) - { - /// Likey new format (use_compact_format_in_distributed_parts_names=1) - return path; - } + size_t user_pw_end = masked_path.find('@', node_pos); + if (user_pw_end == std::string::npos) + { + /// Likey new format (use_compact_format_in_distributed_parts_names=1) + return path; + } - size_t pw_start = masked_path.find(':', dir_name_pos); - if (pw_start > user_pw_end) - { - /// No password in path - return path; - } - ++pw_start; + size_t pw_start = masked_path.find(':', node_pos); + if (pw_start > user_pw_end) + { + /// No password in path + return path; + } + ++pw_start; - size_t pw_length = user_pw_end - pw_start; - /// Replace with a single '*' to hide even the password length. - masked_path.replace(pw_start, pw_length, 1, '*'); + size_t pw_length = user_pw_end - pw_start; + /// Replace with a single '*' to hide even the password length. + masked_path.replace(pw_start, pw_length, 1, '*'); + + /// "," cannot be in the node specification since it will be encoded in hex. + node_pos = masked_path.find(',', node_pos); + } masked_path.push_back('/'); diff --git a/tests/config/config.d/test_cluster_with_incorrect_pw.xml b/tests/config/config.d/test_cluster_with_incorrect_pw.xml index 79d122f7b9b..109e35afc37 100644 --- a/tests/config/config.d/test_cluster_with_incorrect_pw.xml +++ b/tests/config/config.d/test_cluster_with_incorrect_pw.xml @@ -2,8 +2,15 @@ + true - localhost + 127.0.0.1 + 9000 + + foo + + + 127.0.0.2 9000 foo diff --git a/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference index fda87e4d99e..bd0eac10816 100644 --- a/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference +++ b/tests/queries/0_stateless/01555_system_distribution_queue_mask.reference @@ -1,4 +1,4 @@ masked -2,"default:*@localhost:9000" +3,"default:*@127%2E0%2E0%2E1:9000,default:*@127%2E0%2E0%2E2:9000" no masking 1,"default@localhost:9000" From ab83170bac1a0c898b9a8b40105876be0909733c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 21:44:49 +0300 Subject: [PATCH 142/149] Remove a test --- .../0_stateless/01532_client_sni.reference | 1 - tests/queries/0_stateless/01532_client_sni.sh | 21 ------------------- 2 files changed, 22 deletions(-) delete mode 100644 tests/queries/0_stateless/01532_client_sni.reference delete mode 100755 tests/queries/0_stateless/01532_client_sni.sh diff --git a/tests/queries/0_stateless/01532_client_sni.reference b/tests/queries/0_stateless/01532_client_sni.reference deleted file mode 100644 index 879b5791c7b..00000000000 --- a/tests/queries/0_stateless/01532_client_sni.reference +++ /dev/null @@ -1 +0,0 @@ -0000000e000c0000096c6f63616c686f7374 diff --git a/tests/queries/0_stateless/01532_client_sni.sh b/tests/queries/0_stateless/01532_client_sni.sh deleted file mode 100755 index 0b122713fde..00000000000 --- a/tests/queries/0_stateless/01532_client_sni.sh +++ /dev/null @@ -1,21 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - -# Check that ClickHouse properly uses SNI extension in Client Hello packet in HTTPS connection. - -nc -q0 -l 5678 | xxd -p | grep -oF $'0000000e000c0000096c6f63616c686f7374' & - -${CLICKHOUSE_LOCAL} --query "SELECT * FROM url('https://localhost:5678/', RawBLOB, 'data String')" 2>&1 | grep -v -F 'Timeout' - -# grep -oF '\x00\x00\x00\x0e\x00\x0c\x00\x00\x09\x6c\x6f\x63\x61\x6c\x68\x6f\x73\x74' -# ^^^^^^^^ ^^^^^^^ ^^^^^^^ ^^ ^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -# | | | | | -# server name data | hostname | l o c a l h o s t -# extension id len: 14 | type | -# | | -# hostnames list hostname -# len, 12 len, 9 - -wait From 85ecc9ea104eb76581d4b3426a7609c3882ef8ad Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 6 Nov 2020 22:11:57 +0300 Subject: [PATCH 143/149] Update ASTColumnDeclaration.cpp --- src/Parsers/ASTColumnDeclaration.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 0e0847713c2..27ece3e18c2 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -56,10 +56,10 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta { settings.ostr << ' '; - FormatStateStacked typeFrame = frame; - typeFrame.indent = 0; + FormatStateStacked type_frame = frame; + type_frame.indent = 0; - type->formatImpl(settings, state, typeFrame); + type->formatImpl(settings, state, type_frame); } if (null_modifier) From 7b7289436306088e2b5213dd29b00d86037a6925 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 6 Nov 2020 23:23:08 +0300 Subject: [PATCH 144/149] Simplified code --- src/IO/ReadBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index acd0414ef5d..5c66c3209f6 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -67,7 +67,7 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) { - return offset() != buffer().size() || socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); + return available() || socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); } } From 381ed50befcce99ab0f82dd93aec87e293dde298 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Sat, 7 Nov 2020 01:39:22 +0100 Subject: [PATCH 145/149] Enabling existing testflows tests. --- tests/testflows/rbac/tests/privileges/feature.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/testflows/rbac/tests/privileges/feature.py b/tests/testflows/rbac/tests/privileges/feature.py index 81af29a70e1..bc63824d322 100755 --- a/tests/testflows/rbac/tests/privileges/feature.py +++ b/tests/testflows/rbac/tests/privileges/feature.py @@ -7,7 +7,7 @@ from rbac.helper.common import * def feature(self): tasks = [] - pool = Pool(10) + pool = Pool(16) try: try: @@ -21,6 +21,12 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_constraint", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_ttl", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_settings", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_update", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_delete", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_freeze", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_fetch", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_move", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.grant_option", "feature"), flags=TE), {}) finally: join(tasks) finally: From d78d07f825c9001b7877b119e1832478b21da512 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Sat, 7 Nov 2020 01:48:29 +0100 Subject: [PATCH 146/149] xfails for ALTER MOVE --- tests/testflows/rbac/regression.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/testflows/rbac/regression.py b/tests/testflows/rbac/regression.py index c0307664061..a53de0178eb 100755 --- a/tests/testflows/rbac/regression.py +++ b/tests/testflows/rbac/regression.py @@ -19,6 +19,7 @@ issue_14674 = "https://github.com/ClickHouse/ClickHouse/issues/14674" issue_14810 = "https://github.com/ClickHouse/ClickHouse/issues/14810" issue_15165 = "https://github.com/ClickHouse/ClickHouse/issues/15165" issue_15980 = "https://github.com/ClickHouse/ClickHouse/issues/15980" +issue_16403 = "https://github.com/ClickHouse/ClickHouse/issues/16403" xfails = { "syntax/show create quota/I show create quota current": @@ -89,6 +90,12 @@ xfails = { [(Fail, ".inner table is not created as expected")], "views/materialized view/select from source table privilege granted directly or via role/select from implicit target table, privilege granted through a role": [(Fail, ".inner table is not created as expected")], + "privileges/alter move/:/:/:/:/move partition to implicit target table of a materialized view": + [(Fail, ".inner table is not created as expected")], + "privileges/alter move/:/:/:/:/user without ALTER MOVE PARTITION privilege/": + [(Fail, issue_16403)], + "privileges/alter move/:/:/:/:/user with revoked ALTER MOVE PARTITION privilege/": + [(Fail, issue_16403)], } xflags = { From 85d1f37dde05a24367a769e84839c52b14bee7d6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Nov 2020 13:27:56 +0300 Subject: [PATCH 147/149] Update sumwithoverflow.md --- .../aggregate-functions/reference/sumwithoverflow.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md b/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md index fa603b4b155..1b39e9d0eb1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sumwithoverflow.md @@ -4,6 +4,6 @@ toc_priority: 140 # sumWithOverflow {#sumwithoverflowx} -Computes the sum of the numbers, using the same data type for the result as for the input parameters. If the sum exceeds the maximum value for this data type, the function returns an error. +Computes the sum of the numbers, using the same data type for the result as for the input parameters. If the sum exceeds the maximum value for this data type, it is calculated with overflow. Only works for numbers. From ca7ef4fb90cc61e21ab20e40b60e75031a1e5b81 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Nov 2020 13:50:23 +0300 Subject: [PATCH 148/149] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index d08e7833b33..29d4dc9ba48 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -68,6 +68,7 @@ toc_title: Adopters | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | | OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | | Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | +| Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | | Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | | PostHog | Product Analytics | Main Product | — | — | [Release Notes, Oct 2020](https://posthog.com/blog/the-posthog-array-1-15-0) | | Postmates | Delivery | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=188) | From c2e6ddddbd7cb35122576a25513e11827c3cdf20 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 7 Nov 2020 13:54:36 +0300 Subject: [PATCH 149/149] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 29d4dc9ba48..89b64bfecb8 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -45,6 +45,7 @@ toc_title: Adopters | FunCorp | Games | | — | — | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | +| ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | | Infovista | Networks | Analytics | — | — | [Slides in English, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) | | InnoGames | Games | Metrics, Logging | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) |