2017-04-13 13:42:29 +00:00
|
|
|
#pragma once
|
|
|
|
#include <Interpreters/Context.h>
|
2017-04-13 16:12:56 +00:00
|
|
|
#include <Interpreters/Cluster.h>
|
|
|
|
#include <DataStreams/BlockIO.h>
|
2017-04-13 13:42:29 +00:00
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
|
|
|
#include <atomic>
|
|
|
|
#include <chrono>
|
|
|
|
#include <condition_variable>
|
|
|
|
#include <mutex>
|
|
|
|
#include <thread>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-04-13 16:12:56 +00:00
|
|
|
BlockIO executeDDLQueryOnCluster(const String & query, const String & cluster_name, Context & context);
|
|
|
|
|
|
|
|
|
2017-04-17 17:04:31 +00:00
|
|
|
struct DDLLogEntry;
|
|
|
|
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
class DDLWorker
|
|
|
|
{
|
|
|
|
public:
|
2017-04-13 16:12:56 +00:00
|
|
|
DDLWorker(const std::string & zk_root_dir, Context & context_);
|
2017-04-13 13:42:29 +00:00
|
|
|
~DDLWorker();
|
|
|
|
|
2017-04-17 17:04:31 +00:00
|
|
|
void enqueueQuery(DDLLogEntry & entry);
|
2017-04-13 16:12:56 +00:00
|
|
|
|
|
|
|
/// Returns root/ path in ZooKeeper
|
|
|
|
std::string getRoot() const
|
|
|
|
{
|
|
|
|
return root_dir;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string getMastersDir() const
|
|
|
|
{
|
|
|
|
return root_dir + "/masters";
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string getCurrentMasterDir() const
|
|
|
|
{
|
|
|
|
return getMastersDir() + "/" + getHostName();
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string getHostName() const
|
|
|
|
{
|
|
|
|
return hostname;
|
|
|
|
}
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
private:
|
|
|
|
void processTasks();
|
2017-04-17 17:04:31 +00:00
|
|
|
bool processTask(const DDLLogEntry & node, const std::string & node_path);
|
|
|
|
|
|
|
|
void createStatusDirs(const std::string & node_name);
|
2017-04-13 16:12:56 +00:00
|
|
|
|
|
|
|
void processQueries();
|
|
|
|
bool processQuery(const std::string & task);
|
2017-04-13 13:42:29 +00:00
|
|
|
|
|
|
|
void run();
|
|
|
|
|
|
|
|
private:
|
|
|
|
Context & context;
|
|
|
|
Logger * log = &Logger::get("DDLWorker");
|
|
|
|
|
2017-04-13 16:12:56 +00:00
|
|
|
std::string hostname;
|
|
|
|
std::string root_dir; /// common dir with queue of queries
|
|
|
|
std::string assign_dir; /// dir with tasks assigned to the server
|
2017-04-17 17:04:31 +00:00
|
|
|
std::string master_dir; /// dir with queries was initiated by the server
|
|
|
|
|
|
|
|
std::string last_processed_node_name;
|
|
|
|
|
|
|
|
std::shared_ptr<zkutil::ZooKeeper> zookeeper;
|
|
|
|
std::shared_ptr<Poco::Event> queue_updated;
|
2017-04-13 13:42:29 +00:00
|
|
|
|
|
|
|
std::atomic<bool> stop_flag;
|
|
|
|
std::condition_variable cond_var;
|
|
|
|
std::mutex lock;
|
|
|
|
std::thread thread;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|