ClickHouse/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

83 lines
3.3 KiB
C++
Raw Normal View History

2020-09-18 10:57:33 +00:00
#pragma once
2021-10-02 07:13:14 +00:00
#include <base/types.h>
2020-09-18 10:57:33 +00:00
#include <optional>
#include <mutex>
#include <vector>
#include <atomic>
#include <boost/noncopyable.hpp>
namespace DB
{
class StorageReplicatedMergeTree;
struct ReplicatedMergeTreeLogEntryData;
/// In some use cases merging can be more expensive than fetching
/// (so instead of doing exactly the same merge cluster-wise you can do merge once and fetch ready part)
/// Fetches may be desirable for other operational reasons (backup replica without lot of CPU resources).
///
2022-12-25 13:18:05 +00:00
/// That class allow to take a decisions about preferred strategy for a concrete merge.
2020-09-18 10:57:33 +00:00
///
/// Since that code is used in shouldExecuteLogEntry we need to be able to:
/// 1) make decision fast
/// 2) avoid excessive zookeeper operations
///
/// Because of that we need to cache some important things,
/// like list of active replicas (to limit the number of zookeeper operations)
///
/// That means we need to refresh the state of that object regularly
2020-11-03 15:58:17 +00:00
///
/// NOTE: This class currently supports only single feature (execute_merges_on_single_replica_time_threshold),
/// may be extended to postpone merges in some other scenarios, namely
/// * always_fetch_merged_part
/// * try_fetch_recompressed_part_timeout
/// * (maybe, not for postpone) prefer_fetch_merged_part_time_threshold
///
/// NOTE: execute_merges_on_single_replica_time_threshold feature doesn't provide any strict guarantees.
/// When some replicas are added / removed we may execute some merges on more than one replica,
/// or not execute merge on any of replicas during execute_merges_on_single_replica_time_threshold interval.
/// (so it may be a bad idea to set that threshold to high values).
///
2020-09-18 10:57:33 +00:00
class ReplicatedMergeTreeMergeStrategyPicker: public boost::noncopyable
{
public:
explicit ReplicatedMergeTreeMergeStrategyPicker(StorageReplicatedMergeTree & storage_);
2020-09-18 10:57:33 +00:00
/// triggers refreshing the cached state (list of replicas etc.)
/// used when we get new merge event from the zookeeper queue ( see queueUpdatingTask() etc )
void refreshState();
2020-11-03 15:58:17 +00:00
/// return true if execute_merges_on_single_replica_time_threshold feature is active
/// and we may need to do a fetch (or postpone) instead of merge
2022-02-10 19:45:52 +00:00
bool shouldMergeOnSingleReplica(const ReplicatedMergeTreeLogEntryData & entry) const;
2021-06-24 08:25:05 +00:00
2020-11-03 15:58:17 +00:00
/// returns the replica name
2020-09-18 10:57:33 +00:00
/// and it's not current replica should do the merge
2022-02-10 19:45:52 +00:00
std::optional<String> pickReplicaToExecuteMerge(const ReplicatedMergeTreeLogEntryData & entry);
2020-09-18 10:57:33 +00:00
2022-02-10 19:45:52 +00:00
/// checks (in zookeeper) if the picked replica finished the merge
bool isMergeFinishedByReplica(const String & replica, const ReplicatedMergeTreeLogEntryData & entry);
2022-02-09 19:56:22 +00:00
2020-09-18 10:57:33 +00:00
private:
StorageReplicatedMergeTree & storage;
/// calculate entry hash based on zookeeper path and new part name
2020-11-03 15:58:17 +00:00
/// ATTENTION: it's not a general-purpose hash, it just allows to select replicas consistently
2020-09-18 10:57:33 +00:00
uint64_t getEntryHash(const ReplicatedMergeTreeLogEntryData & entry) const;
std::atomic<time_t> execute_merges_on_single_replica_time_threshold = 0;
2021-07-05 03:32:56 +00:00
std::atomic<time_t> remote_fs_execute_merges_on_single_replica_time_threshold = 0;
2020-09-18 10:57:33 +00:00
std::atomic<time_t> last_refresh_time = 0;
std::mutex mutex;
2020-11-03 15:58:17 +00:00
/// those 2 members accessed under the mutex, only when
/// execute_merges_on_single_replica_time_threshold enabled
2020-09-18 10:57:33 +00:00
int current_replica_index = -1;
std::vector<String> active_replicas;
};
}