Right now in case of DETACH/ATTACH there can be a window when after the
table had been DETACH'ed someone will still use it, the common example
here is MVs handling.
It happens because TableExclusiveLockHolder does not guards the
shard_ptr of the IStorage, and so if someone holds it, then it can use
it. So if ATTACH will be done for this table then, you can have multiple
instances of it.
It is not possible for DROP, because before using a table, you should
lock it and after table had been DROP'ed you cannot lock it anymore.
So let's do the same for DETACH.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
There are the following problems with this patch:
- Looses files on exception
- Existing current_batch.txt on startup leads to ENOENT error and hung
of distributed sends without ATTACH/DETACH
- Race between creating the queue for sending at table startup and
INSERT, if it had been created from INSERT, then it will not be
initialized from disk
They were addressed in #45491, but it makes code more cmoplex and plus
since, likely, the release is comming, it is better to revert the
change.
This reverts commit 94604f71b7, reversing
changes made to 80f6a45376.
Recently I saw the following, the client executed long distributed query
and terminated the connection, and in this case query cancellation will
be done from PullingAsyncPipelineExecutor dtor, but during cancellation
one of nodes sent ECONNRESET, and this leads to an exception from
PullingAsyncPipelineExecutor::cancel(), and this leads to a deadlock
when multiple threads waits each others, because cancel() for
LazyOutputFormat wasn't called.
Here is as relevant portion of logs:
2023.01.04 08:26:09.236208 [ 37968 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Debug> executeQuery: (from 10.61.13.253:44266, user: default) TooLongDistributedQueryToPost
...
2023.01.04 08:26:09.262424 [ 37968 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Trace> MergeTreeInOrderSelectProcessor: Reading 1 ranges in order from part 9_330_538_18, approx. 61440 rows starting from 0
2023.01.04 08:26:09.266399 [ 26788 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Trace> Connection (s4.ch:9000): Connecting. Database: (not specified). User: default
2023.01.04 08:26:09.266849 [ 26788 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Trace> Connection (s4.ch:9000): Connected to ClickHouse server version 22.10.1.
2023.01.04 08:26:09.267165 [ 26788 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Debug> Connection (s4.ch:9000): Sent data for 2 scalars, total 2 rows in 3.1587e-05 sec., 62635 rows/sec., 68.00 B (2.03 MiB/sec.), compressed 0.4594594594594595 times to 148.00 B (4.41 MiB/sec.)
2023.01.04 08:39:13.047170 [ 37968 ] {f2ed6149-146d-4a3d-874a-b0b751c7b567} <Error> PullingAsyncPipelineExecutor: Code: 210. DB::NetException: Connection reset by peer, while writing to socket (10.7.142.115:9000). (NETWORK_ERROR), Stack trace (when copying this message, always include the lines below):
0. ./.build/./contrib/libcxx/include/exception:133: Poco::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0x1818234c in /usr/lib/debug/usr/bin/clickhouse.debug
1. ./.build/./src/Common/Exception.cpp:69: DB::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int, bool) @ 0x1004fbda in /usr/lib/debug/usr/bin/clickhouse.debug
2. ./.build/./src/Common/NetException.h:12: DB::WriteBufferFromPocoSocket::nextImpl() @ 0x14e352f3 in /usr/lib/debug/usr/bin/clickhouse.debug
3. ./.build/./src/IO/BufferBase.h:39: DB::Connection::sendCancel() @ 0x15c21e6b in /usr/lib/debug/usr/bin/clickhouse.debug
4. ./.build/./src/Client/MultiplexedConnections.cpp:0: DB::MultiplexedConnections::sendCancel() @ 0x15c4d5b7 in /usr/lib/debug/usr/bin/clickhouse.debug
5. ./.build/./src/QueryPipeline/RemoteQueryExecutor.cpp:627: DB::RemoteQueryExecutor::tryCancel(char const*, std::__1::unique_ptr<DB::RemoteQueryExecutorReadContext, std::__1::default_delete<DB::RemoteQueryExecutorReadContext> >*) @ 0x14446c09 in /usr/lib/debug/usr/bin/clickhouse.debug
6. ./.build/./contrib/libcxx/include/__iterator/wrap_iter.h💯 DB::ExecutingGraph::cancel() @ 0x15d2c0de in /usr/lib/debug/usr/bin/clickhouse.debug
7. ./.build/./contrib/libcxx/include/__memory/unique_ptr.h:300: DB::PullingAsyncPipelineExecutor::cancel() @ 0x15d32055 in /usr/lib/debug/usr/bin/clickhouse.debug
8. ./.build/./contrib/libcxx/include/__memory/unique_ptr.h:312: DB::PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() @ 0x15d31f4f in /usr/lib/debug/usr/bin/clickhouse.debug
9. ./.build/./src/Server/TCPHandler.cpp:0: DB::TCPHandler::processOrdinaryQueryWithProcessors() @ 0x15cde919 in /usr/lib/debug/usr/bin/clickhouse.debug
10. ./.build/./src/Server/TCPHandler.cpp:0: DB::TCPHandler::runImpl() @ 0x15cd8554 in /usr/lib/debug/usr/bin/clickhouse.debug
11. ./.build/./src/Server/TCPHandler.cpp:1904: DB::TCPHandler::run() @ 0x15ce6479 in /usr/lib/debug/usr/bin/clickhouse.debug
12. ./.build/./contrib/poco/Net/src/TCPServerConnection.cpp:57: Poco::Net::TCPServerConnection::start() @ 0x18074f07 in /usr/lib/debug/usr/bin/clickhouse.debug
13. ./.build/./contrib/libcxx/include/__memory/unique_ptr.h:54: Poco::Net::TCPServerDispatcher::run() @ 0x180753ed in /usr/lib/debug/usr/bin/clickhouse.debug
14. ./.build/./contrib/poco/Foundation/src/ThreadPool.cpp:213: Poco::PooledThread::run() @ 0x181e3807 in /usr/lib/debug/usr/bin/clickhouse.debug
15. ./.build/./contrib/poco/Foundation/include/Poco/SharedPtr.h:156: Poco::ThreadImpl::runnableEntry(void*) @ 0x181e1483 in /usr/lib/debug/usr/bin/clickhouse.debug
16. ? @ 0x7ffff7e55fd4 in ?
17. ? @ 0x7ffff7ed666c in ?
(version 22.10.1.1)
And here is the state of the threads:
<details>
<summary>system.stack_trace</summary>
```sql
SELECT
arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') AS sym
FROM system.stack_trace
WHERE query_id = 'f2ed6149-146d-4a3d-874a-b0b751c7b567'
SETTINGS allow_introspection_functions=1
Row 1:
──────
sym:
pthread_cond_wait
std::__1::condition_variable::wait(std::__1::unique_lock<std::__1::mutex>&)
bool ConcurrentBoundedQueue<DB::Chunk>::emplaceImpl<DB::Chunk>(std::__1::optional<unsigned long>, DB::Chunk&&)
DB::IOutputFormat::work()
DB::ExecutionThreadContext::executeTask()
DB::PipelineExecutor::executeStepImpl(unsigned long, std::__1::atomic<bool>*)
Row 2:
──────
sym:
pthread_cond_wait
Poco::EventImpl::waitImpl()
DB::PipelineExecutor::joinThreads()
DB::PipelineExecutor::executeImpl(unsigned long)
DB::PipelineExecutor::execute(unsigned long)
Row 3:
──────
sym:
pthread_cond_wait
Poco::EventImpl::waitImpl()
DB::PullingAsyncPipelineExecutor::Data::~Data()
DB::PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor()
DB::TCPHandler::processOrdinaryQueryWithProcessors()
DB::TCPHandler::runImpl()
DB::TCPHandler::run()
Poco::Net::TCPServerConnection::start()
Poco::Net::TCPServerDispatcher::run()
Poco::PooledThread::run()
Poco::ThreadImpl::runnableEntry(void*)
```
</details>
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>