Commit Graph

176 Commits

Author SHA1 Message Date
Ivan Lezhankin
9fd048cdbd Allow to select virtual columns in materialized view 2019-06-18 19:32:37 +03:00
Ivan Lezhankin
436052c67b Merge branch 'master' into issue-5286 2019-06-18 16:03:29 +03:00
Ivan Lezhankin
d907d8e8ca Commit offsets for SELECTing from Kafka table too 2019-06-17 19:27:18 +03:00
proller
fc7ae399c4 Use system rapidjson; Fix types for macos; 2019-06-14 13:28:30 +03:00
Alexey Milovidov
c846d0f6af Namespace comments are unneeded according to the code style.
find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l -P '}\s*//+\s*namespace\s*' | xargs sed -i -r -e 's/}\s*\/\/+\s*namespace\s*.*$/}/'
2019-06-13 13:43:37 +03:00
Alexey Milovidov
8b87511e25 Namespace comments are unneeded according to the code style.
find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l -P '}\s*//+\s*namespace\s*' | xargs sed -i -r -e 's/}\s*\/\/+\s*namespace\s*\w+/}/'
2019-06-13 13:37:13 +03:00
Ivan Lezhankin
97558c1b41 Merge remote-tracking branch 'upstream/master' into issue-5286 2019-06-10 14:15:14 +03:00
Ivan
f70eed78d8
Don't do intermediate commits in Kafka tables by default (#5445)
* Don't do intermediate commits in Kafka tables by default
* Use default settings' values as defaults in code
2019-06-03 19:40:22 +03:00
Ivan Lezhankin
8326021d74 Fix style 2019-06-03 17:36:59 +03:00
Ivan Lezhankin
b311984879 Merge remote-tracking branch 'upstream/master' into issue-5286 2019-06-03 17:19:09 +03:00
Ivan Lezhankin
1eccbc39c5 Don't add virtual column to empty block 2019-05-28 00:01:24 +03:00
Ivan Lezhankin
13212c9b01 Fix the hang on dropping Kafka table when there is no mat. views 2019-05-27 20:25:34 +03:00
alexey-milovidov
81ed764d6a
Merge pull request #5285 from abyss7/issue-5085
Don't crash the server when Kafka consumers have failed to start
2019-05-25 16:09:26 +03:00
Ivan Lezhankin
19a850ad75 Use read callback to populate virtual columns in Kafka Engine 2019-05-23 16:20:25 +03:00
Ivan Lezhankin
800854119e Add buffer callback to fill in virtual columns 2019-05-23 14:15:18 +03:00
Ivan Lezhankin
dd906eabdc [WIP] refactoring 2019-05-22 22:38:43 +03:00
Ivan Lezhankin
cf39c4cc47 Embed virtual columns into IStorage 2019-05-21 14:24:32 +03:00
Ivan Lezhankin
a14013dff7 Add setting to regulate Kafka committing policy. 2019-05-16 18:20:30 +03:00
alexey-milovidov
8e2bdde93b
Update StorageKafka.cpp 2019-05-15 23:57:24 +03:00
Ivan
1ea9e3019d
Freeze the Kafka buffer after first empty response (#5283)
* Check inside inferior streams for cancellation while reading.
* Stop reading from Kafka buffer after first empty read.
2019-05-15 19:11:50 +03:00
Ivan Lezhankin
337ddf5047 Catch cppkafka exceptions about consumer creation failures 2019-05-15 17:23:52 +03:00
Ivan
4a5832b18a
Check the time limit every (flush_interval / poll_timeout) number of rows from Kafka (#5249) 2019-05-14 18:52:03 +03:00
Vitaly Baranov
28d9a80072 Rename macro APPLY_FOR_SETTINGS => LIST_OF_SETTINGS. 2019-04-29 03:22:59 +03:00
Vitaly Baranov
8277e9d8f1 Refactoring of Settings based on new template class SettingsCollection.
Add new class SettingsConstraints.
2019-04-26 19:11:05 +03:00
proller
ba4687e110
Cmake: dont use LINK_MODE (but use BUILD_SHARED_LIBS) (#5091) 2019-04-24 12:06:28 +03:00
Ivan Lezhankin
94bba1918f Do not drop Kafka consumer buffers after closing stream 2019-04-22 16:23:05 +03:00
Ivan
957d23267b
Minimal implementation of row-level security CLICKHOUSE-4315 (#4792)
For detailed description see the related PR
2019-03-29 23:31:06 +03:00
Alexey Milovidov
25ab402d21 ParserCreateQuery: moved code to cpp 2019-03-24 01:45:28 +03:00
proller
fce9db12b9 Move Interpreters/Settings* -> Core/Settings* 2019-03-22 15:08:30 +03:00
Ivan Lezhankin
4f6a494af8 Revert references where they were before refactoring. 2019-03-15 19:14:13 +03:00
Ivan Lezhankin
4c1a0ca255 Merge remote-tracking branch 'upstream/master' into iast-as 2019-03-12 15:51:18 +03:00
proller
8227f5fe63 Arcadia fix 2019-03-12 15:06:17 +03:00
Ivan Lezhankin
d1ead22073 Rename As<…> → as<…> 2019-03-11 16:22:51 +03:00
Ivan Lezhankin
a7e838e328 Replace all other casts to method IAST::As<Derived>
Also remove some redundant methods like IsIdentifier() et. al.
2019-03-11 15:49:39 +03:00
Ivan Lezhankin
0e325c7a4a Replace const typeid_casts with As<Derived>() method invocations 2019-03-08 12:48:20 +03:00
Vitaly Baranov
309df832ec IStorage::read() function, "max_block_size" parameter: change type UInt64 -> size_t. 2019-02-19 14:56:08 +03:00
proller
bdefba6c32 fix link order 2019-02-12 15:59:40 +03:00
proller
558ebbcc31 New library clickhouse_storage_kafka 2019-02-12 14:17:46 +03:00
proller
4078a62573 Fix build without contrib/cppkafka submodule 2019-02-11 19:34:32 +03:00
Ivan Lezhankin
15ad409875 Tests are now passing locally 2019-02-11 14:54:30 +03:00
Ivan Lezhankin
2bd14117f4 Merge remote-tracking branch 'upstream/master' into issue-2675 2019-02-11 13:57:40 +03:00
Alexey Milovidov
58a6accee5 Updated test #4246 2019-02-10 19:55:12 +03:00
Ivan Lezhankin
39e2563f93 Create new consumers on-demand
and give them some time for graceful assignment.
2019-01-30 20:58:49 +03:00
Ivan Lezhankin
681266abd0 Poll messages in batches of max_block_size size. 2019-01-25 15:48:59 +03:00
Ivan Lezhankin
9636bdcd67 Fix build 2019-01-24 15:44:58 +03:00
Ivan Lezhankin
862d419f31 Fix losing the next message after a broken one. 2019-01-24 14:58:25 +03:00
Ivan Lezhankin
425d483cfb Add integration tests 2019-01-24 14:58:25 +03:00
Ivan Lezhankin
c8e605327d Split StorageKafka.cpp on smaller files per class 2019-01-24 14:58:24 +03:00
Ivan Lezhankin
05ab15cc84 Merge IBlockInputStream and IProfilingBlockInputStream into one 2019-01-23 17:48:50 +03:00
maiha
6c83beafdd dbms: Fixed misspells in comments 2019-01-23 04:56:53 +09:00
Ivan Lezhankin
5f74094edb Fix a setting description. 2019-01-18 16:15:36 +03:00
Ivan Lezhankin
89ffb577b1 Implement additional Kafka engine setting for allowed parsing errors 2019-01-18 15:48:38 +03:00
Ivan Lezhankin
337c092c7e Use cppkafka instead of raw C interface 2019-01-14 14:15:57 +03:00
Ivan Lezhankin
9c35598373 Add cppkafka to contrib 2019-01-14 14:15:57 +03:00
Alexey Milovidov
19f4652959 Fixed bugs found by PVS-Studio 2019-01-09 18:44:20 +03:00
Alexey Milovidov
1d4701b0bc Added some warnings from clang's -Weverything 2019-01-04 15:10:00 +03:00
Alexey Milovidov
e33e5150b7 Miscellaneous [#CLICKHOUSE-2] 2019-01-02 09:44:36 +03:00
Alexey Milovidov
2c6e49c3f3 More checks [#CLICKHOUSE-2] 2018-12-21 19:24:47 +03:00
Alexey Milovidov
661a117b91 Added assertion [#CLICKHOUSE-2] 2018-12-21 19:03:40 +03:00
Alexey Milovidov
52fea90a86 Removed useless code [#CLICKHOUSE-2] 2018-11-28 17:33:40 +03:00
Ivan Lezhankin
74fe350451 Refactor partition commands handling into separate interface method. 2018-11-20 17:59:34 +03:00
alexey-milovidov
b1db9ec23c
Merge pull request #3436 from amosbird/master
Flexible fields assignment
2018-11-01 20:22:28 +03:00
chertus
d2518fdb3f more refactoring: funcs to extract database_and_table from AST 2018-10-29 22:04:28 +03:00
Amos Bird
3cf2177268 Flexible fields assignment
Field f;

before:

f = Int64(-42);

after:

f = -42;
2018-10-24 08:47:33 +08:00
Marek Vavruša
37e03bd938 StorageKafka: check dependencies recursively, add max_block setting
Changes:

* Check table dependencies recursively before starting streaming
* Use actual number of created consumers when streaming
* Add size limits hint to Kafka consumer stream
* Configurable `kafka_max_block_size` per table
2018-10-18 09:20:34 -07:00
proller
ca03cd6606 Sync with arcadia 2018-10-01 18:43:48 +03:00
alexey-milovidov
bb0f76dd25
Update StorageKafka.cpp 2018-10-01 04:29:17 +03:00
alexey-milovidov
b9e6b02719
Update StorageKafka.cpp 2018-10-01 04:26:52 +03:00
Marek Vavruša
ea32434888 Storages/Kafka: fixed deadlocks, moved to background pool
This fixes two deadlocks in Kafka engine found previously:
* When exception is thrown before starting reading, consumer
  was never returned to the storage. Now it is claimed only
  when actually starting reading.
* Fixed lockup on deinitialization when consumer only unsubscribed,
  but didn't close, and the endine then timeouted when waiting for
  consumer destruction.

This also moves the stream thread to background worker pool.
The reason for that is that it will compete with other tasks for
time, so it will form a backpressure on insertion when the system
is busy.
2018-09-25 22:51:07 -07:00
Vojtech Splichal
5aca7f3c6e Fix Storage Kafka: Schema positional argument number
I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en
2018-09-17 13:01:10 +02:00
zhang2014
7a87da78d7 ISSUES-3110 fix merge and distributed engine query stage 2018-09-14 18:16:27 +08:00
Alexey Milovidov
5a096ee099 Attempt to fix an issue mentioned by Matt Keranen in #2958 2018-09-06 05:47:59 +03:00
Nikolai Kochetov
391b4b5927 Merged with master. 2018-08-20 15:25:06 +03:00
Alexey Milovidov
36db216abf Preparation for extra warnings [#CLICKHOUSE-2] 2018-08-10 07:02:56 +03:00
Alexey Milovidov
2016374d20 Merging #2226 2018-08-05 10:05:43 +03:00
Alexander Marshalov
154d7e3228 Added SETTINGS clause for Kafka storage engine 2018-08-03 21:51:28 +03:00