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:
|
2022-03-13 12:23:51 +00:00
|
|
|
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;
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|