ClickHouse/src/Storages/MergeTree/MergeList.h

207 lines
6.1 KiB
C++
Raw Normal View History

2014-09-10 11:34:26 +00:00
#pragma once
#include <Core/Names.h>
#include <Core/Field.h>
#include <Common/Stopwatch.h>
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
2020-09-03 13:00:13 +00:00
#include <Storages/MergeTree/MergeType.h>
#include <Storages/MergeTree/MergeAlgorithm.h>
2021-06-24 14:07:43 +00:00
#include <Storages/MergeTree/MergeTreePartInfo.h>
#include <Storages/MergeTree/BackgroundProcessList.h>
#include <Interpreters/StorageID.h>
#include <boost/noncopyable.hpp>
2015-02-10 21:10:58 +00:00
#include <memory>
2014-09-10 11:34:26 +00:00
#include <list>
#include <mutex>
#include <atomic>
namespace CurrentMetrics
{
extern const Metric Merge;
}
2014-09-10 11:34:26 +00:00
namespace DB
{
2015-04-16 06:12:35 +00:00
struct MergeInfo
{
std::string database;
std::string table;
std::string result_part_name;
std::string result_part_path;
Array source_part_names;
Array source_part_paths;
2018-09-11 11:16:40 +00:00
std::string partition_id;
bool is_mutation;
Float64 elapsed;
Float64 progress;
UInt64 num_parts;
UInt64 total_size_bytes_compressed;
UInt64 total_size_marks;
2019-03-26 12:37:42 +00:00
UInt64 total_rows_count;
UInt64 bytes_read_uncompressed;
UInt64 bytes_written_uncompressed;
UInt64 rows_read;
UInt64 rows_written;
UInt64 columns_written;
UInt64 memory_usage;
UInt64 thread_id;
2020-09-03 13:00:13 +00:00
std::string merge_type;
2020-09-10 14:56:15 +00:00
std::string merge_algorithm;
};
struct FutureMergedMutatedPart;
using FutureMergedMutatedPartPtr = std::shared_ptr<FutureMergedMutatedPart>;
struct MergeListElement;
using MergeListEntry = BackgroundProcessListEntry<MergeListElement, MergeInfo>;
/**
* Since merge is executed with multiple threads, this class
* switches the parent MemoryTracker to account all the memory used.
*/
class MemoryTrackerThreadSwitcher : boost::noncopyable
{
public:
explicit MemoryTrackerThreadSwitcher(MergeListEntry & merge_list_entry_);
~MemoryTrackerThreadSwitcher();
private:
MergeListEntry & merge_list_entry;
MemoryTracker * background_thread_memory_tracker;
MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr;
UInt64 prev_untracked_memory_limit;
UInt64 prev_untracked_memory;
String prev_query_id;
};
using MemoryTrackerThreadSwitcherPtr = std::unique_ptr<MemoryTrackerThreadSwitcher>;
struct MergeListElement : boost::noncopyable
2014-09-10 11:34:26 +00:00
{
const StorageID table_id;
2018-09-11 11:16:40 +00:00
std::string partition_id;
const std::string result_part_name;
const std::string result_part_path;
2021-06-24 14:07:43 +00:00
MergeTreePartInfo result_part_info;
bool is_mutation{};
UInt64 num_parts{};
Names source_part_names;
Names source_part_paths;
Int64 source_data_version{};
Stopwatch watch;
std::atomic<Float64> progress{};
std::atomic<bool> is_cancelled{};
UInt64 total_size_bytes_compressed{};
UInt64 total_size_marks{};
2019-03-26 12:37:42 +00:00
UInt64 total_rows_count{};
std::atomic<UInt64> bytes_read_uncompressed{};
std::atomic<UInt64> bytes_written_uncompressed{};
/// In case of Vertical algorithm they are actual only for primary key columns
std::atomic<UInt64> rows_read{};
std::atomic<UInt64> rows_written{};
/// Updated only for Vertical algorithm
std::atomic<UInt64> columns_written{};
MemoryTracker memory_tracker{VariableContext::Process};
/// Used to adjust ThreadStatus::untracked_memory_limit
UInt64 max_untracked_memory;
/// Used to avoid losing any allocation context
UInt64 untracked_memory = 0;
/// Used for identifying mutations/merges in trace_log
std::string query_id;
2016-07-31 03:53:16 +00:00
UInt64 thread_id;
2020-09-03 13:00:13 +00:00
MergeType merge_type;
/// Detected after merge already started
std::atomic<MergeAlgorithm> merge_algorithm;
MergeListElement(
const StorageID & table_id_,
FutureMergedMutatedPartPtr future_part,
UInt64 memory_profiler_step,
UInt64 memory_profiler_sample_probability,
UInt64 max_untracked_memory_);
MergeInfo getInfo() const;
2021-09-24 13:57:44 +00:00
MergeListElement * ptr() { return this; }
~MergeListElement();
2021-11-05 11:55:30 +00:00
MergeListElement & ref() { return *this; }
2015-04-16 06:12:35 +00:00
};
2014-09-10 11:34:26 +00:00
/** Maintains a list of currently running merges.
* For implementation of system.merges table.
*/
class MergeList final : public BackgroundProcessList<MergeListElement, MergeInfo>
2015-04-16 06:12:35 +00:00
{
2020-10-26 16:38:35 +00:00
private:
using Parent = BackgroundProcessList<MergeListElement, MergeInfo>;
2020-09-04 06:55:19 +00:00
std::atomic<size_t> merges_with_ttl_counter = 0;
2015-04-16 06:12:35 +00:00
public:
MergeList()
: Parent(CurrentMetrics::Merge)
{}
void onEntryDestroy(const Parent::Entry & entry) override
{
if (isTTLMergeType(entry->merge_type))
--merges_with_ttl_counter;
}
void cancelPartMutations(const StorageID & table_id, const String & partition_id, Int64 mutation_version)
{
std::lock_guard lock{mutex};
for (auto & merge_element : entries)
{
if ((partition_id.empty() || merge_element.partition_id == partition_id)
&& merge_element.table_id == table_id
&& merge_element.source_data_version < mutation_version
2021-06-24 14:07:43 +00:00
&& merge_element.result_part_info.getDataVersion() >= mutation_version)
merge_element.is_cancelled = true;
}
}
void cancelInPartition(const StorageID & table_id, const String & partition_id, Int64 delimiting_block_number)
{
2021-06-30 12:55:26 +00:00
std::lock_guard lock{mutex};
2021-06-24 14:07:43 +00:00
for (auto & merge_element : entries)
{
if (merge_element.table_id == table_id
&& merge_element.partition_id == partition_id
&& merge_element.result_part_info.min_block < delimiting_block_number)
merge_element.is_cancelled = true;
}
}
2020-09-04 06:55:19 +00:00
/// Merge consists of two parts: assignment and execution. We add merge to
/// merge list on execution, but checking merge list during merge
/// assignment. This lead to the logical race condition (we can assign more
/// merges with TTL than allowed). So we "book" merge with ttl during
/// assignment, and remove from list after merge execution.
///
/// NOTE: Not important for replicated merge tree, we check count of merges twice:
/// in assignment and in queue before execution.
void bookMergeWithTTL()
{
++merges_with_ttl_counter;
}
size_t getMergesWithTTLCount() const
2020-09-04 06:55:19 +00:00
{
return merges_with_ttl_counter;
}
2014-09-10 11:34:26 +00:00
};
}