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-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;
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Performer of resharding jobs. It works as a background task within a thread.
|
|
|
|
|
/// Its main duties are to keep track of newly submitted resharding jobs and
|
|
|
|
|
/// to execute them sequentially.
|
2016-01-28 01:00:27 +00:00
|
|
|
|
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-29 16:29:39 +00:00
|
|
|
|
/// Possible status values of a coordinator or a performer that
|
2016-03-25 11:48:45 +00:00
|
|
|
|
/// has subscribed to a coordinator.
|
|
|
|
|
enum StatusCode
|
2016-03-01 17:47:53 +00:00
|
|
|
|
{
|
|
|
|
|
STATUS_OK = 0,
|
2016-03-29 16:29:39 +00:00
|
|
|
|
STATUS_ERROR, /// An error occurred on a performer.
|
|
|
|
|
STATUS_ON_HOLD /// Job is stopped.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
};
|
|
|
|
|
|
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-29 16:29:39 +00:00
|
|
|
|
/// Start the job tracker thread.
|
2016-01-28 01:00:27 +00:00
|
|
|
|
void start();
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Stop the job tracker thread. If any job is in progress, put it on hold
|
|
|
|
|
/// for future execution.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void shutdown();
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Send a request for resharding on the current performer.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void submitJob(const ReshardingJob & job);
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Is the job tracker thread started?
|
2016-01-28 01:00:27 +00:00
|
|
|
|
bool isStarted() const;
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Create a new coordinator for a distributed job. Called from the initiator.
|
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);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Clear all the information related to a given coordinator.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void deleteCoordinator(const std::string & coordinator_id);
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Subscribe the performer, from which this method is called, to a given coordinator.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
UInt64 subscribe(const std::string & coordinator_id, const std::string & query);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Cancel the aforementionned subscription.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
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-29 16:29:39 +00:00
|
|
|
|
/// Get the number of partitions of a distributed job. Called from performers.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
size_t getPartitionCount(const std::string & coordinator_id);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Get the number of performers.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
size_t getNodeCount(const std::string & coordinator_id);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Wait for all the preliminary sanity checks to be completed on all the performers.
|
|
|
|
|
/// Called from performers.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void waitForCheckCompletion(const std::string & coordinator_id);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Wait for all the required unsubscribe operations to be completed.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
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
|
|
|
|
|
2016-03-03 14:37:15 +00:00
|
|
|
|
zkutil::RWLock createDeletionLock(const std::string & coordinator_id);
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Dump the status messages of the coordinator and all the performers.
|
2016-03-25 11:48:45 +00:00
|
|
|
|
std::string dumpCoordinatorState(const std::string & coordinator_id);
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
static std::string computeHashFromPart(const std::string & path);
|
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
private:
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Anomalies that may be detected by the current performer.
|
2016-03-25 11:48:45 +00:00
|
|
|
|
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:
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Keep track of newly submitted jobs. Perform them sequentially.
|
|
|
|
|
void trackAndPerform();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Wake up the tracker thread if it was ever sleeping.
|
|
|
|
|
void wakeUpTrackerThread();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Perform all the jobs that were already pending when the ClickHouse instance
|
|
|
|
|
/// on this node was starting.
|
2016-01-28 01:00:27 +00:00
|
|
|
|
void performPendingJobs();
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Sequentially perform jobs which are specified by their corresponding
|
|
|
|
|
/// znodes in ZooKeeper.
|
2016-01-28 01:00:27 +00:00
|
|
|
|
void perform(const Strings & job_nodes);
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Perform one job.
|
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-29 16:29:39 +00:00
|
|
|
|
/// Upload all the partitions resulting from source partition resharding to their
|
|
|
|
|
/// respective target shards.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void publishShardedPartitions();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// On each target shard attach its corresponding new sharded partition.
|
2016-09-01 10:24:06 +00:00
|
|
|
|
/// If COPY is not specified in the ALTER TABLE ... RESHARD ... query,
|
|
|
|
|
/// drop source partitions on each performer.
|
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
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Delete temporary data from the local node and ZooKeeper.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void softCleanup();
|
|
|
|
|
void hardCleanup();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Forcibly abort the job tracker thread if requested.
|
|
|
|
|
void abortTrackingIfRequested();
|
|
|
|
|
/// Forcibly abort the distributed job coordinated by the given coordinator
|
|
|
|
|
/// if requested.
|
2016-03-02 13:35:30 +00:00
|
|
|
|
void abortCoordinatorIfRequested(const std::string & coordinator_id);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Forcibly abort the recovery of the current distributed job if requested.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void abortRecoveryIfRequested();
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Forcibly abort the current job if requested.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
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
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Prepare the current job for execution.
|
|
|
|
|
void initializeJob();
|
|
|
|
|
/// Remove the current job's data from its coordinator.
|
|
|
|
|
/// If there is no job left, also delete the coordinator.
|
|
|
|
|
void finalizeJob();
|
|
|
|
|
/// Wait for all the necesssary sharded partitions uploads to their respective
|
|
|
|
|
/// target shards to be completed.
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void waitForUploadCompletion();
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Wait for all the performers to be checked in for leader election.
|
2016-03-25 11:48:45 +00:00
|
|
|
|
void waitForElectionCompletion();
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Wait for all the changes required by the current distributed resharding job
|
|
|
|
|
/// to be applied by all the performers on all the target shards.
|
2016-03-25 11:48:45 +00:00
|
|
|
|
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-29 16:29:39 +00:00
|
|
|
|
/// Detect offline performers under a given coordinator.
|
2016-03-09 13:40:47 +00:00
|
|
|
|
bool detectOfflineNodes(const std::string & coordinator_id);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Detect offline performers under the current job.
|
2016-03-09 13:40:47 +00:00
|
|
|
|
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();
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Store onto ZooKeeper information about the target shards. It is used
|
|
|
|
|
/// in order to create the log of operations that apply changes all the
|
|
|
|
|
/// changes required by the resharding operation.
|
|
|
|
|
void storeTargetShardsInfo();
|
|
|
|
|
|
|
|
|
|
/// Retrieve from ZooKeeper the aforementioned information about the target shards.
|
|
|
|
|
ShardList getTargetShardsInfo(const std::string & hostname, const std::string & job_name);
|
2016-03-25 11:48:45 +00:00
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Create the log of operations.
|
2016-03-25 11:48:45 +00:00
|
|
|
|
void createLog();
|
|
|
|
|
|
|
|
|
|
void electLeader();
|
|
|
|
|
bool isLeader();
|
|
|
|
|
|
2016-03-29 16:29:39 +00:00
|
|
|
|
/// Access the global lock handler.
|
|
|
|
|
zkutil::RWLock getGlobalLock();
|
|
|
|
|
/// Acccess the given coordinator lock handler.
|
|
|
|
|
zkutil::RWLock getCoordinatorLock(const std::string & coordinator_id,
|
2016-03-25 11:48:45 +00:00
|
|
|
|
bool usable_in_emergency = false);
|
2016-03-29 16:29:39 +00:00
|
|
|
|
|
|
|
|
|
/// The following functions are to design access barrier handlers we use
|
|
|
|
|
/// to synchronize the progression of distributed jobs.
|
|
|
|
|
zkutil::SingleBarrier getCheckBarrier(const std::string & coordinator_id);
|
|
|
|
|
zkutil::SingleBarrier getOptOutBarrier(const std::string & coordinator_id, size_t count);
|
|
|
|
|
zkutil::SingleBarrier getRecoveryBarrier();
|
|
|
|
|
zkutil::SingleBarrier getUploadBarrier();
|
|
|
|
|
zkutil::SingleBarrier getElectionBarrier();
|
|
|
|
|
zkutil::SingleBarrier getCommitBarrier();
|
2016-03-25 11:48:45 +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-29 16:29:39 +00:00
|
|
|
|
std::string getPartitionPath() const;
|
|
|
|
|
/// Get the ZooKeeper path of the job currently running on this performer.
|
|
|
|
|
std::string getLocalJobPath() 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;
|
2016-03-29 16:29:39 +00:00
|
|
|
|
std::thread job_tracker;
|
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-05-28 17:31:50 +00:00
|
|
|
|
zkutil::EventPtr event = std::make_shared<Poco::Event>();
|
2016-09-01 10:24:06 +00:00
|
|
|
|
|
|
|
|
|
/// Helper that acquires an alive ZooKeeper session.
|
2016-03-25 12:21:10 +00:00
|
|
|
|
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
|
|
|
|
|
|
|
|
|
}
|