Commit Graph

131837 Commits

Author SHA1 Message Date
koloshmet
238741dafe fixed style 2023-12-28 17:56:04 +00:00
koloshmet
4305457883 fixed tests 2023-12-28 17:56:04 +00:00
koloshmet
67e469bee5 refreshable view query test 2023-12-28 17:56:04 +00:00
koloshmet
c52aa984ee refreshable materialized views 2023-12-28 17:56:04 +00:00
Alexey Milovidov
a9ac8dfb74 Update CHANGELOG.md 2023-12-28 18:31:15 +01:00
Nikita Mikhaylov
e15b1c6e5f Fixed 2023-12-28 17:25:27 +00:00
Alexander Tokmakov
95e4b0002f fix a bug in PartsSplitter 2023-12-28 17:25:36 +01:00
Kseniia Sumarokova
8e8fd84cb7
Merge pull request #58293 from ClickHouse/fix-s3-queue-test
Fix test_storage_s3_queue/test.py::test_drop_table
2023-12-28 17:18:11 +01:00
Azat Khuzhin
ecf7188d52 Fix use-after-free in KafkaConsumer due to statistics callback
CI founds [1]:

    Exception: Sanitizer assert found for instance �=================================================================
    ==1==ERROR: AddressSanitizer: heap-use-after-free on address 0x5250006a4100 at pc 0x55d4ed46d2e2 bp 0x7f7e33b40190 sp 0x7f7e33b3f950
    WRITE of size 5390 at 0x5250006a4100 thread T2 (TCPHandler)
       8 0x55d50eba9497 in DB::KafkaConsumer::setRDKafkaStat(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&) build_docker/./src/Storages/Kafka/KafkaConsumer.h:117:22
       12 0x55d51e0eebfe in cppkafka::stats_callback_proxy(rd_kafka_s*, char*, unsigned long, void*) build_docker/./contrib/cppkafka/src/configuration.cpp:92:5
       13 0x55d51e151e3d in rd_kafka_poll_cb build_docker/./contrib/librdkafka/src/rdkafka.c:3790:7
       14 0x55d51e15531b in rd_kafka_consumer_close build_docker/./contrib/librdkafka/src/rdkafka.c:3200:31
       15 0x55d51e0f3241 in cppkafka::Consumer::close() build_docker/./contrib/cppkafka/src/consumer.cpp:293:33
       16 0x55d51e0f3241 in cppkafka::Consumer::~Consumer() build_docker/./contrib/cppkafka/src/consumer.cpp:82:9
       20 0x55d50eb8d12e in DB::KafkaConsumer::~KafkaConsumer() build_docker/./src/Storages/Kafka/KafkaConsumer.cpp:179:1

    0x5250006a4100 is located 0 bytes inside of 8736-byte region [0x5250006a4100,0x5250006a6320)
    freed by thread T2 (TCPHandler) here:
       0 0x55d4ed4a26b2 in operator delete(void*, unsigned long) (/usr/bin/clickhouse+0xa94b6b2) (BuildId: 74ec4a14a5109c41de109e82d56d8d863845144d)
       1 0x55d50eb8ca55 in void std::__1::__libcpp_operator_delete[abi:v15000]<void*, unsigned long>(void*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:256:3
       2 0x55d50eb8ca55 in void std::__1::__do_deallocate_handle_size[abi:v15000]<>(void*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:282:10
       3 0x55d50eb8ca55 in std::__1::__libcpp_deallocate[abi:v15000](void*, unsigned long, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/new:296:14
       4 0x55d50eb8ca55 in std::__1::allocator<char>::deallocate[abi:v15000](char*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/__memory/allocator.h:128:13
       5 0x55d50eb8ca55 in std::__1::allocator_traits<std::__1::allocator<char>>::deallocate[abi:v15000](std::__1::allocator<char>&, char*, unsigned long) build_docker/./contrib/llvm-project/libcxx/include/__memory/allocator_traits.h:282:13
       6 0x55d50eb8ca55 in std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>::~basic_string() build_docker/./contrib/llvm-project/libcxx/include/string:2334:9
       7 0x55d50eb8ca55 in DB::KafkaConsumer::~KafkaConsumer() build_docker/./src/Storages/Kafka/KafkaConsumer.cpp:179:1

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/0/745d9bb47f3425e28e5660ed7c730038ffece4ee/integration_tests__asan__analyzer__%5B6_6%5D/integration_run_parallel4_0.log

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-12-28 15:48:43 +01:00
Dmitry Novik
50e531bf93 Improve system.errors documentation 2023-12-28 14:47:30 +00:00
Azat Khuzhin
4a14112af1 Move StorageKafka::createConsumer() into KafkaConsumer
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit ebad1bf4f3)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
87f3f6619a Fix data-race between StorageKafka::startup() and cleanConsumers()
Actually now we can create consumer object in the ctor, no need to do
this in startup(), since consumer now do not connects to kafka.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 03218202d3)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
3c139d7135 Update comment for statistics.interval.ms librdkafka option
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 1f03a21033)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
6f85306510 Use separate thread for kafka consumers cleanup
Since pool may exceed threads, while we need to run this thread always
to avoid memory leaking.

And this should not be a problem since librdkafka has multiple threads
for each consumer (5!) anyway.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 06a9e9a9ca)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
8ac68b64d7 Allow setThreadName() to truncate thread name instead of throw an error
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit a7453f7f14)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
7d2b82c37c Add ability to configure TTL for kafka consumers
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit b19b70b8fc)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
bea1610219 Preserve KafkaConsumer objects
This will make system.kafka_consumers more useful, since after TTL
consumer object will be removed prio this patch, but after, all
information will be preserved.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 2ff0bfb0a1)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
71fdde76c2 Enable stats for system.kafka_consumers back by default
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit db74549940)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
d66be02dc3 Create consumers for Kafka tables on fly (but keep them for 1min since last used)
Pool of consumers created a problem for librdkafka internal statistics,
you need to read from the queue always, while in ClickHouse consumers
created regardless are there any readers or not (attached materialized
views or direct SELECTs).

