Commit Graph

273 Commits

Author SHA1 Message Date
Anton Popov
f99374cca6 Merge remote-tracking branch 'origin/sparse-serialization' into HEAD 2021-07-20 18:20:21 +03:00
Amos Bird
dbfb699690
Asynchronously drain connections. 2021-07-19 21:53:29 +08:00
alexey-milovidov
bc907bd27c
Merge pull request #26336 from azat/dist-per-table-monitor-settings
Add ability to set Distributed directory monitor settings via CREATE TABLE
2021-07-17 01:49:40 +03:00
alexey-milovidov
1701cc429d
Merge pull request #26353 from azat/optimize_distributed_group_by_sharding_key-fix
Fix optimize_distributed_group_by_sharding_key for multiple columns
2021-07-17 01:45:10 +03:00
Azat Khuzhin
f3d3ec44a6 Add ability to set Distributed directory monitor settings via CREATE TABLE 2021-07-16 04:10:47 +03:00
Nikolai Kochetov
f36d14f68f Add separate step to read from remote. 2021-07-15 19:15:16 +03:00
Azat Khuzhin
7b209694d5 Fix optimize_distributed_group_by_sharding_key for multiple columns
Before we incorrectly check that columns from GROUP BY was a subset of
columns from sharding key, while this is not right, consider the
following example:

    select k1, any(k2), sum(v) from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v), cityHash64(k1, k2)) group by k1

Here the columns from GROUP BY is a subset of columns from sharding key,
but the optimization cannot be applied, since there is no guarantee that
particular shard contains distinct values of k1.

So instead we should check that GROUP BY contains all columns that is
required for calculating sharding key expression, i.e.:

    select k1, k2, sum(v) from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v), cityHash64(k1, k2)) group by k1, k2
2021-07-15 09:09:58 +03:00
Anton Popov
5d175bf557 dynamic columns: support distributed tables 2021-07-12 17:54:02 +03:00
Anton Popov
3ed7f5a6cc dynamic subcolumns: add snapshot for storage 2021-07-09 06:15:41 +03:00
Azat Khuzhin
533df9507f Fix log message for optimize_skip_unused_shards_limit 2021-07-07 00:17:39 +03:00
Alexander Tokmakov
1b2416007e fix 2021-07-01 19:43:59 +03:00
Alexander Tokmakov
d9a77e3a1a improve CREATE OR REPLACE query 2021-07-01 16:21:38 +03:00
Raúl Marín
bfc122df64 Fix some typos in Storage classes 2021-06-28 19:03:56 +02:00
alexey-milovidov
1b644b9a31
Merge pull request #25663 from azat/dist-startup
Improve startup time of Distributed engine.
2021-06-27 18:22:45 +03:00
alexey-milovidov
f6e67d3dc1
Update StorageDistributed.cpp 2021-06-27 18:22:34 +03:00
Alexander Tokmakov
3a25b05765 fix rename Distributed table 2021-06-24 13:00:33 +03:00
Azat Khuzhin
a616ae8861 Improve startup time of Distributed engine.
- create directory monitors in parallel (this also includes rmdir in
  case of directory is empty, since even if the directory is empty it
  may take some time to remove it, due to waiting for journal or if the
  directory is large, i.e. it had lots of files before, since remember
  ext4 does not truncate the directory size on each unlink [1])
- initialize increment in parallel too (since it does readdir())

  [1]: https://lore.kernel.org/linux-ext4/930A5754-5CE6-4567-8CF0-62447C97825C@dilger.ca/
2021-06-24 10:27:51 +03:00
Anton Popov
d8b6f15ef4
Merge pull request #23027 from azat/distributed-push-down-limit
Add ability to push down LIMIT for distributed queries
2021-06-20 23:08:50 +03:00
Maksim Kita
67e9b85951 Merge ext into common 2021-06-16 23:28:41 +03:00
alexey-milovidov
34d12063f8
Merge pull request #23349 from azat/dist-respect-insert_allow_materialized_columns
Respect insert_allow_materialized_columns for INSERT into Distributed()
2021-06-14 07:23:00 +03:00
Nikita Mikhaylov
82b8d45cd7
Merge pull request #23518 from nikitamikhaylov/copier-stuck
Bugfixes and improvements of `clickhouse-copier`
2021-06-09 11:36:42 +03:00
Azat Khuzhin
18e8f0eb5e Add ability to push down LIMIT for distributed queries
This way the remote nodes will not need to send all the rows, so this
will decrease network io and also this will make queries w/
optimize_aggregation_in_order=1/LIMIT X and w/o ORDER BY faster since it
initiator will not need to read all the rows, only first X (but note
that for this you need to your data to be sharded correctly or you may
get inaccurate results).

