Commit Graph

94 Commits

Author SHA1 Message Date
Nikolai Kochetov
e29f27e333
Merge pull request #14815 from ClickHouse/storage-read-query-plan
Return query plan from storage.
2020-09-15 18:34:11 +03:00
Nikolai Kochetov
6eac433c55 Review fixes. 2020-09-15 13:40:39 +03:00
Artem Zuikov
51ba12c2c3
Try speedup build (#14809) 2020-09-15 12:55:57 +03:00
Nikolai Kochetov
bcfc83022a Return query plan from storage. 2020-09-14 17:13:58 +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
Dongdong Yang
7e48edf42c fix issue #14202 2020-08-28 13:29:15 +08:00
Nikolai Kochetov
9b67cd9faf Merge branch 'master' into refactor-pipes-3 2020-08-10 10:50:17 +03:00
Alexey Milovidov
edd89a8610 Fix half of typos 2020-08-08 03:47:03 +03:00
Nikolai Kochetov
d83c7cc0e6 Merge branch 'master' into refactor-pipes-3 2020-08-06 17:32:50 +03:00
Nikolai Kochetov
09fbce1b1e Merge branch 'master' into refactor-pipes-3 2020-08-04 11:32:34 +03:00
Vitaly Baranov
a804f9499d Use references while iterating through settings. 2020-08-04 04:00:38 +03:00
Nikolai Kochetov
e411916bde Refactor Pipe [part 1]. 2020-08-03 14:33:11 +03:00
Vitaly Baranov
18b21511a9
Merge pull request #13013 from vitlibar/implement-custom-settings
Implement custom settings
2020-08-02 05:01:14 +03:00
Vitaly Baranov
56665a15f7 Rework and rename the template class SettingsCollection => BaseSettings. 2020-07-31 20:54:18 +03:00
Vitaly Baranov
300727afa3 Rework the StringField* classes and make conversion String => StringField* explicit. 2020-07-31 19:11:27 +03:00
Vitaly Baranov
90602b869a Make SettingsChanges a class. 2020-07-31 19:11:27 +03:00
Vitaly Baranov
4ad99f1472 Make type names of settings shorter: SettingUInt64 => UInt64. 2020-07-31 19:11:27 +03:00
Alexey Milovidov
362655fa43 Fix some bad tests 2020-07-30 22:59:49 +03:00
Alexey Milovidov
a9b02e8184 Fix trivial bug in code 2020-07-30 22:33:20 +03:00
Azat Khuzhin
e2d9176617 Fix SIGSEGV in StorageKafka when broker is unavailable 2020-07-22 11:28:07 +03:00
Azat Khuzhin
610382b693 kafka: fix SIGSEGV if there is an message with error in the middle of the batch
ReadBufferFromKafkaConsumer does not handle the case when there is
message with an error on non first position in the current batch, since
it goes through messages in the batch after poll and stop on first valid
message.

But later it can try to use message as valid:
- while storing offset
- get topic name
- ...

And besides the message itself is also invalid (you can find this in the
gdb traces below).

So just filter out messages win an error error after poll.

SIGSEGV was with the following stacktrace:
    (gdb) bt
    3  0x0000000010f05b4d in rd_kafka_offset_store (app_rkt=0x0, partition=0, offset=0) at ../contrib/librdkafka/src/rdkafka_offset.c:656
    4  0x0000000010e69657 in cppkafka::Consumer::store_offset (this=0x7f2015210820, msg=...) at ../contrib/cppkafka/include/cppkafka/message.h:225
    5  0x000000000e68f208 in DB::ReadBufferFromKafkaConsumer::storeLastReadMessageOffset (this=0x7f206a136618) at ../contrib/libcxx/include/iterator:1508
    6  0x000000000e68b207 in DB::KafkaBlockInputStream::readImpl (this=0x7f202c689020) at ../src/Storages/Kafka/KafkaBlockInputStream.cpp:150
    7  0x000000000dd1178d in DB::IBlockInputStream::read (this=this@entry=0x7f202c689020) at ../src/DataStreams/IBlockInputStream.cpp:60
    8  0x000000000dd34c0a in DB::copyDataImpl<> () at ../src/DataStreams/copyData.cpp:21
    9  DB::copyData () at ../src/DataStreams/copyData.cpp:62
    10 0x000000000e67c8f2 in DB::StorageKafka::streamToViews () at ../contrib/libcxx/include/memory:3823
    11 0x000000000e67d218 in DB::StorageKafka::threadFunc () at ../src/Storages/Kafka/StorageKafka.cpp:488

And some information from it:

    (gdb) p this.current.__i
    $14 = (std::__1::__wrap_iter<cppkafka::Message const*>::iterator_type) 0x7f1ca8f58660

    # current-1
    (gdb) p $14-1
    $15 = (const cppkafka::Message *) 0x7f1ca8f58600
    (gdb) p $16.handle_
    $17 = {__ptr_ = {<std::__1::__compressed_pair_elem<rd_kafka_message_s*, 0, false>> = { __value_ = 0x7f203577f938}, ...}
    (gdb) p *(rd_kafka_message_s*)0x7f203577f938
    $24 = {err = RD_KAFKA_RESP_ERR__TRANSPORT, rkt = 0x0, partition = 0, payload = 0x7f202f0339c0, len = 63, key = 0x0, key_len = 0, offset = 0, _private = 0x7f203577f8c0}

    # current
    (gdb) p $14-0
    $28 = (const cppkafka::Message *) 0x7f1ca8f58660
    (gdb) p $28.handle_.__ptr_
    $29 = {<std::__1::__compressed_pair_elem<rd_kafka_message_s*, 0, false>> = { __value_ = 0x7f184f129bf0}, ...}
    (gdb) p *(rd_kafka_message_s*)0x7f184f129bf0
    $30 = {err = RD_KAFKA_RESP_ERR_NO_ERROR, rkt = 0x7f1ed44fe000, partition = 1, payload = 0x7f1fc9bc6036, len = 242, key = 0x0, key_len = 0, offset = 2394853582209,

    # current+1
    (gdb) p (*($14+1)).handle_.__ptr_
    $44 = {<std::__1::__compressed_pair_elem<rd_kafka_message_s*, 0, false>> = { __value_ = 0x7f184f129d30}, ...}
    (gdb) p *(rd_kafka_message_s*)0x7f184f129d30
    $45 = {err = RD_KAFKA_RESP_ERR_NO_ERROR, rkt = 0x7f1ed44fe000, partition = 1, payload = 0x7f1fc9bc612f, len = 31, key = 0x0, key_len = 0, offset = 2394853582210,
      _private = 0x7f184f129cc0}

    # distance from the beginning
    (gdb) p messages.__end_-messages.__begin_
    $34 = 65536
    (gdb) p ($14-0)-messages.__begin_
    $37 = 8965
    (gdb) p ($14-1)-messages.__begin_
    $38 = 8964

    # parsing info
    (gdb) p allowed
    $39 = false
    (gdb) p new_rows
    $40 = 1
    (gdb) p total_rows
    $41 = 8964

    # current buffer is invalid
    (gdb) p *buffer.__ptr_
    $50 = {<DB::ReadBuffer> = {<DB::BufferBase> = {pos = 0x7f202f0339c0 "FindCoordinator response error: Local: Broker transport failure", bytes = 47904863385, working_buffer = {
            begin_pos = 0x7f202f0339c0 "FindCoordinator response error: Local: Broker transport failure",
            end_pos = 0x7f202f0339c0 "FindCoordinator response error: Local: Broker transport failure"}, internal_buffer = {

v0: check message errors in ReadBufferFromKafkaConsumer::nextImpl() (but
this may lead to using of that messages after and SIGSEGV again, doh).
v2: skip messages with an error after poll.
2020-07-10 11:41:44 +03:00
tavplubix
488a8cbc9f
Merge pull request #12120 from filimonov/query_context_for_system_logs
Add query context for system logs and to Buffer
2020-07-07 09:55:20 +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
Azat Khuzhin
bd5e5e9462 kafka: remove outdated comment
As stated by @filimonov it is not relevant (after #11599)
2020-07-03 09:23:39 +03:00
Azat Khuzhin
de8bc99d48 kafka: avoid superior polling after DROP/DETACH TABLE
Before this patch isStalled() was checked before polledDataUnusable(),
and after DROP TABLE isStalled() == true (although this looks tricky).
2020-07-03 09:23:39 +03:00
Azat Khuzhin
8508200020 kafka: improve logging during engine shutdown
This will help with tracking possible issues, when you need to know was
buffer released or not.
2020-07-03 09:23:39 +03:00
Azat Khuzhin
e7179c402c kafka: check that the data is still usable after parsing 2020-07-03 09:23:38 +03:00
Azat Khuzhin
6104872cae kafka: fix SIGSEGV on DROP TABLE
After #11599 it is possible that messages of the
ReadBufferFromKafkaConsumer will be cleaned-up right in
read_kafka_message callback (from KafkaBlockInputStream) if the stop
flag isset (i.e. DROP TABLE is waiting the consumer), and if
read_kafka_message already processed some rows it will not return 0 and
the loop after will try to get current topic from the buffer, which uses
messages in the underlying and this will got SIGSEGV:

    12:14:56.173262 [ 55421 ] {f7930856-d478-4e41-af56-24ce7b693e95} <Debug> executeQuery: (from 0.0.0.0:0, user: ) DROP TABLE IF EXISTS data.queue
    12:14:56.173285 [ 55421 ] {f7930856-d478-4e41-af56-24ce7b693e95} <Trace> StorageKafka (newly_queue): Waiting for cleanup
    12:14:56.180016 [ 55390 ] {} <Trace> BaseDaemon: Received signal 11
    12:14:56.180267 [ 4914 ] {} <Fatal> BaseDaemon: ########################################
    12:14:56.181879 [ 4914 ] {} <Fatal> BaseDaemon: (version 20.6.1.1, build id: 4CE0298F08583658) (from thread 55468) (no query) Received signal Segmentation fault (11)
    12:14:56.181900 [ 4914 ] {} <Fatal> BaseDaemon: Address: 0x8 Access: read. Address not mapped to object.
    12:14:56.181909 [ 4914 ] {} <Fatal> BaseDaemon: Stack trace:
    12:14:56.184676 [ 4914 ] {} <Fatal> BaseDaemon: 3. /ch/contrib/cppkafka/include/cppkafka/message.h:111: DB::KafkaBlockInputStream::readImpl() @ 0xe343f1c in /usr/lib/debug/usr/bin/clickhouse
    12:14:56.185553 [ 4914 ] {} <Fatal> BaseDaemon: 4. /ch/contrib/libcxx/include/vector:1003: DB::IBlockInputStream::read() @ 0xd9d95bd in /usr/lib/debug/usr/bin/clickhouse
    12:14:56.188238 [ 4914 ] {} <Fatal> BaseDaemon: 5. /ch/src/DataStreams/copyData.cpp:26: DB::copyData() @ 0xd9f712a in /usr/lib/debug/usr/bin/clickhouse
    12:14:56.188780 [ 4914 ] {} <Fatal> BaseDaemon: 6. /ch/contrib/libcxx/include/vector:1532: DB::StorageKafka::streamToViews() @ 0xe335e73 in /usr/lib/debug/usr/bin/clickhouse
    12:14:56.189331 [ 4914 ] {} <Fatal> BaseDaemon: 7. /ch/src/Storages/Kafka/StorageKafka.cpp:491: DB::StorageKafka::threadFunc() @ 0xe336738 in /usr/lib/debug/usr/bin/clickhouse

55421 thread (shows that it still waiting for deactivation):

    5  std::__1::lock_guard<>::lock_guard () at ../contrib/libcxx/include/__mutex_base:90
    6  DB::BackgroundSchedulePoolTaskInfo::deactivate (this=0x7fc7e4465f20) at ../src/Core/BackgroundSchedulePool.cpp:59
    7  DB::StorageKafka::shutdown (this=0x7fc7e45e4600) at ../contrib/libcxx/include/memory:3821

And just in case thread where read_kafka_message is called:

    0  DB::ReadBufferFromKafkaConsumer::nextImpl (this=0x7fd4901d4118) at ../contrib/libcxx/include/atomic:1491
    1  DB::ReadBuffer::next (this=0x7fd4901d4118) at ../src/IO/ReadBuffer.h:59
    2  DB::ReadBuffer::eof (this=0x7fd4901d4118) at ../src/IO/ReadBuffer.h:81
    3  DB::skipWhitespaceIfAny (buf=...) at ../src/IO/ReadHelpers.h:945
    4  DB::JSONEachRowRowInputFormat::readRow (ext=..., columns=..., this=0x7fd499a7a020) at ../src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp:222
    5  DB::JSONEachRowRowInputFormat::readRow (this=0x7fd499a7a020, columns=..., ext=...) at ../src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp:218
    6  DB::IRowInputFormat::generate (this=0x7fd499a7a020) at ../src/Processors/Formats/IRowInputFormat.cpp:64
    7  DB::ISource::work (this=0x7fd499a7a020) at ../src/Processors/ISource.cpp:48
    8  DB::KafkaBlockInputStream::<lambda()>::operator()(void) const () at ../contrib/libcxx/include/memory:3826
    9  DB::KafkaBlockInputStream::readImpl (this=0x7fd46e718820) at ../contrib/libcxx/include/new:340

Cc: @filimonov
2020-07-03 09:22:44 +03:00
alesapin
b9e74f4e82 Merge branch 'master' into atomic_metadata5 2020-06-22 12:03:53 +03:00
alexey-milovidov
076fe8e8ba
Update KafkaBlockInputStream.cpp 2020-06-20 20:34:42 +03:00
alesapin
4c0879ae30 Better logging in storages 2020-06-19 20:17:13 +03:00
alesapin
c9fa5d2ec3 Better naming 2020-06-19 18:39:41 +03:00
Mikhail Filimonov
9609bd9dee Kafka better states, formats based on PeekableReadBuffer, and other minor fixes.
Add formats tests, fixes for JSONCompactEachRowWithNamesAndTypes, TSVWithNamesAndTypes. Some CR fixes
Add sanitizing for kafka_max_block_size and kafka_poll_max_batch_size
2020-06-19 10:49:05 +02:00
alesapin
71f99a274d Compileable getSampleBlockWithColumns in StorageInMemoryMetadata 2020-06-16 17:25:08 +03:00
alesapin
53cb5210de Move getSampleBlockNonMaterialized to StorageInMemoryMetadata 2020-06-16 15:48:10 +03:00
alesapin
36ba0192df Metadata in read and write methods of IStorage 2020-06-15 22:08:58 +03:00
alesapin
af2fe2ba55 Compilable setColumns, setConstraints, setIndices 2020-06-15 19:55:33 +03:00
Mikhail Filimonov
26d93fdfba Fix the test, adjust default for queued.min.messages 2020-06-04 14:14:24 +02:00
Mikhail Filimonov
eef914cbdf Remove debug print 2020-06-03 20:34:22 +02: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
alesapin
c22e8f6b9b
Merge pull request #11283 from filimonov/kafka_virtual_column_headers
Virtual columns for Kafka headers
2020-06-01 12:06:00 +03:00
alesapin
364ebe9246
Merge pull request #11260 from filimonov/kafka_virtual_column_timestamp_ms
Add _timestamp_ms virtual columns to Kafka engine
2020-06-01 12:04:17 +03:00
alexey-milovidov
9984e989c1
Merge pull request #11317 from ClickHouse/logger-fmt-2
Preparation for structured logging
2020-06-01 00:11:29 +03:00
alexey-milovidov
9a362eb02e
Merge pull request #11252 from filimonov/kafka-clientid
Kafka clientid (finishing #11073)
2020-05-31 18:37:30 +03:00
Alexey Milovidov
25f941020b Remove namespace pollution 2020-05-31 00:57:37 +03:00