Commit Graph

48 Commits

Author SHA1 Message Date
Yury Bogomolov
c8b128aad4 add predefined macros support in kafka engine settings 2023-07-21 01:06:49 +04:00
Antonio Andelic
b11f744252
Correctly disable async insert with deduplication when it's not needed (#50663)
* Correctly disable async insert when it's not used

* Better

* Add comment

* Better

* Fix tests

---------

Co-authored-by: Nikita Mikhaylov <mikhaylovnikitka@gmail.com>
2023-06-07 20:33:08 +02:00
Robert Schulze
81bf43157f
Allow configuration of Kafka topics with periods
The Kafka table engine allows global configuration and per-Kafka-topic
configuration. The latter uses syntax <kafka_TOPIC>, e.g. for topic
"football":

  <kafka_football>
      <retry_backoff_ms>250</retry_backoff_ms>
      <fetch_min_bytes>100000</fetch_min_bytes>
  </kafka_football>

Some users had to find out the hard way that such configuration doesn't
take effect if the topic name contains a period, e.g. "sports.football".
The reason is that ClickHouse configuration framework already uses
periods as level separators to descend the configuration hierarchy.
(Besides that, per-topic configuration at the same level as global
configuration could be considered ugly.)

Note that Kafka topics may contain characters "a-zA-Z0-9._-" (*) and
a tree-like topic organization using periods is quite common in
practice.

This PR deprecates the existing per-topic configuration syntax (but
continues to support it for backward compat) and introduces a new
per-topic configuration syntax below the global Kafka configuration of
the form:

<kafka>
   <topic name="football">
       <retry_backoff_ms>250</retry_backoff_ms>
       <fetch_min_bytes>100000</fetch_min_bytes>
   </topic>
</kafka>

The period restriction doesn't apply to XML attributes, so <topic
name="sports.football"> will work. Also, everything Kafka-related is
below <kafka>.

Considered but rejected alternatives:
- Extending Poco ConfigurationView with custom separators (e.g."/"
  instead of "."). Won't work easily because ConfigurationView only
  builds a path but defers descending the configuration tree to the
  normal configuration classes.
- Reloading the configuration file in StorageKafka (instead of reading
  the loaded file) but with a custom separator. This mode is supported
  by XML configuration. Too ugly and error-prone since the true
  configuration is composed from multiple configuration files.

(*) https://stackoverflow.com/a/37067544
2023-02-22 20:35:09 +00:00
Kruglov Pavel
b124875257
Merge branch 'master' into improve-streaming-engines 2022-11-03 13:22:06 +01:00
avogar
8e13d1f1ec Improve and refactor Kafka/StorageMQ/NATS and data formats 2022-10-28 16:41:10 +00:00
Azat Khuzhin
4e76629aaf Fixes for -Wshorten-64-to-32
- lots of static_cast
- add safe_cast
- types adjustments
  - config
  - IStorage::read/watch
  - ...
- some TODO's (to convert types in future)

P.S. That was quite a journey...

v2: fixes after rebase
v3: fix conflicts after #42308 merged
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-10-21 13:25:19 +02:00
Robert Schulze
45c0ec44b0
Fix build, pt. II 2022-05-04 10:15:25 +02:00
Robert Schulze
777b5bc15b
Don't let storages inherit from boost::noncopyable
... IStorage has deleted copy ctor / assignment already
2022-05-03 09:07:08 +02:00
Robert Schulze
330212e0f4
Remove inherited create() method + disallow copying
The original motivation for this commit was that shared_ptr_helper used
std::shared_ptr<>() which does two heap allocations instead of
make_shared<>() which does a single allocation. Turned out that
1. the affected code (--> Storages/) is not on a hot path (rendering the
performance argument moot ...)
2. yet copying Storage objects is potentially dangerous and was
   previously allowed.

Hence, this change

- removes shared_ptr_helper and as a result all inherited create() methods,

- instead, Storage objects are now created using make_shared<>() by the
  caller (for that to work, many constructors had to be made public), and

- all Storage classes were marked as noncopyable using boost::noncopyable.

