Commit Graph

90 Commits

Author SHA1 Message Date
Alexander Tokmakov
2e7e195e77 change alter_lock to std::timed_mutex 2021-10-26 13:37:00 +03:00
Alexey Milovidov
fe6b7c77c7 Rename "common" to "base" 2021-10-02 10:13:14 +03:00
Nikolai Kochetov
b997214620 Rename QueryPipeline to QueryPipelineBuilder. 2021-09-14 20:48:18 +03:00
Alexander Tokmakov
13466a7cc3 minor fix 2021-09-03 20:06:38 +03:00
Alexander Tokmakov
42378b5913 fix 2021-08-20 17:05:53 +03:00
Alexey Milovidov
24cef99065 Merge branch 'master' into fix-bad-cast 2021-08-08 04:00:29 +03:00
alexey-milovidov
c5207fc237
Merge pull request #26466 from azat/optimize-dist-select
Rework SELECT from Distributed optimizations
2021-08-08 03:59:32 +03:00
mergify[bot]
dc57254982
Merge branch 'master' into improve_create_or_replace 2021-08-03 11:39:07 +00:00
Azat Khuzhin
2fb95d9ee0 Rework SELECT from Distributed query stages optimization
Before this patch it wasn't possible to optimize simple SELECT * FROM
dist ORDER BY (w/o GROUP BY and DISTINCT) to more optimal stage
(QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit),
since that code was under
allow_nondeterministic_optimize_skip_unused_shards, rework it and make
it possible.

Also now distributed_push_down_limit is respected for
optimize_distributed_group_by_sharding_key.

Next step will be to enable distributed_push_down_limit by default.

v2: fix detection of aggregates
2021-08-02 21:04:29 +03:00
Alexey Milovidov
edfeb0957f Fix strange code 2021-07-24 04:52:18 +03:00
Nikolai Kochetov
2dc5c89b66 Update Storage::write 2021-07-23 17:25:35 +03:00
Alexander Tokmakov
d9a77e3a1a improve CREATE OR REPLACE query 2021-07-01 16:21:38 +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
Maksim Kita
67e9b85951 Merge ext into common 2021-06-16 23:28:41 +03:00
Anton Popov
3acbd12c54 enable reading of subcolumn for distributed tables 2021-05-25 03:49:24 +03:00
feng lv
c6f8ab9826 fix 2021-05-13 02:05:53 +00:00
Amos Bird
cd6414639e
add metadata_snapshot to getQueryProcessingStage 2021-05-11 18:12:26 +08:00
feng lv
4ffe199d39 Implement table comments 2021-04-23 12:18:23 +00: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
Maxim Akhmedov
725fa17961 Introduce IStorage::distributedWrite method for distributed INSERT SELECT. 2021-04-05 02:14:27 +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
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
Azat Khuzhin
456cbaf747 Distributed: Hide private part of the interface 2021-03-03 23:29:11 +03:00
feng lv
51021c1164 forbid to drop a column if it's referenced by materialized view 2021-02-28 05:24:39 +00: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
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
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
Amos Bird
1d9d586e20
Make global_context consistent. 2020-11-20 18:23:14 +08:00
Nikolai Kochetov
195c941c4e Merge branch 'master' into storage-read-query-plan 2020-11-10 15:02:22 +03:00
Alexey Milovidov
5314185e25 Merge branch 'master' into azat-optimize_skip_unused_shards-optimization 2020-11-08 00:17:59 +03:00
Nikolai Kochetov
c10f733587 Merge branch 'master' into storage-read-query-plan 2020-11-06 15:43:46 +03:00
Alexander Tokmakov
ac32809b6a fix #16482 2020-11-02 19:40:39 +03:00
Nikolai Kochetov
7fa045cff8 Merge branch 'master' into storage-read-query-plan 2020-10-22 13:31:10 +03:00
Alexander Tokmakov
72b1339656 Revert "Revert "Write structure of table functions to metadata""
This reverts commit c65d1e5c70.
2020-10-14 15:19:29 +03:00
tavplubix
c65d1e5c70
Revert "Write structure of table functions to metadata" 2020-10-14 13:59:29 +03:00
Azat Khuzhin
b838214a35 Pass non-const SelectQueryInfo (and drop mutable qualifiers) 2020-10-02 22:42:35 +03:00
Azat Khuzhin
587cde853e Avoid skipping unused shards twice (for query processing stage and read itself) 2020-10-02 22:42:09 +03:00
Nikolai Kochetov
576ffadb17 Fix explain for ISourceStep. 2020-09-30 15:22:30 +03:00
Nikolai Kochetov
dea90009e3 Fix build 2020-09-25 16:03:12 +03:00
Alexander Tokmakov
1ca9a92b21 Merge branch 'master' into write_structure_of_table_functions 2020-09-18 21:09:23 +03:00
Nikolai Kochetov
b26f11c00c Support StorageDistributed::read for QueryPlan. 2020-09-18 17:16:53 +03:00
Pavel Kovalenko
01ab28a182 Don't throw exception if Distributed storage has multi-volume storage policy configuration. 2020-09-15 12:26:56 +03:00
Alexander Tokmakov
b840d741d0 Merge branch 'master' into write_structure_of_table_functions 2020-09-04 13:00:07 +03:00
Azat Khuzhin
10b4f3b41f Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key
Previous set of QueryProcessingStage does not allow to do this.
But after WithMergeableStateAfterAggregation had been introduced the
following queries can be optimized too under
optimize_distributed_group_by_sharding_key:
- GROUP BY sharding_key LIMIT
- GROUP BY sharding_key LIMIT BY
- GROUP BY sharding_key ORDER BY

And right now it is still not supports:
- WITH TOTALS (looks like it can be supported)
- WITH ROLLUP (looks like it can be supported)
- WITH CUBE
- SETTINGS extremes=1 (looks like it can be supported)
But will be implemented separatelly.

vX: fixes
v2: fix WITH *
v3: fix extremes
v4: fix LIMIT OFFSET (and make a little bit cleaner)
v5: fix HAVING
v6: fix ORDER BY
v7: rebase against 20.7
v8: move out WithMergeableStateAfterAggregation
v9: add optimize_distributed_group_by_sharding_key into test names
2020-09-03 00:52:51 +03:00
Alexander Tokmakov
56695727b2 Merge branch 'master' into write_structure_of_table_functions 2020-09-01 20:15:13 +03:00
Alexander Tokmakov
969940b4c9 write table tructure for table function remote(...) 2020-08-26 23:55:40 +03:00
Alexey Milovidov
2a09aa53cc Support parallel INSERT for more table engines 2020-08-26 19:41:30 +03:00