Note, that having lots of processing stages will increase the complexity
of interpreter (it is already not that clean and simple right now).

Although using separate QueryProcessingStage looks pretty natural.

Another option is to make WithMergeableStateAfterAggregation always, but
in this case you will not be able to disable only this optimization,
i.e. if there will be some issue with it.

v2: fix OFFSET
v3: convert 01814_distributed_push_down_limit test to .sh and add retries
v4: add test with OFFSET
v5: add new query stage into the bash completion
v6/tests: use LIMIT O,L syntax over LIMIT L OFFSET O since it is broken in ANTLR parser
          https://clickhouse-test-reports.s3.yandex.net/23027/a18a06399b7aeacba7c50b5d1e981ada5df19745/functional_stateless_tests_(antlr_debug).html#fail1
v7/tests: set use_hedged_requests to 0, to avoid excessive log entries on retries
          https://clickhouse-test-reports.s3.yandex.net/23027/a18a06399b7aeacba7c50b5d1e981ada5df19745/functional_stateless_tests_flaky_check_(address).html#fail1
2021-06-09 02:29:50 +03:00
Amos Bird
78fca8f8fa
Fix possible race condition when getting cluster 2021-06-04 21:09:59 +08:00
Nikita Mikhaylov
312bb96eeb Merge branch 'master' of github.com:ClickHouse/ClickHouse into copier-stuck 2021-06-02 01:04:47 +03:00
Nikita Mikhaylov
6d19dea761 better 2021-05-31 17:38:20 +03:00
Nikita Mikhaylov
90ab394769 better 2021-05-31 17:37:10 +03:00
kssenii
3dee003f9b Merge branch 'master' of github.com:ClickHouse/ClickHouse into poco-file-to-std-fs 2021-05-20 19:20:09 +03:00
Azat Khuzhin
4d737a5481 Respect insert_allow_materialized_columns for INSERT into Distributed() 2021-05-20 07:40:46 +03:00
Alexander Kuzmenkov
e9b69bbd70
Merge pull request #23906 from azat/fix-distributed_group_by_no_merge
distributed_group_by_no_merge fixes
2021-05-19 16:16:08 +03:00
Alexander Kuzmenkov
09cb467812
Update StorageDistributed.cpp 2021-05-19 16:14:33 +03:00
kssenii
9b8df78fdd Merge branch 'master' of github.com:ClickHouse/ClickHouse into poco-file-to-std-fs 2021-05-17 17:42:05 +03:00
feng lv
c6f8ab9826 fix 2021-05-13 02:05:53 +00:00
kssenii
0527f0ea33 Merge branch 'master' of github.com:ClickHouse/ClickHouse into poco-file-to-std-fs 2021-05-12 16:54:18 +03:00
Amos Bird
cd6414639e
add metadata_snapshot to getQueryProcessingStage 2021-05-11 18:12:26 +08:00
Azat Khuzhin
eefd67fce5 Disable optimize_distributed_group_by_sharding_key with window functions 2021-05-06 00:44:22 +03:00
feng lv
39f68bf5ff fix conflict 2021-05-02 16:33:45 +00:00
kssenii
ee06936596 Merge branch 'master' of github.com:ClickHouse/ClickHouse into poco-file-to-std-fs 2021-05-01 17:24:31 +03:00
feng lv
aed2f337e9 Fix CLEAR COLUMN does not work after #21303 2021-04-30 05:02:32 +00:00
kssenii
deb4903af8 Merge branch 'master' of github.com:ClickHouse/ClickHouse into poco-file-to-std-fs 2021-04-28 20:57:13 +03:00
kssenii
eeb71672a0 Change in Storages/* 2021-04-27 16:49:37 +03:00
feng lv
4ffe199d39 Implement table comments 2021-04-23 12:18:23 +00:00
Amos Bird
096d76627e
Skip unavaiable shards when writing to distributed tables 2021-04-21 10:30:40 +08:00
Maksim Kita
e361f5943f
Merge pull request #22999 from azat/no-optimize_skip_unused_shards-single-node
Do not perform optimize_skip_unused_shards for cluster with one node
2021-04-15 14:36:56 +03:00
Nikita Mikhaylov
7a68820342 style 2021-04-13 22:39:42 +03:00
Nikita Mikhaylov
081ea84a41 save 2021-04-13 22:39:41 +03:00
tavplubix
1525e38a3c
Merge pull request #22990 from ClickHouse/tavplubix-patch-1
Fix excessive warning in StorageDistributed with cross-replication
2021-04-13 18:58:12 +03:00
Azat Khuzhin
a497d4d462 Do not perform optimize_skip_unused_shards for cluster with one node 2021-04-12 22:18:31 +03:00
tavplubix
a995962e6a
Update StorageDistributed.cpp 2021-04-12 14:58:24 +03:00
Azat Khuzhin
79bd8d4d3f Respect optimize_skip_unused_shards_rewrite_in with optimize_skip_unused_shards_limit 2021-04-12 10:37:28 +03:00
Azat Khuzhin
e439914d38 Fix optimized cluster logic for optimize_skip_unused_shards 2021-04-12 10:37:28 +03:00
Azat Khuzhin
fbb386dca5 Rewrite IN in query for remote shards to exclude values that does not belongs to shard
v2: fix optimize_skip_unused_shards_rewrite_in for sharding_key wrapped into function
v3: fix column name for optimize_skip_unused_shards_rewrite_in
v4: fix optimize_skip_unused_shards_rewrite_in with Null
v5:
- squash with Remove query argument for IStreamFactory::createForShard()
- use proper column after function execution (using sharding_key_column_name)
- update the test reference since (X) now is tuple(X)
2021-04-12 10:37:28 +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
Nikolai Kochetov
6102652c99 Merge branch 'master' into better-filter-push-down 2021-04-06 13:38:03 +03:00
Maxim Akhmedov
725fa17961 Introduce IStorage::distributedWrite method for distributed INSERT SELECT. 2021-04-05 02:14:27 +03:00
Nikolai Kochetov
c3c393a7aa Merge branch 'master' into refactor-actions-dag 2021-03-18 14:33:07 +03:00
Nikolai Kochetov
e8d7349c79
Merge branch 'master' into dist-query-zero-shards-fix 2021-03-16 12:00:08 +03:00
Azat Khuzhin
61d40c3600 Fix optimize_skip_unused_shards for zero shards case
v2: move check to the beginning of the StorageDistributed::read()
2021-03-10 09:05:14 +03:00
Azat Khuzhin
3474ea044e Avoid processing optimize_skip_unused_shards twice 2021-03-09 10:05:56 +03:00
Azat Khuzhin
ed09897eb1 Pass optimize_skip_unused_shards_limit to the bottom layer
And now optimize_skip_unused_shards_limit=0 is not a special case
anymore.
2021-03-08 10:05:56 +03:00
Azat Khuzhin
16f4c02d42 Add optimize_skip_unused_shards_limit
Limit for number of sharding key values, turns off
optimize_skip_unused_shards if the limit is reached
2021-03-26 06:09:00 +03:00
Nikolai Kochetov
a669f7d641 Merge branch 'master' into refactor-actions-dag 2021-03-05 18:21:14 +03:00
Nikolai Kochetov
9a39459888 Refactor ActionsDAG 2021-03-04 20:38:12 +03:00
Azat Khuzhin
6965ac26c3 Distributed: Add ability to delay/throttle INSERT until pending data will be reduced
Add two new settings for the Distributed engine:
- bytes_to_delay_insert
- max_delay_to_insert

If at the beginning of INSERT there will be too much pending data, more
then bytes_to_delay_insert, then the INSERT will wait until it will be
shrinked, and not more then max_delay_to_insert seconds.

If after this there will be still too much pending, it will throw an
exception.

Also new profile events were added (by analogy to the MergeTree):
- DistributedDelayedInserts (although you can use system.errors instead
  of this, but still)
- DistributedRejectedInserts
- DistributedDelayedInsertsMilliseconds
2021-03-03 23:30:23 +03:00
Azat Khuzhin
b43046ba06 Distributed: More accurate distribution_queue counters
So now system.distribution_queue will show accurate statistics, so tests
does not requires sleep anymore.

But note that with too much distributed pending this will iterate over
all directories.
2021-03-03 23:30:03 +03:00
Azat Khuzhin
b5a5778589 Distributed: Add ability to limit amount of pending bytes for async INSERT
Right now with distributed_directory_monitor_batch_inserts=1 and
insert_distributed_sync=0 INSERT into Distributed table will store
blocks that should be sent to remote (and in case of
prefer_localhost_replica=0 to the localhost too) on the local
filesystem, and sent it in background.

However there is no limit for this storage, and if the remote is
unavailable (or some other error), these pending blocks may take
significant space, and this is not always desired behaviour.

Add new Distributed setting - bytes_to_throw_insert, that will set the
limit for how much pending bytes is allowed, if the limit will be
reached an exception will be throw.

By default was set to 0, to avoid surprises.
2021-03-03 23:30:00 +03:00
Azat Khuzhin
ce09b7ff89 Distributed: Implement totalBytes() (system.tables.total_bytes) 2021-03-03 23:29:11 +03:00
Anton Popov
a4c00ab5dc
Merge pull request #21303 from ucasFL/forbid
Forbid to drop a column if it's referenced by materialized view
2021-03-03 02:55:06 +03:00
feng lv
a26c9e64a9 fix
fix
2021-03-02 03:20:03 +00:00
feng lv
51021c1164 forbid to drop a column if it's referenced by materialized view 2021-02-28 05:24:39 +00:00
Nikolai Kochetov
d328bfa41f Review fixes. Add setting max_optimizations_to_apply. 2021-02-26 19:29:56 +03:00
Azat Khuzhin
809fa7e4cc Sync SYSTEM FLUSH DISTRIBUTED with TRUNCATE 2021-02-10 23:10:37 +03:00
Azat Khuzhin
ce91c257b2 Lockless SYSTEM FLUSH DISTRIBUTED
Right now SYSTEM FLUSH DISTRIBUTED will block:
- INSERT into this Distributed table (requireDirectoryMonitor())
- SELECT * FROM system.distribution_queue
2021-02-08 22:07:30 +03:00
Kruglov Pavel
d94e8624d7
Merge branch 'master' into shard-id 2021-02-06 16:48:17 +03:00
Aleksei Semiglazov
921518db0a CLICKHOUSE-606: query deduplication based on parts' UUID
* add the query data deduplication excluding duplicated parts in MergeTree family engines.

query deduplication is based on parts' UUID which should be enabled first with merge_tree setting
assign_part_uuids=1

allow_experimental_query_deduplication setting is to enable part deduplication, default ot false.

data part UUID is a mechanism of giving a data part a unique identifier.
Having UUID and deduplication mechanism provides a potential of moving parts
between shards preserving data consistency on a read path:
duplicated UUIDs will cause root executor to retry query against on of the replica explicitly
asking to exclude encountered duplicated fingerprints during a distributed query execution.

NOTE: this implementation don't provide any knobs to lock part and hence its UUID. Any mutations/merge will
update part's UUID.

* add _part_uuid virtual column, allowing to use UUIDs in predicates.

Signed-off-by: Aleksei Semiglazov <asemiglazov@cloudflare.com>

address comments
2021-02-02 16:53:39 +00:00
feng lv
4279c7da41 add setting insert_shard_id
add test

fix style

fix
2021-02-02 04:26:59 +00:00
kreuzerkrieg
29a2ef3089 Add IStoragePolicy interface 2021-01-26 10:55:28 +02:00
Azat Khuzhin
2e55bd2285 Accept IDisk in DirectoryMonitor (for further fsync) 2021-01-09 16:31:42 +03:00
Azat Khuzhin
b5ace27014 Add fsync support for Distributed engine.
Two new settings (by analogy with MergeTree family) has been added:

- `fsync_after_insert` - Do fsync for every inserted. Will decreases
  performance of inserts.

- `fsync_tmp_directory` - Do fsync for temporary directory (that is used
  for async INSERT only) after all part operations (writes, renames,
  etc.).

Refs: #17380 (p1)
2021-01-09 11:31:32 +03:00
Azat Khuzhin
714d5a067a Expose supports_parallel_insert via system.table_engines 2021-01-08 14:57:24 +03:00
Alexey Milovidov
190402b7d5 Do not insert empty blocks on sync Distributed INSERT 2021-01-06 02:54:22 +03:00
Amos Bird
6fc225e676
Distributed insertion to one random shard (#18294)
* Distributed insertion to one random shard

* add some tests

* add some documentation

* Respect shards' weights

* fine locking

Co-authored-by: Ivan Lezhankin <ilezhankin@yandex-team.ru>
2020-12-23 19:04:05 +03:00
Azat Khuzhin
5365718f01
Fix optimize_distributed_group_by_sharding_key for query with OFFSET only (#16996)
* Fix optimize_distributed_group_by_sharding_key for query with OFFSET only

* Fix 01244_optimize_distributed_group_by_sharding_key flakiness
2020-12-02 20:11:39 +03:00
tavplubix
085359c110
Merge pull request #17274 from ClickHouse/fix_ast_formatting_in_logs
Fix AST formatting in log messages
2020-11-24 19:00:56 +03:00
Alexander Tokmakov
60a5782c75 fix AST formatting in log messages 2020-11-22 20:23:12 +03:00
Amos Bird
1d9d586e20
Make global_context consistent. 2020-11-20 18:23:14 +08:00
Nikolai Kochetov
46f70dd0de Merge branch 'master' into actions-dag-f14 2020-11-12 11:54:44 +03:00
tavplubix
058aa8f85e
Merge pull request #16824 from ClickHouse/replace_stringstreams_with_buffers
Replace std::*stringstreams with DB::*Buffers
2020-11-12 01:11:44 +03:00
Nikolai Kochetov
1846bb3cac Merge branch 'master' into actions-dag-f14 2020-11-11 13:08:57 +03:00
Nikolai Kochetov
1db8e77371 Add comments. Update ActionsDAG::Index 2020-11-10 17:54:59 +03:00
Nikolai Kochetov
195c941c4e Merge branch 'master' into storage-read-query-plan 2020-11-10 15:02:22 +03:00
Alexander Tokmakov
5cdfcfb307 remove other stringstreams 2020-11-09 22:12:44 +03:00
Nikolai Kochetov
6717c7a0af Merge branch 'master' into actions-dag-f14 2020-11-09 14:57:48 +03:00
alexey-milovidov
f4ba5f1f9a
Merge pull request #16772 from ClickHouse/fix-stringstream
Fix "server failed to start" error
2020-11-08 14:27:08 +03:00
Alexey Milovidov
ba4ae00121 Whitespace 2020-11-08 00:30:40 +03:00
Alexey Milovidov
1ea3afadbc Merge with master 2020-11-08 00:28:39 +03:00
Alexey Milovidov
5314185e25 Merge branch 'master' into azat-optimize_skip_unused_shards-optimization 2020-11-08 00:17:59 +03:00
Alexey Milovidov
fd84d16387 Fix "server failed to start" error 2020-11-07 03:14:53 +03:00
Nikolai Kochetov
c10f733587 Merge branch 'master' into storage-read-query-plan 2020-11-06 15:43:46 +03:00
Nikolai Kochetov
9aeb757da4 Merge branch 'master' into actions-dag-f14 2020-11-06 15:04:20 +03:00
Azat Khuzhin
f23995d290 Remove empty directories for async INSERT at start of Distributed engine
Will be created by DistributedBlockOutputStream on demand.
2020-11-05 23:50:30 +03:00