2017-04-13 13:42:29 +00:00
|
|
|
#pragma once
|
2020-05-20 20:16:32 +00:00
|
|
|
|
2017-04-13 16:12:56 +00:00
|
|
|
#include <Interpreters/Cluster.h>
|
2020-12-10 22:05:02 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-04-13 16:12:56 +00:00
|
|
|
#include <DataStreams/BlockIO.h>
|
2018-09-28 13:44:39 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2017-04-13 13:42:29 +00:00
|
|
|
#include <common/logger_useful.h>
|
2019-06-26 14:52:20 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2017-04-13 13:42:29 +00:00
|
|
|
|
|
|
|
#include <atomic>
|
|
|
|
#include <chrono>
|
|
|
|
#include <condition_variable>
|
|
|
|
#include <mutex>
|
|
|
|
#include <thread>
|
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
namespace zkutil
|
|
|
|
{
|
|
|
|
class ZooKeeper;
|
|
|
|
}
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
class Context;
|
2017-06-13 04:45:30 +00:00
|
|
|
class ASTAlterQuery;
|
2020-01-24 16:20:36 +00:00
|
|
|
class AccessRightsElements;
|
2017-04-27 15:19:11 +00:00
|
|
|
struct DDLLogEntry;
|
2017-07-26 19:31:32 +00:00
|
|
|
struct DDLTask;
|
2020-09-02 22:35:47 +00:00
|
|
|
using DDLTaskPtr = std::unique_ptr<DDLTask>;
|
2017-04-13 16:12:56 +00:00
|
|
|
|
|
|
|
|
2018-04-17 19:33:58 +00:00
|
|
|
/// Pushes distributed DDL query to the queue
|
2020-04-05 23:03:20 +00:00
|
|
|
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context);
|
2020-06-20 22:44:52 +00:00
|
|
|
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const AccessRightsElements & query_requires_access, bool query_requires_grant_option = false);
|
|
|
|
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, AccessRightsElements && query_requires_access, bool query_requires_grant_option = false);
|
2017-04-17 17:04:31 +00:00
|
|
|
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
class DDLWorker
|
|
|
|
{
|
|
|
|
public:
|
2020-09-02 22:35:47 +00:00
|
|
|
DDLWorker(int pool_size_, const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix);
|
2017-04-13 13:42:29 +00:00
|
|
|
~DDLWorker();
|
|
|
|
|
2017-04-21 12:39:28 +00:00
|
|
|
/// Pushes query into DDL queue, returns path to created node
|
2017-04-18 15:44:31 +00:00
|
|
|
String enqueueQuery(DDLLogEntry & entry);
|
2017-04-13 16:12:56 +00:00
|
|
|
|
2017-07-26 19:31:32 +00:00
|
|
|
/// Host ID (name:port) for logging purposes
|
2017-07-28 16:14:49 +00:00
|
|
|
/// Note that in each task hosts are identified individually by name:port from initiator server cluster config
|
2017-07-26 19:31:32 +00:00
|
|
|
std::string getCommonHostID() const
|
2017-04-13 16:12:56 +00:00
|
|
|
{
|
2017-07-26 19:31:32 +00:00
|
|
|
return host_fqdn_id;
|
2017-04-13 16:12:56 +00:00
|
|
|
}
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
private:
|
2019-02-22 16:14:48 +00:00
|
|
|
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
|
|
|
|
|
2019-02-22 18:36:36 +00:00
|
|
|
/// Returns cached ZooKeeper session (possibly expired).
|
2019-02-22 16:14:48 +00:00
|
|
|
ZooKeeperPtr tryGetZooKeeper() const;
|
2019-02-22 18:36:36 +00:00
|
|
|
/// If necessary, creates a new session and caches it.
|
2019-02-22 16:14:48 +00:00
|
|
|
ZooKeeperPtr getAndSetZooKeeper();
|
2020-09-02 22:35:47 +00:00
|
|
|
/// ZooKeeper recover loop (while not stopped).
|
|
|
|
void recoverZooKeeper();
|
2019-02-22 16:14:48 +00:00
|
|
|
|
2020-09-02 22:35:47 +00:00
|
|
|
void checkCurrentTasks();
|
|
|
|
void scheduleTasks();
|
|
|
|
void saveTask(const String & entry_name);
|
2017-04-27 15:19:11 +00:00
|
|
|
|
2017-08-01 14:41:00 +00:00
|
|
|
/// Reads entry and check that the host belongs to host list of the task
|
2020-09-02 22:35:47 +00:00
|
|
|
/// Returns non-empty DDLTaskPtr if entry parsed and the check is passed
|
|
|
|
DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper);
|
2017-07-28 16:14:49 +00:00
|
|
|
|
2020-09-02 22:35:47 +00:00
|
|
|
void enqueueTask(DDLTaskPtr task);
|
|
|
|
void processTask(DDLTask & task);
|
2017-04-27 15:19:11 +00:00
|
|
|
|
2019-06-26 14:52:20 +00:00
|
|
|
/// Check that query should be executed on leader replica only
|
2020-03-18 00:57:00 +00:00
|
|
|
static bool taskShouldBeExecutedOnLeader(const ASTPtr ast_ddl, StoragePtr storage);
|
2019-06-26 14:52:20 +00:00
|
|
|
|
|
|
|
/// Executes query only on leader replica in case of replicated table.
|
|
|
|
/// Queries like TRUNCATE/ALTER .../OPTIMIZE have to be executed only on one node of shard.
|
|
|
|
/// Most of these queries can be executed on non-leader replica, but actually they still send
|
|
|
|
/// query via RemoteBlockOutputStream to leader, so to avoid such "2-phase" query execution we
|
|
|
|
/// execute query directly on leader.
|
|
|
|
bool tryExecuteQueryOnLeaderReplica(
|
2017-07-26 19:31:32 +00:00
|
|
|
DDLTask & task,
|
2019-06-26 14:52:20 +00:00
|
|
|
StoragePtr storage,
|
2017-04-27 15:19:11 +00:00
|
|
|
const String & rewritten_query,
|
2019-02-22 16:14:48 +00:00
|
|
|
const String & node_path,
|
|
|
|
const ZooKeeperPtr & zookeeper);
|
2017-04-17 17:04:31 +00:00
|
|
|
|
2017-07-28 16:14:49 +00:00
|
|
|
void parseQueryAndResolveHost(DDLTask & task);
|
|
|
|
|
2017-07-27 18:44:55 +00:00
|
|
|
bool tryExecuteQuery(const String & query, const DDLTask & task, ExecutionStatus & status);
|
|
|
|
|
2017-04-19 14:21:27 +00:00
|
|
|
/// Checks and cleanups queue's nodes
|
2019-02-22 18:36:36 +00:00
|
|
|
void cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zookeeper);
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2017-08-10 19:12:52 +00:00
|
|
|
/// Init task node
|
2020-03-18 00:57:00 +00:00
|
|
|
static void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
|
2017-04-25 15:21:03 +00:00
|
|
|
|
2017-07-28 16:14:49 +00:00
|
|
|
|
2019-02-22 18:36:36 +00:00
|
|
|
void runMainThread();
|
|
|
|
void runCleanupThread();
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2018-09-28 13:44:39 +00:00
|
|
|
void attachToThreadGroup();
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
private:
|
2020-11-03 12:51:26 +00:00
|
|
|
Context context;
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log;
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2017-07-26 19:31:32 +00:00
|
|
|
std::string host_fqdn; /// current host domain name
|
|
|
|
std::string host_fqdn_id; /// host_name:port
|
2017-04-27 15:19:11 +00:00
|
|
|
std::string queue_dir; /// dir with queue of queries
|
2017-04-17 17:04:31 +00:00
|
|
|
|
2019-02-22 16:14:48 +00:00
|
|
|
mutable std::mutex zookeeper_mutex;
|
|
|
|
ZooKeeperPtr current_zookeeper;
|
2017-04-27 15:19:11 +00:00
|
|
|
|
|
|
|
/// Save state of executed task to avoid duplicate execution on ZK error
|
2020-09-02 22:35:47 +00:00
|
|
|
std::vector<std::string> last_tasks;
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2019-02-22 18:36:36 +00:00
|
|
|
std::shared_ptr<Poco::Event> queue_updated_event = std::make_shared<Poco::Event>();
|
|
|
|
std::shared_ptr<Poco::Event> cleanup_event = std::make_shared<Poco::Event>();
|
2020-09-02 22:35:47 +00:00
|
|
|
std::atomic<bool> stop_flag = false;
|
2019-02-22 16:14:48 +00:00
|
|
|
|
2019-02-22 18:36:36 +00:00
|
|
|
ThreadFromGlobalPool main_thread;
|
|
|
|
ThreadFromGlobalPool cleanup_thread;
|
2017-05-31 14:01:08 +00:00
|
|
|
|
2020-09-02 22:35:47 +00:00
|
|
|
/// Size of the pool for query execution.
|
|
|
|
size_t pool_size = 1;
|
|
|
|
ThreadPool worker_pool;
|
|
|
|
|
2017-05-31 14:01:08 +00:00
|
|
|
/// Cleaning starts after new node event is received if the last cleaning wasn't made sooner than N seconds ago
|
2017-08-10 19:12:52 +00:00
|
|
|
Int64 cleanup_delay_period = 60; // minute (in seconds)
|
2017-08-02 14:42:35 +00:00
|
|
|
/// Delete node if its age is greater than that
|
2017-08-10 19:12:52 +00:00
|
|
|
Int64 task_max_lifetime = 7 * 24 * 60 * 60; // week (in seconds)
|
2017-08-02 14:42:35 +00:00
|
|
|
/// How many tasks could be in the queue
|
|
|
|
size_t max_tasks_in_queue = 1000;
|
2017-04-19 14:21:27 +00:00
|
|
|
|
2018-09-28 13:44:39 +00:00
|
|
|
ThreadGroupStatusPtr thread_group;
|
|
|
|
|
2020-11-27 04:27:43 +00:00
|
|
|
std::atomic<UInt64> max_id = 0;
|
|
|
|
|
2019-02-28 07:27:30 +00:00
|
|
|
friend class DDLQueryStatusInputStream;
|
2017-08-13 09:27:38 +00:00
|
|
|
friend struct DDLTask;
|
2017-04-13 13:42:29 +00:00
|
|
|
};
|
|
|
|
|
2017-04-27 15:19:11 +00:00
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
}
|