Commit Graph

106206 Commits

Author SHA1 Message Date
Robert Schulze
531e64dc39
Merge branch 'master' into inv-index-cleanup 2023-01-22 13:20:38 +01:00
Azat Khuzhin
ca8af3d18c Fix possible in-use table after DETACH
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>
2023-01-22 13:18:56 +01:00
Alexander Gololobov
91fb1eab8c
Merge pull request #45373 from ClickHouse/throttler-metrics
Add detailed profile events for throttling
2023-01-22 11:20:45 +01:00
Alexander Tokmakov
1174eaa132
Merge pull request #45492 from azat/revert/dist/async-INSERT-metrics
Revert "Merge pull request #44922 from azat/dist/async-INSERT-metrics"
2023-01-22 00:45:10 +03:00
Azat Khuzhin
a55798626a Revert "Merge pull request #44922 from azat/dist/async-INSERT-metrics"
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.
2023-01-21 22:42:00 +01:00
Maksim Kita
19f1bae5ed
Merge pull request #45254 from kitaisreal/planner-small-fixes
Planner small fixes
2023-01-21 19:54:17 +03:00
Maksim Kita
6594aa8f1e
Merge pull request #44234 from kitaisreal/analyzer-expired-context-crash-fix
Analyzer expired Context crash fix
2023-01-21 18:49:50 +03:00
Azat Khuzhin
2a8f116c18 Forward declaration of ConcurrentBoundedQueue in ThreadStatus
ThreadStatus is the header that recomplies almost all ClickHouse
modules.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-01-21 16:02:09 +01:00
Vitaly Baranov
5ceb64accc Use new copy s3 functions in S3ObjectStorage. 2023-01-21 15:47:58 +01:00
Vitaly Baranov
f0fda580d0
Merge pull request #45188 from vitlibar/backup-to-s3-memory-optimization
Optimize memory consumption during backup to S3
2023-01-21 12:37:35 +01:00
Maksim Kita
5146087db1 Fix style 2023-01-21 12:19:09 +01:00
Nikolai Kochetov
b877c484d2
Merge pull request #45481 from ClickHouse/fix-deadlock-with-allow_asynchronous_read_from_io_pool_for_merge_tree
Fix possible deadlock with allow_asynchronous_read_from_io_pool_for_merge_tree in case of exception from ThreadPool::schedule
2023-01-21 12:05:34 +01:00
Nikolai Kochetov
ec1e2436cc
Merge pull request #45450 from ClickHouse/fix-disabled-two-level-agg
Fix disabled two-level aggregation from HTTP
2023-01-21 12:01:59 +01:00
Sema Checherinda
962894afc8
Merge pull request #44909 from CheSema/intersect-prev-part
Do not merge over a gap with outdated undeleted parts
2023-01-21 11:51:21 +01:00
Maksim Kita
47385a19e7 Remove unnecessary getTotalRowCount function calls 2023-01-21 11:27:07 +01:00
Azat Khuzhin
a64f6b5f3e Fix possible (likely distributed) query hung
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>
2023-01-21 08:05:56 +01:00
Azat Khuzhin
e2fcf0f072 Catch exception on query cancellation
Since we still want to join the thread, yes it will be done in dtor, but
this looks better.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-01-21 08:05:56 +01:00
Azat Khuzhin
0566f72d36 Cleanup PullingAsyncPipelineExecutor::cancel()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-01-21 08:05:56 +01:00
Anton Popov
41a199e175
Fix crash when ListObjects request fails (#45371) 2023-01-20 20:10:23 +01:00
Nikolai Kochetov
dcd84c152a Fix possible deadlock with allow_asynchronous_read_from_io_pool_for_merge_tree in case of exception from ThreadPool::schedule 2023-01-20 18:57:47 +00:00
Han Fei
4bbe90f6b4
Merge pull request #45473 from hanfei1991/hanfei/async-inserts-doc
update docs for async insert deduplication
2023-01-20 18:25:19 +01:00
Han Fei
18a397f8c9 address comments 2023-01-20 18:09:42 +01:00
Han Fei
449ace3373
Update docs/en/operations/settings/settings.md
Co-authored-by: Dan Roscigno <dan@roscigno.com>
2023-01-20 18:07:19 +01:00
Han Fei
9d87bd10ee
Update docs/en/operations/settings/settings.md
Co-authored-by: Dan Roscigno <dan@roscigno.com>
2023-01-20 18:07:08 +01:00
Han Fei
badfbcb3d8
Update docs/en/operations/settings/settings.md
Co-authored-by: Dan Roscigno <dan@roscigno.com>
2023-01-20 18:06:58 +01:00
Han Fei
e9c4cf46cd
Update docs/en/operations/settings/merge-tree-settings.md
Co-authored-by: Dan Roscigno <dan@roscigno.com>
2023-01-20 18:06:46 +01:00
Han Fei
9d254f7d87
Update docs/en/operations/settings/merge-tree-settings.md
Co-authored-by: Dan Roscigno <dan@roscigno.com>
2023-01-20 18:06:32 +01:00
Nikolay Degterinsky
c2ac4fece0 Add more retries to AST Fuzzer 2023-01-20 16:46:24 +00:00
robot-ch-test-poll4
2066581d8f
Merge pull request #45451 from evillique/default_granularity
Add default GRANULARITY argument for secondary indexes
2023-01-20 17:46:21 +01:00
avogar
86336940f8 Better comment 2023-01-20 16:41:59 +00:00
avogar
4432ee9927 Fix aborts in arrow lib 2023-01-20 16:40:33 +00:00
vdimir
e30ab0874b
Review fixes 2023-01-20 16:30:34 +00:00
Alexander Tokmakov
910d6dc0ce
Merge pull request #45342 from ClickHouse/exception_message_patterns
Save message format strings for DB::Exception
2023-01-20 18:46:52 +03:00
Nikita Mikhaylov
9ab454366c Better test 2023-01-20 16:40:01 +01:00
Kseniia Sumarokova
01320da02b
Update BoundedReadBuffer.cpp 2023-01-20 16:25:02 +01:00
ltrk2
810c9ba50c Produce a null map of the correct size 2023-01-20 10:24:42 -05:00
ltrk2
9d798ea1bc Document functions 2023-01-20 10:24:42 -05:00
ltrk2
65b9c69c90 Introduce non-throwing variants of hasToken 2023-01-20 10:24:42 -05:00
Nikolay Degterinsky
02142596fb Add docs 2023-01-20 15:22:13 +00:00
avogar
550a703fbc Make a bit better 2023-01-20 14:58:39 +00:00
Antonio Andelic
136e4ec1b3
Merge pull request #45273 from azat/fix-test-log-level
Fix log level "Test" for send_logs_level in client
2023-01-20 15:36:05 +01:00
Alexander Tokmakov
ec5d7d0a3a
Update src/Functions/FunctionsConversion.h
Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com>
2023-01-20 17:33:01 +03:00
Kruglov Pavel
28ddcc2432
Merge branch 'master' into tsv-csv-detect-header 2023-01-20 15:08:38 +01:00
Robert Schulze
b55c8ddd65
Merge pull request #45472 from ClickHouse/rs-duplicate-writing-guide
Don't duplicate writing guide, instead point to existing writing guide
2023-01-20 15:05:13 +01:00
Sema Checherinda
b76b612d23
fix typo 2023-01-20 14:55:58 +01:00
Nikolai Kochetov
039901b395 Fixing build 2023-01-20 13:49:50 +00:00
Han Fei
5fc4998f10 update docs for async insert deduplication 2023-01-20 14:42:11 +01:00
Robert Schulze
4ac17d71fa
Merge pull request #45470 from ClickHouse/rs-doc-typos
Fix typos
2023-01-20 14:39:27 +01:00
Robert Schulze
3b182e0fec
Don't duplicate writing guide, instead point to existing writing guide 2023-01-20 13:36:31 +00:00
Robert Schulze
3f2e4c8217
Fix typos 2023-01-20 13:20:25 +00:00