Commit Graph

273 Commits

Author SHA1 Message Date
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
Alexander Tokmakov
821b64b420 apply review suggestions 2023-05-22 15:18:29 +02:00
AVMusorin
418a61a68c
Allow using Alias column type for KafkaEngine
```
create table kafka
(
 a UInt32,
 a_str String Alias toString(a)
) engine = Kafka;

create table data
(
  a UInt32;
  a_str String
) engine = MergeTree
order by tuple();

create materialized view data_mv to data
(
  a UInt32,
  a_str String
) as
select a, a_str from kafka;
```
Alias type works as expected in comparison with MATERIALIZED/EPHEMERAL
or column with default expression.

Ref: https://github.com/ClickHouse/ClickHouse/pull/47138

Co-authored-by: Azat Khuzhin <a3at.mail@gmail.com>
2023-05-15 15:39:58 +02:00
Azat Khuzhin
2f9a7cba8a Fix stack-use-after-scope in KafkaProducer::produce()
ASan report:

    ==1==ERROR: AddressSanitizer: stack-use-after-scope on address 0x7f7540186b20 at pc 0x55656ef458ad bp 0x7f754114acf0 sp 0x7f754114a4b8
    READ of size 2 at 0x7f7540186b20 thread T2 (TCPHandler)
        0 0x55656ef458ac in __asan_memcpy (/usr/bin/clickhouse+0x11b028ac) (BuildId: 6ba63057df561f6dc16fb8fd7623fd858bf327d1)
        1 0x55659f62daf8 in rd_kafka_msg_new00 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:192:3
        2 0x55659f62daf8 in rd_kafka_msg_new0 build_docker/./contrib/librdkafka/src/rdkafka_msg.c:257:8
        3 0x55659f630635 in rd_kafka_producev build_docker/./contrib/librdkafka/src/rdkafka_msg.c:642:23
        4 0x55659f4a865c in cppkafka::Producer::do_produce() build_docker/./contrib/cppkafka/src/producer.cpp:128:19
        5 0x55659f4a8306 in cppkafka::Producer::produce() build_docker/./contrib/cppkafka/src/producer.cpp:69:5
        6 0x5565961e748c in DB::KafkaProducer::produce() build_docker/./src/Storages/Kafka/KafkaProducer.cpp:70:23
        7 0x5565962155b2 in DB::MessageQueueSink::consume(DB::Chunk) build_docker/./src/Storages/MessageQueueSink.cpp:64:23
        8 0x5565980b9e8e in DB::SinkToStorage::onConsume(DB::Chunk) build_docker/./src/Processors/Sinks/SinkToStorage.cpp:18:5

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-05-10 12:15:17 +02:00
Michael Kolupaev
683077890f Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
kssenii
8537ee400d Fix kafka test 2023-03-20 11:22:25 +01:00
kssenii
bb0beb7449 Merge remote-tracking branch 'upstream/master' into named-collections-finish 2023-03-17 13:02:36 +01:00
Robert Schulze
5b91631722
Merge pull request #47138 from AVMusorin/disallow-default-columns-kafka-engine
Prohibit DEFAULT/EPHEMERAL/ALIAS/MATERIALIZED columns in KafkaEngine
2023-03-07 20:36:46 +01:00
kssenii
8567e3976b Fix kafka 2023-03-06 12:30:25 +01:00
kssenii
c06af1f1e7 Fix clang-tidy 2023-03-05 22:12:51 +01:00
kssenii
8f2d75cef8 Fix tests 2023-03-05 12:56:00 +01:00
AVMusorin
0f2ae72141
prohibit DEFAULT/EPHEMERAL/ALIAS in KafkaEngine 2023-03-03 16:25:42 +01:00
Kseniia Sumarokova
386663953c
Merge branch 'master' into named-collections-finish 2023-03-03 12:23:38 +01:00
Kruglov Pavel
ce8e49a9a0
Merge pull request #45693 from Avogar/fix-kafka-protobuf
Fix formats parser resetting, test processing bad messages in kafka
2023-03-02 11:50:45 +01:00
Kruglov Pavel
a06596bdba
Fix special build 2023-02-28 14:33:41 +01:00
Robert Schulze
3618e37dd2
Reduce indentation by early out 2023-02-25 15:36:59 +00:00
Robert Schulze
6402050125
Fixup comments 2023-02-24 16:59:38 +01:00
Robert Schulze
6906cafa99
Change to S3 cfg syntax 2023-02-23 20:04:41 +00: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
kssenii
b3ca976f5f Finish for streaming engines 2023-02-21 14:50:55 +01:00
Kruglov Pavel
e56dead5aa
Merge branch 'master' into fix-kafka-protobuf 2023-02-15 15:16:25 +01:00
Kseniia Sumarokova
0b55021c68
Merge pull request #46337 from kssenii/rabbitmq-tests-fix-2
Fix flaky test_storage_rabbitmq::test_rabbitmq_address
2023-02-14 11:44:34 +01:00
kssenii
614d57d245 Fix race 2023-02-13 16:02:41 +01:00
Antonio Andelic
95db6fb1a6
Merge pull request #45659 from ClickHouse/add-support-for-settings-alias
Add support for settings aliases/Rename `replication_alter_partitions_sync` to `alter_sync`
2023-02-13 15:31:12 +01:00
Kruglov Pavel
e139070043
Merge branch 'master' into fix-kafka-protobuf 2023-02-13 12:38:18 +01:00
Alexey Milovidov
384ae944fe Merge branch 'master' into filimonov-kafka-Local-No-offset-stored 2023-02-12 04:01:07 +01:00
Antonio Andelic
85cfee4bb9 Better alias definition 2023-02-01 13:54:03 +00:00
Antonio Andelic
ec04cf6876 Merge branch 'master' into add-support-for-settings-alias 2023-02-01 09:01:01 +00:00
Alexey Milovidov
54bb8b1ceb
Merge branch 'master' into fix-kafka-protobuf 2023-01-29 00:59:40 +03:00
avogar
0a99d421f5 Fix formats parser resetting, test processing bad messages in kafka 2023-01-27 13:22:50 +00:00
Antonio Andelic
714fad1529 Add support for settings alias 2023-01-26 14:06:46 +00:00
Alexander Tokmakov
3f6594f4c6 forbid old ctor of Exception 2023-01-23 22:18:05 +01:00
Alexander Tokmakov
70d1adfe4b
Better formatting for exception messages (#45449)
* save format string for NetException

* format exceptions

* format exceptions 2

* format exceptions 3

* format exceptions 4

* format exceptions 5

* format exceptions 6

* fix

* format exceptions 7

* format exceptions 8

* Update MergeTreeIndexGin.cpp

* Update AggregateFunctionMap.cpp

* Update AggregateFunctionMap.cpp

* fix
2023-01-24 00:13:58 +03:00
avogar
2ca3a74b9a Fix style 2022-12-30 16:46:21 +00:00
avogar
a74ffa70bc Don't use async producing in Kafka 2022-12-30 16:40:04 +00:00
avogar
755b08a49e Fix comments 2022-12-15 19:47:10 +00:00
Kruglov Pavel
c5b2e4cc23
Merge branch 'master' into improve-streaming-engines 2022-12-15 18:44:35 +01:00
Vitaly Baranov
e1f7f04752
Referential dependencies for RESTORE (#43834)
* Rename DDLDependencyVisitor -> DDLLoadingDependencyVisitor.

* Move building a loading graph to TablesLoader.

* Implement referential dependencies for tables and use them
when restoring tables from a backup.

* Remove StorageID::operator < (because of its inconsistency with ==).

* Add new tests.

* Fix test.

* Fix memory leak.

Co-authored-by: Nikita Mikhaylov <mikhaylovnikitka@gmail.com>
2022-12-02 15:05:46 +01:00
Yakko Majuri
e68a3b0c7a
fix(typo): Passible -> Possible 2022-11-03 11:25:07 -03:00
Kruglov Pavel
b124875257
Merge branch 'master' into improve-streaming-engines 2022-11-03 13:22:06 +01:00
avogar
e39e61fc71 Fix heap-use-after-free in PeekableReadBuffer 2022-11-01 12:58:20 +00:00
avogar
d5f68e013d Fix style 2022-10-28 17:09:08 +00: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
filimonov
8e930cbfce
style 2022-10-18 14:33:04 +02:00
filimonov
75137bc3e8
Attempt to fix 'Local: No offset stored message' from Kafka
See https://github.com/ClickHouse/ClickHouse/issues/12295 

Test is needed
2022-10-17 14:44:43 +02:00
Robert Schulze
f24fab7747
Fix some #include atrocities 2022-09-28 13:49:28 +00:00
Robert Schulze
6d70b4a1f6
Generate config_version.h into ${CONFIG_INCLUDE_PATH}
This makes the target location consistent with other auto-generated
files like config_formats.h, config_core.h, and config_functions.h and
simplifies the build of clickhouse_common.
2022-09-28 12:48:26 +00:00
Maksim Kita
276d3628e9 Small style fixes 2022-09-15 13:05:18 +02:00
Ilya Golshtein
74561b40ae stylecheck renewed 2022-09-02 17:42:18 +03:00