Commit Graph

182 Commits

Author SHA1 Message Date
Alexey Milovidov
b320905f34 Whitespaces 2021-07-05 03:02:47 +03:00
tavplubix
2b62a09aa3
Merge pull request #25373 from ClickHouse/improve_replicated_database_tests2
Improve Replicated database tests 2
2021-06-17 08:34:06 +03:00
Alexander Tokmakov
c235539620 minor fixes 2021-06-17 00:20:35 +03:00
Maksim Kita
67e9b85951 Merge ext into common 2021-06-16 23:28:41 +03:00
Alexander Tokmakov
0ad6205fa6 logs for debuging test failures with Replicated and Keeper 2021-04-15 21:34:53 +03:00
Ivan
495c6e03aa
Replace all Context references with std::weak_ptr (#22297)
* Replace all Context references with std::weak_ptr

* Fix shared context captured by value

* Fix build

* Fix Context with named sessions

* Fix copy context

* Fix gcc build

* Merge with master and fix build

* Fix gcc-9 build
2021-04-11 02:33:54 +03:00
Azat Khuzhin
f157278b72 Safer SCOPE_EXIT
It executes the code in the dtor, that should never throw.
2021-04-06 21:10:37 +03:00
3ldar-nasyrov
37a17749ea
fixed exceeded amount of tries typo
'retires' -> 'retries'
2021-03-16 23:17:07 +03:00
tavplubix
35a770a7f6
add comment 2021-03-05 15:50:10 +03:00
Alexander Tokmakov
614e0d9ba0 just another fix for ddl worker 2021-03-05 04:17:51 +03:00
Azat Khuzhin
7d51ae3212 DDLWorker: Fix processing of tasks in parallel (correct queue begin)
Otherwise it will:
- start from incorrect task and bail because finished node already
  exists
- and also process alreayd processed items
2021-02-28 05:45:12 +03:00
Azat Khuzhin
955974a8b5 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-28 05:45:12 +03:00
Azat Khuzhin
3c7e765b27 DDLWorker: process unfinished tasks only after reinitialization 2021-02-28 05:45:12 +03:00
Azat Khuzhin
54b1496408 DDLWorker: optimize processing of unfinished tasks
Do not look at zookeeper if was_executed is false
2021-02-28 05:45:12 +03:00
Azat Khuzhin
eeda1fe756 DDLWorker: wait for pending async tasks 2021-02-28 05:45:12 +03:00
Azat Khuzhin
0cd67ed051 DDLWorker: remove only completed tasks (significant for distributed_ddl.pool_size > 1)
Otherwise it will SIGSEGV due to invalid-read (UAF).
2021-02-28 05:45:12 +03:00
Azat Khuzhin
7481b05d27 DDLWorker: avoid comparing pointers for getting min element
The current_tasks always updated with emplace_back, so front() is ok.
2021-02-28 05:45:11 +03:00
Azat Khuzhin
78c6e0527d DDLWorker: Fix reference to zookeeper with distributed_ddl.pool_size > 1 (thread pool) 2021-02-28 05:45:11 +03:00
Alexander Tokmakov
2a36d6cb55 review suggestions 2021-02-20 02:41:58 +03:00
Alexander Tokmakov
1aac7b3471 Merge branch 'master' into database_replicated 2021-02-17 00:39:56 +03:00
Alexander Tokmakov
7b54b892b5 fix 2021-02-16 17:05:58 +03:00
Alexander Tokmakov
9c7cf9e92e remove some debug code 2021-02-15 13:26:34 +03:00
Alexander Tokmakov
320fd6b264 startup without zk 2021-02-15 03:04:46 +03:00
Alexander Tokmakov
184ec67dac better ddl queue cleanup 2021-02-12 22:23:50 +03:00
Alexander Tokmakov
f64f9b672b fix 2021-02-12 19:22:01 +03:00
Alexander Tokmakov
c9cf63e958 fix 2021-02-12 01:23:40 +03:00
Alexander Tokmakov
9afb16759e fix 2021-02-11 18:14:38 +03:00
Alexander Tokmakov
15256d86e5 better replica recovery and queue cleanup 2021-02-10 23:30:40 +03:00
Alexander Tokmakov
2c6a0e74fb better replica creation 2021-02-09 18:14:20 +03:00
alesapin
cb1194d37a Add final to some classes 2021-02-09 12:05:54 +03:00
Alexander Tokmakov
78c1d69b8c better code 2021-02-08 22:36:17 +03:00
tavplubix
ac477d9850
Merge pull request #19771 from ClickHouse/thread_state_improvements
Minor code improvements around ThreadStatus
2021-02-08 22:34:55 +03:00
Alexander Tokmakov
7ce0ef2561 show clusters for replicated db 2021-02-08 12:14:17 +03:00
Alexander Tokmakov
18f6b5bbad add timeouts 2021-02-04 22:41:44 +03:00
Alexander Tokmakov
066fb4c82b fix 2021-02-03 23:02:37 +03:00
Alexander Tokmakov
d010f97db0 Merge branch 'master' into database_replicated 2021-02-03 20:13:25 +03:00
tavplubix
50362840bb
Merge pull request #19924 from amosbird/ddlworker2
Initialize MaxDDLEntryID upon restarting
2021-02-03 13:22:22 +03:00
Alexander Tokmakov
0073c87d5d fix 2021-02-02 13:32:42 +03:00
tavplubix
37c178f5e1
Update DDLWorker.cpp 2021-02-02 10:32:01 +08:00
tavplubix
8e6b09b1ae
Update DDLWorker.cpp 2021-02-02 10:32:01 +08:00
Amos Bird
a7eb2ce6d0
initialize MaxDDLEntryId upon restarting 2021-02-02 10:32:01 +08:00
Alexander Tokmakov
9da445e740 execute initial query in the same thread 2021-02-01 22:29:47 +03:00
yiguolei
768e461d28 add functional test 2021-02-01 10:40:48 +08:00
Alexander Tokmakov
a57456a3fd fix 2021-01-28 22:02:39 +03:00
Alexander Tokmakov
ffaa8e34a6 minor code improvements around ThreadStatus 2021-01-28 16:57:36 +03:00
Alexander Tokmakov
52e5c0aad7 fix thread status 2021-01-28 16:48:17 +03:00
yiguolei
b0d645eea8 check active replicas and detached tables 2021-01-28 20:12:00 +08:00
yiguolei
6693f77c32 fix code style 2021-01-27 13:56:36 +08:00
guoleiyi
9d086f445d Should fail ddl query as soon as possible if table is shutdown 2021-01-27 10:58:07 +08:00
Alexander Tokmakov
f20d5e3b41 fix 2021-01-26 20:51:25 +03:00
Alexander Tokmakov
3146a1a954 fix 2021-01-25 21:59:23 +03:00
Alexander Tokmakov
769257acf4 fix drop table on cluster hang 2021-01-25 15:44:10 +03:00
Alexander Tokmakov
7f97a11c84 Merge branch 'master' into database_replicated 2021-01-18 17:09:39 +03:00
tavplubix
6022311a37
Merge pull request #18285 from weeds085490/hotfix/abort_when_failed_to_init_ddlworker
exit clickhouse when failing to initialize zookeeper with logical err…
2021-01-11 17:04:33 +03:00
tavplubix
213c653ef8
Merge pull request #17656 from bharatnc/ncb/ddl-worker-queue-table
add  system.distributed_ddl_queue table
2021-01-11 14:59:26 +03:00
bharatnc
1e579ac375 Fixes to code, build and style checks 2021-01-07 11:10:49 -08:00
bharatnc
f0ea07b493 DDLLogEntry.parse() to populate query and initiator 2021-01-07 11:10:41 -08:00
bharatnc
5af6f11325 DLLWorkerQueueTable - remove unimplemented function 2021-01-07 11:10:40 -08:00
bharatnc
2c1f9e2a77 DDLWorkerQueueTable - add intial scaffolding 2021-01-07 11:10:40 -08:00
Amos Bird
a32f6dcf1b
Apply alter settings on cluster to all replicas 2021-01-06 20:08:46 +08:00
Alexander Tokmakov
f2fca15393 try fix it better way 2020-12-30 15:25:00 +03:00
roverxu
f90719fef8 exit clickhouse when failing to initialize zookeeper with logical error in DDLWorker 2020-12-21 10:05:23 +08:00
Amos Bird
8d0e949462
ddlworker max entry metric 2020-12-12 14:20:49 +08:00
Alexander Tokmakov
9f3c77f62e add zk ops into task 2020-12-04 23:12:32 +03:00
Ilya Yatsishin
2e9f10ab3e
Merge pull request #17767 from bharatnc/ncb/refact-zk-path
Improve zookeeper path concatenation inside DDLWorker
2020-12-04 14:34:05 +03:00
Ivan
315ff4f0d9
ANTLR4 Grammar for ClickHouse and new parser (#11298) 2020-12-04 05:15:44 +03:00
bharatnc
a8fea625ae simplify path concat 2020-12-03 10:29:32 -08:00
Alexander Tokmakov
39532f7d9e slightly better DDLWorker initialization and restarting 2020-12-03 21:14:27 +03:00
bharatnc
d05d7e8b06 fix missing import 2020-12-03 09:08:17 -08:00
bharatnc
0efe16f9ce concatenate paths using fs::path 2020-12-03 09:04:24 -08:00
bharatnc
44e8e30bf1 removed unused imports 2020-12-03 08:43:22 -08:00
Alexander Tokmakov
1a4bd67736 fixes 2020-12-01 20:20:42 +03:00
Alexander Tokmakov
ab197a49c8 better code, fixes 2020-11-30 23:12:43 +03:00
Alexander Tokmakov
fe7c3cb865 fix race on is_circular_replicated 2020-11-30 18:31:02 +03:00
Alexander Tokmakov
f1a52a609b separate DatabaseReplicatedDDLWorker 2020-11-27 17:04:03 +03:00
Alexander Tokmakov
5ccf975a4a Merge branch 'master' into database_replicated 2020-11-24 14:38:02 +03:00
Alexander Tokmakov
dad21ee684 maintain metadata in zk 2020-11-24 13:24:39 +03:00
Alexander Tokmakov
7ab4445e99 try another approach 2020-11-20 19:06:27 +03:00
Alexander Tokmakov
2283906a11 try support replica recovery 2020-11-19 13:34:45 +03:00
alesapin
d4dbf928b4
Update src/Interpreters/DDLWorker.cpp
Co-authored-by: tavplubix <avtokmakov@yandex-team.ru>
2020-11-17 14:04:27 +03:00
alesapin
9f71f03be3 Fix DDL worker task execution on single replica 2020-11-16 15:34:12 +03:00
Alexander Tokmakov
b0262b3d06 better replica creation 2020-11-13 21:35:45 +03:00
Alexander Tokmakov
0731bfb7be Merge branch 'master' into database_replicated 2020-11-12 21:04:18 +03:00
Alexander Tokmakov
2a6c0b9180 try reuse DDLWorker in DatabaseReplicated 2020-11-12 20:35:29 +03:00
Alexander Tokmakov
cbcdee0cf9 split DDLWorker.cpp 2020-11-03 16:47:26 +03:00
Alexander Tokmakov
d8ae9fcdb4 fixes, add shard name 2020-11-03 14:30:53 +03:00
Alexey Milovidov
f573aeb97e Whitespace 2020-10-28 03:32:31 +03:00
alesapin
dc1a5abea9 Remove cluster type check during alter 2020-10-16 14:59:58 +03:00
alesapin
cfe9a0c4a1 Fix event subscription in DDLWorker 2020-09-30 17:58:27 +03:00
Azat Khuzhin
c9baceb760 Change thread name for DDLWorker executor threads 2020-09-17 21:07:14 +03:00
Azat Khuzhin
dd867b787f Allow parallel execution of distributed DDL
Add distributed_ddl.pool_size to control maximum parallel to handle
distributed DDL.

Also:
- convert Exception constructors to fmt-like
- use sleepFor* over std::this_thread::sleep_for()
2020-09-12 02:32:08 +03:00
Winter Zhang
3d1d64ec60
Fix currentDatabase function cannot be used in ON CLUSTER ddl query. (#14211) 2020-09-09 12:58:59 +03:00
alesapin
0177b35998 Review fixes 2020-08-24 12:07:37 +03:00
alesapin
f5730a3a5f Merge branch 'master' into fix_ddl_worker_timeouts 2020-08-24 11:50:11 +03:00
Alexey Milovidov
eeb769d2d4 Remove useless code around zkutil 2020-08-18 22:02:07 +03:00
Alexey Milovidov
12f66fa82c Fix 99% of typos 2020-08-08 04:01:47 +03:00
alesapin
77b8c9e332 Fix DDL worker 2020-08-07 12:18:34 +03:00
Vitaly Baranov
c39eb8f71b Fix partial revokes (complex cases). 2020-06-30 18:47:02 +03:00
Alexey Milovidov
5211a42c04 Remove leader election, step 3 2020-06-19 17:18:58 +03:00
Nikita Mikhaylov
edff6eac6e comment 2020-06-17 11:44:52 +03:00