mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-05 14:02:21 +00:00
parent
f64344f7c5
commit
cb7db6d232
@ -622,27 +622,17 @@ void StorageMergeTree::loadMutations()
|
||||
increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first);
|
||||
}
|
||||
|
||||
|
||||
bool StorageMergeTree::merge(
|
||||
bool aggressive,
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
bool deduplicate,
|
||||
String * out_disable_reason)
|
||||
std::optional<StorageMergeTree::MergeMutateSelectedEntry> 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);
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
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`.
|
||||
std::optional<CurrentlyMergingPartsTagger> merging_tagger;
|
||||
CurrentlyMergingPartsTaggerPtr 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.
|
||||
@ -717,14 +707,31 @@ bool StorageMergeTree::merge(
|
||||
}
|
||||
*out_disable_reason += "Cannot select parts for optimization";
|
||||
}
|
||||
return false;
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
merging_tagger.emplace(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false);
|
||||
merging_tagger = std::make_unique<CurrentlyMergingPartsTagger>(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,
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
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
|
||||
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,22 +790,18 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::movePartsTask()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool StorageMergeTree::tryMutatePart()
|
||||
std::optional<StorageMergeTree::MergeMutateSelectedEntry> StorageMergeTree::selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String */* disable_reason */)
|
||||
{
|
||||
auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations);
|
||||
StorageMetadataPtr metadata_snapshot = getInMemoryMetadataPtr();
|
||||
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;
|
||||
/// You must call destructor with unlocked `currently_processing_in_background_mutex`.
|
||||
std::optional<CurrentlyMergingPartsTagger> tagger;
|
||||
{
|
||||
std::lock_guard lock(currently_processing_in_background_mutex);
|
||||
|
||||
CurrentlyMergingPartsTaggerPtr tagger;
|
||||
|
||||
if (current_mutations_by_version.empty())
|
||||
return false;
|
||||
return {};
|
||||
|
||||
auto mutations_end_it = current_mutations_by_version.end();
|
||||
for (const auto & part : getDataPartsVector())
|
||||
@ -861,17 +864,24 @@ bool StorageMergeTree::tryMutatePart()
|
||||
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)
|
||||
return false;
|
||||
|
||||
tagger = std::make_unique<CurrentlyMergingPartsTagger>(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();
|
||||
|
||||
auto merge_mutate_entry = selectPartsToMutate(metadata_snapshot, nullptr);
|
||||
if (!merge_mutate_entry)
|
||||
return false;
|
||||
|
||||
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);
|
||||
|
||||
|
@ -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<StorageMergeTree>, 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<CurrentlyMergingPartsTagger>;
|
||||
|
||||
struct MergeMutateSelectedEntry
|
||||
{
|
||||
FutureMergedMutatedPart future_part;
|
||||
CurrentlyMergingPartsTaggerPtr tagger;
|
||||
MergeList::EntryPtr merge_entry;
|
||||
MutationCommands commands;
|
||||
};
|
||||
|
||||
std::optional<MergeMutateSelectedEntry> selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason);
|
||||
std::optional<MergeMutateSelectedEntry> 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:
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
Loading…
Reference in New Issue
Block a user