Otherwise, this statistics messages got queued and never released,
which:
- creates live memory leak
- and also makes destroy very slow, due to librdkafka internals (it
  moves entries from this queue into another linked list, but in a
  with sorting, which is incredibly slow for linked lists)

So the idea is simple, let's create a pool of consumers only when they
are required, and destroy them after some timeout (right now it is 60
seconds) if nobody uses them, that way this problem should gone.

This should also reduce number of internal librdkafka threads, when
nobody reads from Kafka tables.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit e7592c140e)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
a6841c8915 Properly set shutdown_called in StorageKafka::shutdown()
Fixes: https://github.com/ClickHouse/ClickHouse/pull/42777
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 51d4f583e6)
2023-12-28 15:32:39 +01:00
Azat Khuzhin
3541d9a05f Remove StorageKafka::num_created_consumers (in favor of all_consumers.size())
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit 123d63e824)
2023-12-28 15:32:39 +01:00
vdimir
e20cb2d052
Disable max_bytes_before_external* in 00172_hits_joins 2023-12-28 13:35:34 +00:00
Antonio Andelic
8ace9978b3 Handle another case for preprocessing 2023-12-28 13:20:47 +00:00
Alexander Tokmakov
bdada351c8 Revert "Merge pull request #58274 from ClickHouse/revert-58267"
This reverts commit 583b9637c2, reversing
changes made to 224e937620.
2023-12-28 14:07:59 +01:00
Alexander Tokmakov
5fcbf9cfb0 Revert "Merge pull request #58251 from ClickHouse/reintroduce-compatibility-with-a-misfeature"
This reverts commit a811d5b761, reversing
changes made to 583b9637c2.
2023-12-28 14:06:56 +01:00
vdimir
017e4998a1
Fix timeouts in 01732_race_condition_storage_join_long 2023-12-28 12:55:51 +00:00
Alexander Tokmakov
d58b76ce06
Merge pull request #58296 from ClickHouse/revert-56946-mv
Revert "Refreshable materialized views (takeover)"
2023-12-28 13:33:22 +01:00
Alexander Tokmakov
38fe70c68a
Revert "Refreshable materialized views (takeover)" 2023-12-28 13:12:20 +01:00
kssenii
529a1d67cb Fix 2023-12-28 12:48:56 +01:00
Alexey Milovidov
c6731df0cf Update changelog 2023-12-28 12:13:43 +01:00
Alexey Milovidov
09352205d8
Merge pull request #58288 from ClickHouse/fix-release
Fix an error in the release script - it didn't allow to make 23.12.
2023-12-28 12:09:23 +01:00
Alexey Milovidov
0c57260ad3
Merge pull request #58290 from ClickHouse/auto/v23.12.1.1368-stable
Update version_date.tsv and changelogs after v23.12.1.1368-stable
2023-12-28 12:09:12 +01:00
Duc Canh Le
238c5e66d5 use ChunkInfo to carry part level
Signed-off-by: Duc Canh Le <duccanh.le@ahrefs.com>
2023-12-28 11:01:18 +00:00
robot-clickhouse
27657a3eb6 Update version_date.tsv and changelogs after v23.12.1.1368-stable 2023-12-28 10:32:09 +00:00
Alexey Milovidov
4dbd07fbb9
Merge pull request #58286 from ClickHouse/23.13-prepare
Update version after release
2023-12-28 11:27:58 +01:00
Alexey Milovidov
4bb8592434 Update autogenerated version to 23.13.1.1 and contributors 2023-12-28 11:22:16 +01:00
Alexey Milovidov
9d1e124102 Let me do the release 2023-12-28 11:21:58 +01:00
Alexey Milovidov
d764d8d72b Update Changelog 2023-12-28 11:19:16 +01:00
vdimir
f53281a823
fix 02862_index_inverted_incorrect_args 2023-12-28 10:00:05 +00:00
Alexey Milovidov
9f895fd038 Add a new item into the changelog 2023-12-28 09:33:01 +01:00
Alexey Milovidov
a2faa65b08
Merge pull request #56946 from ClickHouse/mv
Refreshable materialized views (takeover)
2023-12-28 09:29:44 +01:00
Alexey Milovidov
3f10ad7573 Remove reverted item from the changelog 2023-12-28 07:16:57 +01:00
Alexey Milovidov
643dddb619
Merge pull request #58277 from ClickHouse/revert-57969-punycode-is-cool
Revert "Implement punycode encoding/decoding"
2023-12-28 07:12:36 +01:00
Alexey Milovidov
345d29a3c1
Revert "Implement punycode encoding/decoding" 2023-12-28 09:12:22 +03:00
Alexey Milovidov
a719c67bd0 Remove reverted item from the changelog 2023-12-28 07:03:45 +01:00
Duc Canh Le
e622cf9ee9 address review comments
Signed-off-by: Duc Canh Le <duccanh.le@ahrefs.com>
2023-12-28 05:01:55 +00:00
Alexey Milovidov
524d53199d
Merge branch 'master' into mv 2023-12-28 04:11:48 +01:00
Alexey Milovidov
a811d5b761
Merge pull request #58251 from ClickHouse/reintroduce-compatibility-with-a-misfeature
Reintroduce compatibility with `is_deleted` on a syntax level
2023-12-28 04:11:04 +01:00
Alexey Milovidov
583b9637c2
Merge pull request #58274 from ClickHouse/revert-58267
Revert #58267
2023-12-28 04:10:18 +01:00
Alexey Milovidov
c7efd2afea Revert #58267 2023-12-28 04:09:33 +01:00