2017-04-13 13:42:29 +00:00
|
|
|
#pragma once
|
2020-05-20 20:16:32 +00:00
|
|
|
|
2018-09-28 13:44:39 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2020-12-30 07:56:22 +00:00
|
|
|
#include <Common/DNSResolver.h>
|
2019-01-14 19:22:09 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2021-07-09 14:05:35 +00:00
|
|
|
#include <Common/ZooKeeper/IKeeper.h>
|
2020-11-03 13:47:26 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
|
|
|
#include <Parsers/IAST_fwd.h>
|
2020-11-05 09:52:23 +00:00
|
|
|
#include <Interpreters/Context.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;
|
|
|
|
}
|
|
|
|
|
2020-11-03 13:47:26 +00:00
|
|
|
namespace Poco
|
|
|
|
{
|
|
|
|
class Logger;
|
|
|
|
namespace Util { class AbstractConfiguration; }
|
|
|
|
}
|
|
|
|
|
2021-02-09 15:14:20 +00:00
|
|
|
namespace Coordination
|
|
|
|
{
|
|
|
|
struct Stat;
|
|
|
|
}
|
|
|
|
|
2022-01-17 11:52:51 +00:00
|
|
|
namespace zkutil
|
|
|
|
{
|
|
|
|
class ZooKeeperLock;
|
|
|
|
}
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2017-06-13 04:45:30 +00:00
|
|
|
class ASTAlterQuery;
|
2017-04-27 15:19:11 +00:00
|
|
|
struct DDLLogEntry;
|
2020-11-27 14:04:03 +00:00
|
|
|
struct DDLTaskBase;
|
|
|
|
using DDLTaskPtr = std::unique_ptr<DDLTaskBase>;
|
2020-11-13 18:35:45 +00:00
|
|
|
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
|
2020-01-24 16:20:36 +00:00
|
|
|
class AccessRightsElements;
|
2020-11-13 18:35:45 +00:00
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
class DDLWorker
|
|
|
|
{
|
|
|
|
public:
|
2021-04-10 23:33:54 +00:00
|
|
|
DDLWorker(int pool_size_, const std::string & zk_root_dir, ContextPtr context_, const Poco::Util::AbstractConfiguration * config, const String & prefix,
|
2021-08-11 03:40:06 +00:00
|
|
|
const String & logger_name = "DDLWorker", const CurrentMetrics::Metric * max_entry_metric_ = nullptr, const CurrentMetrics::Metric * max_pushed_entry_metric_ = nullptr);
|
2020-11-27 14:04:03 +00:00
|
|
|
virtual ~DDLWorker();
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2017-04-21 12:39:28 +00:00
|
|
|
/// Pushes query into DDL queue, returns path to created node
|
2020-11-27 14:04:03 +00:00
|
|
|
virtual 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
|
|
|
}
|
|
|
|
|
2020-12-01 17:20:42 +00:00
|
|
|
void startup();
|
2021-02-12 16:22:01 +00:00
|
|
|
virtual void shutdown();
|
2020-11-13 18:35:45 +00:00
|
|
|
|
2020-11-29 11:45:32 +00:00
|
|
|
bool isCurrentlyActive() const { return initialized && !stop_flag; }
|
|
|
|
|
2020-11-27 14:04:03 +00:00
|
|
|
protected:
|
2019-02-22 16:14:48 +00:00
|
|
|
|
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-12-03 18:14:27 +00:00
|
|
|
/// Iterates through queue tasks in ZooKeeper, runs execution of new tasks
|
2021-02-23 05:26:46 +00:00
|
|
|
void scheduleTasks(bool reinitialized);
|
2017-04-27 15:19:11 +00:00
|
|
|
|
2020-12-03 18:14:27 +00:00
|
|
|
DDLTaskBase & saveTask(DDLTaskPtr && task);
|
|
|
|
|
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
|
2020-11-27 14:04:03 +00:00
|
|
|
virtual DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper);
|
2017-07-28 16:14:49 +00:00
|
|
|
|
2021-02-19 23:41:58 +00:00
|
|
|
void processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper);
|
2021-02-03 20:02:37 +00:00
|
|
|
void updateMaxDDLEntryID(const String & entry_name);
|
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
|
2021-02-15 10:26:34 +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
|
2022-05-09 19:13:02 +00:00
|
|
|
/// query via RemoteQueryExecutor to leader, so to avoid such "2-phase" query execution we
|
2019-06-26 14:52:20 +00:00
|
|
|
/// execute query directly on leader.
|
|
|
|
bool tryExecuteQueryOnLeaderReplica(
|
2020-11-27 14:04:03 +00:00
|
|
|
DDLTaskBase & 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,
|
2021-11-22 16:46:34 +00:00
|
|
|
const ZooKeeperPtr & zookeeper,
|
2022-01-17 11:52:51 +00:00
|
|
|
std::unique_ptr<zkutil::ZooKeeperLock> & execute_on_leader_lock);
|
2017-04-17 17:04:31 +00:00
|
|
|
|
2021-02-19 23:41:58 +00:00
|
|
|
bool tryExecuteQuery(const String & query, DDLTaskBase & task, const ZooKeeperPtr & zookeeper);
|
2017-07-27 18:44:55 +00:00
|
|
|
|
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);
|
2021-02-09 15:14:20 +00:00
|
|
|
virtual bool canRemoveQueueEntry(const String & entry_name, const Coordination::Stat & stat);
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2017-08-10 19:12:52 +00:00
|
|
|
/// Init task node
|
2021-02-16 14:05:58 +00:00
|
|
|
void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
|
2017-04-25 15:21:03 +00:00
|
|
|
|
DDLWorker: avoid NULL dereference on termination and failed zookeeper initialization
Log snipped shows the problem:
2021.02.24 04:40:29.349181 [ 39 ] {} <Warning> DDLWorker: DDLWorker is configured to use multiple threads. It's not recommended because queries can be reordered. Also it may cause some unknown issues to appear.
2021.02.24 04:40:29.349516 [ 39 ] {} <Information> Application: Ready for connections.
2021.02.24 04:40:29.349602 [ 74 ] {} <Debug> DDLWorker: Started DDLWorker cleanup thread
2021.02.24 04:40:29.349639 [ 73 ] {} <Debug> DDLWorker: Starting DDLWorker thread
2021.02.24 04:40:29.349698 [ 73 ] {} <Debug> DDLWorker: Started DDLWorker thread
2021.02.24 04:40:29.352548 [ 73 ] {} <Error> virtual void DB::DDLWorker::initializeMainThread(): Code: 999, e.displayText() = Coordination::Exception: All connection tries failed while connecting to ZooKeeper. nodes: 192.168.112.3:2181
Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
(Connection loss), Stack trace (when copying this message, always include the lines below):
0. Coordination::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, Coordination::Error, int) @ 0xfe93923 in /usr/bin/clickhouse
1. Coordination::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, Coordination::Error) @ 0xfe93ba2 in /usr/bin/clickhouse
2. Coordination::ZooKeeper::connect(std::__1::vector<Coordination::ZooKeeper::Node, std::__1::allocator<Coordination::ZooKeeper::Node> > const&, Poco::Timespan) @ 0xfed3a01 in /usr/bin/clickhouse
3. Coordination::ZooKeeper::ZooKeeper(std::__1::vector<Coordination::ZooKeeper::Node, std::__1::allocator<Coordination::ZooKeeper::Node> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, Poco::Timespan, Poco::Timespan, Poco::Timespan) @ 0xfed2222 in /usr/bin/clickhouse
4. zkutil::ZooKeeper::init(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int, int, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) @ 0xfe961cd in /usr/bin/clickhouse
5. zkutil::ZooKeeper::ZooKeeper(Poco::Util::AbstractConfiguration const&, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&) @ 0xfe97a96 in /usr/bin/clickhouse
6. void std::__1::allocator_traits<std::__1::allocator<zkutil::ZooKeeper> >::__construct<zkutil::ZooKeeper, Poco::Util::AbstractConfiguration const&, char const (&) [10]>(std::__1::integral_constant<bool, true>, std::__1::allocator<zkutil::ZooKeeper>&, zkutil::ZooKeeper*, Poco::Util::AbstractConfiguration const&, char const (&) [10]) @ 0xed98387 in /usr/bin/clickhouse
7. DB::Context::getZooKeeper() const @ 0xed75190 in /usr/bin/clickhouse
8. DB::DDLWorker::getAndSetZooKeeper() @ 0xedb81c9 in /usr/bin/clickhouse
9. DB::DDLWorker::initializeMainThread() @ 0xedc9eb0 in /usr/bin/clickhouse
10. DB::DDLWorker::runMainThread() @ 0xedb5d01 in /usr/bin/clickhouse
11. ThreadFromGlobalPool::ThreadFromGlobalPool<void (DB::DDLWorker::*)(), DB::DDLWorker*>(void (DB::DDLWorker::*&&)(), DB::DDLWorker*&&)::'lambda'()::operator()() @ 0xedcafa1 in /usr/bin/clickhouse
12. ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) @ 0x892651f in /usr/bin/clickhouse
13. ? @ 0x8929fb3 in /usr/bin/clickhouse
14. start_thread @ 0x8ea7 in /lib/x86_64-linux-gnu/libpthread-2.31.so
15. __clone @ 0xfddef in /lib/x86_64-linux-gnu/libc-2.31.so
(version 21.3.1.1)
...
2021.02.24 04:40:30.025278 [ 41 ] {} <Trace> BaseDaemon: Received signal 15
2021.02.24 04:40:30.025336 [ 41 ] {} <Information> Application: Received termination signal (Terminated)
...
2021.02.24 04:40:30.582078 [ 39 ] {} <Information> Application: Closed all listening sockets.
2021.02.24 04:40:30.582124 [ 39 ] {} <Information> Application: Closed connections.
2021.02.24 04:40:30.583770 [ 39 ] {} <Information> Application: Shutting down storages.
2021.02.24 04:40:30.583932 [ 39 ] {} <Information> Context: Shutdown disk data
2021.02.24 04:40:30.583951 [ 39 ] {} <Information> Context: Shutdown disk default
2021.02.24 04:40:30.584163 [ 46 ] {} <Trace> SystemLog (system.query_log): Terminating
2021.02.24 04:40:30.586025 [ 39 ] {} <Trace> BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
2021.02.24 04:40:34.352701 [ 73 ] {} <Debug> DDLWorker: Initialized DDLWorker thread
2021.02.24 04:40:34.352758 [ 73 ] {} <Debug> DDLWorker: Scheduling tasks
2021-02-24 05:07:31 +00:00
|
|
|
/// Return false if the worker was stopped (stop_flag = true)
|
|
|
|
virtual bool initializeMainThread();
|
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
|
|
|
|
2021-05-31 14:49:02 +00:00
|
|
|
ContextMutablePtr 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
|
2021-02-04 19:41:44 +00:00
|
|
|
std::optional<String> last_skipped_entry_name;
|
2021-03-04 23:17:07 +00:00
|
|
|
std::optional<String> first_failed_task_name;
|
2020-12-03 18:14:27 +00:00
|
|
|
std::list<DDLTaskPtr> current_tasks;
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2022-03-22 13:43:52 +00:00
|
|
|
/// This flag is needed for debug assertions only
|
|
|
|
bool queue_fully_loaded_after_initialization_debug_helper = false;
|
|
|
|
|
2021-07-09 14:05:35 +00:00
|
|
|
Coordination::Stat queue_node_stat;
|
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-11-29 11:45:32 +00:00
|
|
|
std::atomic<bool> initialized = false;
|
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;
|
2020-12-30 12:25:00 +00:00
|
|
|
std::unique_ptr<ThreadPool> worker_pool;
|
2020-09-02 22:35:47 +00:00
|
|
|
|
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
|
|
|
|
2020-11-27 04:27:43 +00:00
|
|
|
std::atomic<UInt64> max_id = 0;
|
2021-02-15 10:26:34 +00:00
|
|
|
const CurrentMetrics::Metric * max_entry_metric;
|
2021-08-11 03:40:06 +00:00
|
|
|
const CurrentMetrics::Metric * max_pushed_entry_metric;
|
2017-04-13 13:42:29 +00:00
|
|
|
};
|
|
|
|
|
2017-04-27 15:19:11 +00:00
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
}
|