In sum, we are (likely) not making things faster but the code becomes
cleaner and harder to misuse.
2022-05-02 08:46:52 +02:00
Anton Popov
0ba78c3c3a Merge remote-tracking branch 'upstream/master' into HEAD 2022-03-16 15:28:09 +00:00
Maksim Kita
2fdcf53a76 Fix clang-tidy warnings in Server, Storages folders 2022-03-14 18:17:35 +00:00
Anton Popov
99ebabd822 Merge remote-tracking branch 'upstream/master' into HEAD 2021-12-17 19:02:29 +03:00
Anton Popov
6f4d9a53b2 Merge remote-tracking branch 'origin/sparse-serialization' into HEAD 2021-12-01 15:54:33 +03:00
kssenii
2ea4e7c4e7 Better 2021-11-29 15:05:02 +00:00
kssenii
980c87c466 Add named collection for kafka / rabbit 2021-11-24 14:53:01 +03:00
Anton Popov
a20922b2d3 Merge remote-tracking branch 'origin/sparse-serialization' into HEAD 2021-11-09 15:36:25 +03:00
Alexey Milovidov
fe6b7c77c7 Rename "common" to "base" 2021-10-02 10:13:14 +03:00
Anton Popov
e36736b50c Merge remote-tracking branch 'origin/sparse-serialization' into HEAD 2021-08-02 22:52:02 +03:00
Nikolai Kochetov
2dc5c89b66 Update Storage::write 2021-07-23 17:25:35 +03:00
Anton Popov
3ed7f5a6cc dynamic subcolumns: add snapshot for storage 2021-07-09 06:15:41 +03:00
Maksim Kita
67e9b85951 Merge ext into common 2021-06-16 23:28:41 +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
Peng Jian
85cfb927e6 move HandleKafkaErrorMode to KafkaSettings 2021-03-31 17:16:51 +08:00
Peng Jian
909d5ad2b5 Handle errors for Kafka engine 2021-03-31 17:15:57 +08:00
Azat Khuzhin
64cb6405ac Drop IStorage::supportsSettings() (replaced with StorageFeatures::supports_settings) 2021-01-08 14:36:26 +03:00
Amos Bird
1d9d586e20
Make global_context consistent. 2020-11-20 18:23:14 +08:00
Alexey Milovidov
5314185e25 Merge branch 'master' into azat-optimize_skip_unused_shards-optimization 2020-11-08 00:17:59 +03:00
Azat Khuzhin
1f51de362f Take memory allocations from librdkafka threads into account 2020-10-07 23:18:20 +03:00
Azat Khuzhin
3cd71f3357 Cleanup interceptors for librdkafka in StorageKafka
Wrap them into StorageKafkaInterceptors to allow access to private
fields and add logging inside interceptors if something fails.

This is also preparation for ThreadStatus interceptor.
2020-10-07 21:42:13 +03:00
Azat Khuzhin
b838214a35 Pass non-const SelectQueryInfo (and drop mutable qualifiers) 2020-10-02 22:42:35 +03:00
Peng Jian
fd82272cfe Fix code style, and update docs for Kafka engine 2020-09-06 11:17:19 +08:00
Peng Jian
e6bfd9d586 1. Add new setting for Kafka engine, named kafka_thread_per_consumer which default value is false. 2. Create separate thread pool for Kafka engine. 2020-09-06 11:17:19 +08:00
Peng Jian
de0a40aedc fix code style 2020-09-06 11:17:19 +08:00
Peng Jian
61ac52cf29 Multithreading implementation of Kafka engine 2020-09-06 11:17:18 +08:00
Nikolai Kochetov
09fbce1b1e Merge branch 'master' into refactor-pipes-3 2020-08-04 11:32:34 +03:00
Nikolai Kochetov
e411916bde Refactor Pipe [part 1]. 2020-08-03 14:33:11 +03:00
Vitaly Baranov
90602b869a Make SettingsChanges a class. 2020-07-31 19:11:27 +03:00
Mikhail Filimonov
93517b4e82 Same change for Kafka - just in case, and to make it conform. 2020-07-06 20:48:33 +02:00
alesapin
36ba0192df Metadata in read and write methods of IStorage 2020-06-15 22:08:58 +03:00
Mikhail Filimonov
94261b9786 Fix issues detected by clang tidy, fix issue with flush timeout 2020-06-03 19:02:24 +02:00
Mikhail Filimonov
95f4c46aa4 Better control of context copies, some cosmetic changes, client.software.name and client.software.version now initialized 2020-06-03 19:01:45 +02:00
Mikhail Filimonov
4956ade67d Support for all format settings in Kafka, expose some setting on table level, better defaults. 2020-06-03 19:01:41 +02:00
Mikhail Filimonov
b97ff8136b Make function static 2020-05-28 20:10:54 +02:00
Mikhail Filimonov
6b694d54b4 Better kafka client.id 2020-05-28 15:39:55 +02:00
madianjun
e3c7ab2ca7 Add Kafka client.id 2020-05-28 08:32:51 +02:00
alesapin
f981649213 Fix pushing to views stream and refactor virtuals 2020-04-28 13:38:57 +03:00
alesapin
18c550df15 Better virtuals logic 2020-04-27 16:55:30 +03:00
Ivan Lezhankin
06446b4f08 dbms/ → src/ 2020-04-03 18:14:31 +03:00