ClickHouse/dbms/include/DB/Storages/MergeTree/ReshardingWorker.h

396 lines
14 KiB
C
Raw Normal View History

2016-01-28 01:00:27 +00:00
#pragma once
2016-03-01 17:47:53 +00:00
#include <DB/Storages/MergeTree/ReshardingJob.h>
2016-01-28 01:00:27 +00:00
#include <DB/Storages/AlterCommands.h>
#include <common/logger_useful.h>
2016-03-01 17:47:53 +00:00
#include <zkutil/RWLock.h>
#include <zkutil/SingleBarrier.h>
2016-01-28 01:00:42 +00:00
#include <Poco/Util/LayeredConfiguration.h>
2016-01-28 01:00:27 +00:00
#include <Poco/SharedPtr.h>
2016-03-01 17:47:53 +00:00
2016-01-28 01:00:27 +00:00
#include <string>
#include <thread>
#include <atomic>
2016-03-01 17:47:53 +00:00
#include <functional>
2016-01-28 01:00:27 +00:00
namespace DB
{
class Context;
2016-03-01 17:47:53 +00:00
class Cluster;
2016-01-28 01:00:27 +00:00
class StorageReplicatedMergeTree;
/** Исполнитель задач перешардирования.
* Рабоает в фоновом режиме внутри одного потока.
* Следит за появлением задач и назначает их на выполнение.
* Задачи выполняются последовательно.
*/
class ReshardingWorker final
{
2016-03-25 11:48:45 +00:00
friend class AnomalyMonitor;
2016-03-01 17:47:53 +00:00
public:
using PartitionList = std::vector<std::string>;
2016-03-25 11:48:45 +00:00
/// Possible status values of a coordinator or a node that
/// has subscribed to a coordinator.
enum StatusCode
2016-03-01 17:47:53 +00:00
{
STATUS_OK = 0,
STATUS_ERROR, /// Произошла ошибка на одном исполнителе.
STATUS_ON_HOLD /// Задача приостановлена.
};
2016-01-28 01:00:27 +00:00
public:
2016-01-28 01:00:42 +00:00
ReshardingWorker(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name, Context & context_);
2016-01-28 01:00:27 +00:00
ReshardingWorker(const ReshardingWorker &) = delete;
ReshardingWorker & operator=(const ReshardingWorker &) = delete;
~ReshardingWorker();
2016-03-09 13:40:47 +00:00
/// Start the thread which performs resharding jobs.
2016-01-28 01:00:27 +00:00
void start();
2016-03-09 13:40:47 +00:00
/// Stop the thread which performs resharding jobs.
/// If any job is in progress, put it on hold for further execution.
2016-03-01 17:47:53 +00:00
void shutdown();
2016-01-28 01:00:27 +00:00
/// Прислать запрос на перешардирование.
2016-03-01 17:47:53 +00:00
void submitJob(const ReshardingJob & job);
2016-01-28 01:00:27 +00:00
/// Был ли поток запущен?
bool isStarted() const;
2016-03-01 17:47:53 +00:00
/// Создать новый координатор распределённой задачи. Вызывается с инициатора.
std::string createCoordinator(const Cluster & cluster);
2016-03-09 13:40:47 +00:00
/// Register a query into a coordinator.
2016-03-01 17:47:53 +00:00
void registerQuery(const std::string & coordinator_id, const std::string & query);
/// Удалить координатор.
void deleteCoordinator(const std::string & coordinator_id);
/// Подписаться к заданному координатору. Вызывается с исполнителя.
UInt64 subscribe(const std::string & coordinator_id, const std::string & query);
/// Отменить подпись к заданному координатору. Вызывается с исполнителя.
void unsubscribe(const std::string & coordinator_id);
/// Увеличить количество партиций входящих в одну распределённую задачу. Вызывается с исполнителя.
void addPartitions(const std::string & coordinator_id, const PartitionList & partition_list);
2016-03-09 13:40:47 +00:00
/// Rearrange partitions into two categories: coordinated job, uncoordinated job.
/// Returns an iterator to the beginning of the list of uncoordinated jobs.
ReshardingWorker::PartitionList::iterator categorizePartitions(const std::string & coordinator_id,
ReshardingWorker::PartitionList & partition_list);
2016-03-01 17:47:53 +00:00
/// Получить количество партиций входящих в одну распределённую задачу. Вызывается с исполнителя.
size_t getPartitionCount(const std::string & coordinator_id);
/// Получить количество учавствующих узлов.
size_t getNodeCount(const std::string & coordinator_id);
/// Ждать завершение проверок на всех исполнителях. Вызывается с исполнителя.
void waitForCheckCompletion(const std::string & coordinator_id);
/// Ждать завершение всех необходмых отмен подписей.
void waitForOptOutCompletion(const std::string & coordinator_id, size_t count);
2016-03-09 13:40:47 +00:00
/// Set the shard-independent status of a given coordinator.
2016-03-25 11:48:45 +00:00
void setStatus(const std::string & coordinator_id, StatusCode status, const std::string & msg = "");
2016-03-09 13:40:47 +00:00
/// Set the status of a shard under a given coordinator.
2016-03-25 11:48:45 +00:00
void setStatus(const std::string & coordinator_id, const std::string & hostname,
StatusCode status, const std::string & msg = "");
2016-03-01 17:47:53 +00:00
zkutil::RWLock createDeletionLock(const std::string & coordinator_id);
2016-03-25 11:48:45 +00:00
/// Dump the status messages of the coordinator and all the participating nodes.
std::string dumpCoordinatorState(const std::string & coordinator_id);
private:
/// Anomalies that may be detected by the local node.
enum AnomalyType
{
ANOMALY_NONE = 0,
ANOMALY_LOCAL_SHUTDOWN,
ANOMALY_LOCAL_ERROR,
ANOMALY_REMOTE_NODE_UNAVAILABLE,
ANOMALY_REMOTE_ERROR
};
/// This structure stores various required information for log creation.
struct TargetShardInfo
{
TargetShardInfo(size_t shard_no_, const std::string & part_name_, const std::string & hash_)
: part_name{part_name_}, hash{hash_}, shard_no{shard_no_}
{
}
std::string part_name;
std::string hash;
size_t shard_no;
};
using ShardList = std::vector<TargetShardInfo>;
/// Structure that describes an operation to be performed as part of
/// a commit procedure.
class LogRecord final
{
public:
/// Create a new log record.
LogRecord(zkutil::ZooKeeperPtr zookeeper_);
/// Open an already existing log record.
LogRecord(zkutil::ZooKeeperPtr zookeeper_, const std::string & zk_path_);
/// Append this log record into the specified log onto persistent storage.
void enqueue(const std::string & log_path);
/// Update an already existing log record.
void writeBack();
public:
enum Operation
{
OP_DROP = 0,
OP_ATTACH
};
enum State
{
READY = 0,
RUNNING,
DONE
};
private:
/// Serialize this log record.
std::string toString();
public:
zkutil::ZooKeeperPtr zookeeper;
Operation operation;
/// Source partition being resharded.
std::string partition;
/// Hash of the source partition (for drop operations).
std::string partition_hash;
/// Path of this log record on stable storage.
std::string zk_path;
/// List of the parts that constitute a sharded partition along with
/// their respective hash values (for attach operations).
std::map<std::string, std::string> parts_with_hash;
/// Target shard index into the list of specified target shards inside
/// a ALTER TABLE ... RESHARD request.
size_t shard_no = 0;
State state;
};
2016-01-28 01:00:27 +00:00
private:
/// Следить за появлением новых задач. Выполнить их последовательно.
void pollAndExecute();
2016-03-01 17:47:53 +00:00
/// Подтолкнуть планировщик задач.
void jabScheduler();
2016-01-28 01:00:27 +00:00
/// Выполнить задачи, которые были в очереди выполнения при запуске узла.
void performPendingJobs();
/// Выполнить задачи, которые заданы по путям в БД ZooKeeper.
void perform(const Strings & job_nodes);
/// Выполнить одну задачу.
2016-03-25 11:48:45 +00:00
void perform(const std::string & job_descriptor, const std::string & job_name);
2016-01-28 01:00:27 +00:00
/// Разбить куски входящие в партицию на несколько, согласно ключу шардирования.
/// Оновременно перегруппировать эти куски по шардам и слить куски в каждой группе.
/// При завершении этого процесса создаётся новая партиция для каждого шарда.
2016-03-01 17:47:53 +00:00
void createShardedPartitions();
2016-01-28 01:00:27 +00:00
/// Копировать все партиции полученные путём перешардирования на каждую реплику
/// соответствующих шардов.
2016-03-01 17:47:53 +00:00
void publishShardedPartitions();
2016-01-28 01:00:27 +00:00
/// Для каждого шарда добавить данные из новой партиции этого шарда в таблицу на всех
/// репликах входящих в этот же шард. На локальном узле, который выполняет задачу
/// перешардирования, удалить данные из первоначальной партиции.
2016-03-25 11:48:45 +00:00
void commit();
void repairLogRecord(LogRecord & log_record);
void executeLogRecord(LogRecord & log_record);
void executeDrop(LogRecord & log_record);
void executeAttach(LogRecord & log_record);
bool checkAttachLogRecord(LogRecord & log_record);
2016-01-28 01:00:27 +00:00
/// Удалить временные данные с локального узла и ZooKeeper'а.
2016-03-01 17:47:53 +00:00
void softCleanup();
void hardCleanup();
2016-01-28 01:00:27 +00:00
2016-03-09 13:40:47 +00:00
/// Принудительно завершить поток, если выполнено условие.
2016-03-01 17:47:53 +00:00
void abortPollingIfRequested();
void abortCoordinatorIfRequested(const std::string & coordinator_id);
2016-03-01 17:47:53 +00:00
void abortRecoveryIfRequested();
void abortJobIfRequested();
2016-03-09 13:40:47 +00:00
/// Get the current job-independent status of the coordinator.
2016-03-25 11:48:45 +00:00
StatusCode getCoordinatorStatus(const std::string & coordinator_id);
2016-03-09 13:40:47 +00:00
/// Get the status of the current distributed job.
2016-03-25 11:48:45 +00:00
StatusCode getStatus();
2016-03-01 17:47:53 +00:00
/// Зарегистрировать задачу в соответствующий координатор.
void attachJob();
/// Снять задачу с координатора.
void detachJob();
/// Ждать завершение загрузок на всех исполнителях.
void waitForUploadCompletion();
2016-03-25 11:48:45 +00:00
///
void waitForElectionCompletion();
/// Wait for all the partition operations to be completed on all the participating nodes.
void waitForCommitCompletion();
2016-03-01 17:47:53 +00:00
size_t getPartitionCountUnlocked(const std::string & coordinator_id);
2016-03-03 15:38:13 +00:00
2016-03-09 13:40:47 +00:00
/// Detect offline nodes under a given coordinator.
bool detectOfflineNodes(const std::string & coordinator_id);
/// Detect offline nodes under the current job.
bool detectOfflineNodes();
2016-03-01 17:47:53 +00:00
2016-03-25 11:48:45 +00:00
bool isPublished();
void markAsPublished();
bool isLogCreated();
void markLogAsCreated();
bool isCommitted();
void markAsCommitted();
///
void storeTargetShards();
///
ShardList getTargetShards(const std::string & hostname, const std::string & job_name);
///
void createLog();
void electLeader();
bool isLeader();
2016-03-01 17:47:53 +00:00
/// Функции, которые создают необходимые объекты для синхронизации
/// распределённых задач.
zkutil::RWLock createLock();
2016-03-25 11:48:45 +00:00
zkutil::RWLock createCoordinatorLock(const std::string & coordinator_id,
bool usable_in_emergency = false);
zkutil::SingleBarrier createCheckBarrier(const std::string & coordinator_id);
2016-03-01 17:47:53 +00:00
zkutil::SingleBarrier createOptOutBarrier(const std::string & coordinator_id, size_t count);
zkutil::SingleBarrier createRecoveryBarrier(const ReshardingJob & job);
zkutil::SingleBarrier createUploadBarrier(const ReshardingJob & job);
2016-03-25 11:48:45 +00:00
zkutil::SingleBarrier createElectionBarrier(const ReshardingJob & job);
zkutil::SingleBarrier createCommitBarrier(const ReshardingJob & job);
/// Prevent merging jobs from being performed on the partition that we
/// want to reshard on the current host. This operation is persistent:
/// even if a node failure occurred, a partition remains frozen as long
/// as it is not unfrozen explicitely. So use it with care as regards
/// exceptions.
void freezeSourcePartition();
/// Make the partition that we want to reshard available for merging jobs.
void unfreezeSourcePartition();
2016-03-01 17:47:53 +00:00
2016-03-09 13:40:47 +00:00
/// Get the ZooKeeper path of a given coordinator.
2016-03-01 17:47:53 +00:00
std::string getCoordinatorPath(const std::string & coordinator_id) const;
2016-03-09 13:40:47 +00:00
/// Get the ZooKeeper path of a given job partition.
2016-03-01 17:47:53 +00:00
std::string getPartitionPath(const ReshardingJob & job) const;
2016-03-25 11:48:45 +00:00
///
std::string getLocalJobPath(const ReshardingJob & job) const;
2016-01-28 01:00:27 +00:00
2016-03-09 13:40:47 +00:00
/// Common code for softCleanup() and hardCleanup().
2016-03-25 11:48:45 +00:00
void deleteTemporaryData();
2016-03-09 13:40:47 +00:00
/// Common code for detectOfflineNodes().
bool detectOfflineNodesCommon(const std::string & path, const std::string & coordinator_id);
/// Common code for getStatus() and getCoordinatorStatus().
2016-03-25 11:48:45 +00:00
StatusCode getStatusCommon(const std::string & path, const std::string & coordinator_id);
2016-03-09 13:40:47 +00:00
2016-03-25 11:48:45 +00:00
AnomalyType probeForAnomaly();
void processAnomaly(AnomalyType anomaly_type);
private:
/// This class is used to spawn a thread which periodically checks for any
/// anomaly that should lead to abort the job currently being processed.
/// Jobs perform such checks with a negligible performance overhead by calling
/// the method getAnomalyType() which merely reads the value of an atomic variable.
class AnomalyMonitor final
{
public:
AnomalyMonitor(ReshardingWorker & resharding_worker_);
~AnomalyMonitor();
AnomalyMonitor(const AnomalyMonitor &) = delete;
AnomalyMonitor & operator=(const AnomalyMonitor &) = delete;
/// Start the thread.
void start();
/// Shutdown the thread.
void shutdown();
/// Get the type of anomaly that has been registered.
AnomalyType getAnomalyType() const;
private:
void routine();
private:
ReshardingWorker & resharding_worker;
std::thread thread_routine;
std::atomic<ReshardingWorker::AnomalyType> anomaly_type{ReshardingWorker::ANOMALY_NONE};
std::atomic<bool> is_started{false};
std::atomic<bool> must_stop{false};
};
/// Guarded exception-safe handling of the class above.
class ScopedAnomalyMonitor final
{
public:
ScopedAnomalyMonitor(AnomalyMonitor & anomaly_monitor_)
: anomaly_monitor{anomaly_monitor_}
{
anomaly_monitor.start();
}
~ScopedAnomalyMonitor()
{
try
{
anomaly_monitor.shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
ScopedAnomalyMonitor(const ScopedAnomalyMonitor &) = delete;
ScopedAnomalyMonitor & operator=(const ScopedAnomalyMonitor &) = delete;
private:
AnomalyMonitor & anomaly_monitor;
};
2016-03-09 13:40:47 +00:00
2016-01-28 01:00:27 +00:00
private:
2016-03-01 17:47:53 +00:00
ReshardingJob current_job;
std::thread polling_thread;
2016-03-25 11:48:45 +00:00
AnomalyMonitor anomaly_monitor{*this};
2016-03-01 17:47:53 +00:00
2016-03-25 11:48:45 +00:00
std::string task_queue_path;
2016-03-01 17:47:53 +00:00
std::string host_task_queue_path;
std::string distributed_path;
std::string distributed_online_path;
std::string distributed_lock_path;
std::string coordination_path;
2016-01-28 01:00:27 +00:00
Context & context;
2016-03-25 11:48:45 +00:00
Logger * log = &Logger::get("ReshardingWorker");
2016-03-01 17:47:53 +00:00
2016-03-08 13:46:00 +00:00
zkutil::EventPtr event = new Poco::Event;
zkutil::GetZooKeeper get_zookeeper;
2016-03-08 13:46:00 +00:00
2016-01-28 01:00:27 +00:00
std::atomic<bool> is_started{false};
std::atomic<bool> must_stop{false};
};
2016-01-28 01:00:42 +00:00
using ReshardingWorkerPtr = std::shared_ptr<ReshardingWorker>;
2016-01-28 01:00:27 +00:00
}