ClickHouse/src/Databases/DatabaseReplicatedWorker.h

47 lines
2.0 KiB
C++
Raw Normal View History

2020-11-27 14:04:03 +00:00
#pragma once
#include <Interpreters/DDLWorker.h>
namespace DB
{
class DatabaseReplicated;
2021-02-19 23:41:58 +00:00
/// It's similar to DDLWorker, but has the following differences:
/// 1. DDL queue in ZooKeeper is not shared between multiple clusters and databases,
/// each DatabaseReplicated has its own queue in ZooKeeper and DatabaseReplicatedDDLWorker object.
/// 2. Shards and replicas are identified by shard_name and replica_name arguments of database engine,
/// not by address:port pairs. Cluster (of multiple database replicas) is identified by its zookeeper_path.
/// 3. After creation of an entry in DDL queue initiator tries to execute the entry locally
/// and other hosts wait for query to finish on initiator host.
/// If query succeed on initiator, then all hosts must execute it, so they will retry until query succeed.
2021-02-20 12:56:28 +00:00
/// We assume that cluster is homogeneous, so if replicas are in consistent state and query succeed on one host,
2021-02-19 23:41:58 +00:00
/// then all hosts can execute it (maybe after several retries).
/// 4. Each database replica stores its log pointer in ZooKeeper. Cleanup thread removes old entry
/// if its number < max_log_ptr - logs_to_keep.
2020-11-27 14:04:03 +00:00
class DatabaseReplicatedDDLWorker : public DDLWorker
{
public:
DatabaseReplicatedDDLWorker(DatabaseReplicated * db, const Context & context_);
String enqueueQuery(DDLLogEntry & entry) override;
2021-02-04 19:41:44 +00:00
String tryEnqueueAndExecuteEntry(DDLLogEntry & entry, const Context & query_context);
2021-02-12 16:22:01 +00:00
void shutdown() override;
2020-11-27 14:04:03 +00:00
private:
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
bool initializeMainThread() override;
void initializeReplication();
2020-11-27 14:04:03 +00:00
DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override;
2021-02-09 15:14:20 +00:00
bool canRemoveQueueEntry(const String & entry_name, const Coordination::Stat & stat) override;
2020-11-27 14:04:03 +00:00
2021-02-02 19:39:04 +00:00
DatabaseReplicated * const database;
mutable std::mutex mutex;
std::condition_variable wait_current_task_change;
String current_task;
2021-02-09 15:14:20 +00:00
UInt32 logs_to_keep = std::numeric_limits<UInt32>::max();
2020-11-27 14:04:03 +00:00
};
}