ClickHouse/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h

114 lines
3.9 KiB
C++
Raw Normal View History

2014-03-13 12:48:07 +00:00
#pragma once
#include <Core/QueryProcessingStage.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/RangesInDataPart.h>
2015-04-12 04:39:20 +00:00
2014-03-13 12:48:07 +00:00
namespace DB
{
class KeyCondition;
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +00:00
2014-03-13 12:48:07 +00:00
2017-04-16 15:00:33 +00:00
/** Executes SELECT queries on data from the merge tree.
2014-03-13 12:48:07 +00:00
*/
class MergeTreeDataSelectExecutor
{
public:
2019-10-01 16:50:08 +00:00
explicit MergeTreeDataSelectExecutor(const MergeTreeData & data_);
2014-03-13 12:48:07 +00:00
2017-04-16 15:00:33 +00:00
/** When reading, selects a set of parts that covers the desired range of the index.
2018-10-10 16:20:15 +00:00
* max_blocks_number_to_read - if not nullptr, do not read all the parts whose right border is greater than max_block in partition.
*/
2018-09-17 12:28:29 +00:00
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
2018-10-23 08:19:47 +00:00
Pipes read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
2019-02-10 16:55:12 +00:00
UInt64 max_block_size,
2017-06-02 15:54:39 +00:00
unsigned num_streams,
2018-10-10 16:20:15 +00:00
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
2014-03-13 12:48:07 +00:00
Pipes readFromParts(
MergeTreeData::DataPartsVector parts,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
const Context & context,
2019-02-10 16:55:12 +00:00
UInt64 max_block_size,
unsigned num_streams,
2018-10-10 16:20:15 +00:00
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
2014-03-13 12:48:07 +00:00
private:
2018-10-17 03:13:00 +00:00
const MergeTreeData & data;
2014-03-13 12:48:07 +00:00
2020-05-30 21:57:37 +00:00
Poco::Logger * log;
2014-03-13 12:48:07 +00:00
Pipes spreadMarkRangesAmongStreams(
2017-11-24 23:03:58 +00:00
RangesInDataParts && parts,
2017-06-02 15:54:39 +00:00
size_t num_streams,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
2019-02-10 16:55:12 +00:00
UInt64 max_block_size,
bool use_uncompressed_cache,
const SelectQueryInfo & query_info,
const Names & virt_columns,
2019-10-10 16:30:30 +00:00
const Settings & settings,
const MergeTreeReaderSettings & reader_settings) const;
2014-03-13 12:48:07 +00:00
2020-05-13 15:53:47 +00:00
/// out_projection - save projection only with columns, requested to read
Pipes spreadMarkRangesAmongStreamsWithOrder(
2019-05-18 12:21:40 +00:00
RangesInDataParts && parts,
size_t num_streams,
2019-05-18 12:21:40 +00:00
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
2019-05-18 12:21:40 +00:00
UInt64 max_block_size,
bool use_uncompressed_cache,
const SelectQueryInfo & query_info,
2019-07-28 00:41:26 +00:00
const ExpressionActionsPtr & sorting_key_prefix_expr,
const Names & virt_columns,
2019-10-10 16:30:30 +00:00
const Settings & settings,
2020-05-12 18:22:58 +00:00
const MergeTreeReaderSettings & reader_settings,
ExpressionActionsPtr & out_projection) const;
2014-03-13 12:48:07 +00:00
Pipes spreadMarkRangesAmongStreamsFinal(
2017-11-24 23:03:58 +00:00
RangesInDataParts && parts,
2020-04-22 13:52:07 +00:00
size_t num_streams,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
2019-02-10 16:55:12 +00:00
UInt64 max_block_size,
bool use_uncompressed_cache,
const SelectQueryInfo & query_info,
const Names & virt_columns,
2019-10-10 16:30:30 +00:00
const Settings & settings,
2020-05-12 18:22:58 +00:00
const MergeTreeReaderSettings & reader_settings,
ExpressionActionsPtr & out_projection) const;
2017-04-16 15:00:33 +00:00
/// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index.
size_t getApproximateTotalRowsToRead(
const MergeTreeData::DataPartsVector & parts,
2020-06-17 12:39:20 +00:00
const StorageMetadataPtr & metadata_snapshot,
const KeyCondition & key_condition,
const Settings & settings) const;
2014-03-13 12:48:07 +00:00
Optimize PK lookup for queries that match exact PK range Existing code that looks up marks that match the query has a pathological case, when most of the part does in fact match the query. The code works by recursively splitting a part into ranges and then discarding the ranges that definitely do not match the query, based on primary key. The problem is that it requires visiting every mark that matches the query, making the complexity of this sort of look up O(n). For queries that match exact range on the primary key, we can find both left and right parts of the range with O(log 2) complexity. This change implements exactly that. To engage this optimization, the query must: * Have a prefix list of the primary key. * Have only range or single set element constraints for columns. * Have only AND as a boolean operator. Consider a table with `(service, timestamp)` as the primary key. The following conditions will be optimized: * `service = 'foo'` * `service = 'foo' and timestamp >= now() - 3600` * `service in ('foo')` * `service in ('foo') and timestamp >= now() - 3600 and timestamp <= now` The following will fall back to previous lookup algorithm: * `timestamp >= now() - 3600` * `service in ('foo', 'bar') and timestamp >= now() - 3600` * `service = 'foo'` Note that the optimization won't engage when PK has a range expression followed by a point expression, since in that case the range is not continuous. Trace query logging provides the following messages types of messages, each representing a different kind of PK usage for a part: ``` Used optimized inclusion search over index for part 20200711_5710108_5710108_0 with 9 steps Used generic exclusion search over index for part 20200711_5710118_5710228_5 with 1495 steps Not using index on part 20200710_5710473_5710473_0 ``` Number of steps translates to computational complexity. Here's a comparison for before and after for a query over 24h of data: ``` Read 4562944 rows, 148.05 MiB in 45.19249672 sec., 100966 rows/sec., 3.28 MiB/sec. Read 4183040 rows, 135.78 MiB in 0.196279627 sec., 21311636 rows/sec., 691.75 MiB/sec. ``` This is especially useful for queries that read data in order and terminate early to return "last X things" matching a query. See #11564 for more thoughts on this.
2020-07-07 18:10:44 +00:00
MarkRanges markRangesFromPKRange(
2019-03-25 13:55:24 +00:00
const MergeTreeData::DataPartPtr & part,
2020-06-17 12:39:20 +00:00
const StorageMetadataPtr & metadata_snapshot,
const KeyCondition & key_condition,
Optimize PK lookup for queries that match exact PK range Existing code that looks up marks that match the query has a pathological case, when most of the part does in fact match the query. The code works by recursively splitting a part into ranges and then discarding the ranges that definitely do not match the query, based on primary key. The problem is that it requires visiting every mark that matches the query, making the complexity of this sort of look up O(n). For queries that match exact range on the primary key, we can find both left and right parts of the range with O(log 2) complexity. This change implements exactly that. To engage this optimization, the query must: * Have a prefix list of the primary key. * Have only range or single set element constraints for columns. * Have only AND as a boolean operator. Consider a table with `(service, timestamp)` as the primary key. The following conditions will be optimized: * `service = 'foo'` * `service = 'foo' and timestamp >= now() - 3600` * `service in ('foo')` * `service in ('foo') and timestamp >= now() - 3600 and timestamp <= now` The following will fall back to previous lookup algorithm: * `timestamp >= now() - 3600` * `service in ('foo', 'bar') and timestamp >= now() - 3600` * `service = 'foo'` Note that the optimization won't engage when PK has a range expression followed by a point expression, since in that case the range is not continuous. Trace query logging provides the following messages types of messages, each representing a different kind of PK usage for a part: ``` Used optimized inclusion search over index for part 20200711_5710108_5710108_0 with 9 steps Used generic exclusion search over index for part 20200711_5710118_5710228_5 with 1495 steps Not using index on part 20200710_5710473_5710473_0 ``` Number of steps translates to computational complexity. Here's a comparison for before and after for a query over 24h of data: ``` Read 4562944 rows, 148.05 MiB in 45.19249672 sec., 100966 rows/sec., 3.28 MiB/sec. Read 4183040 rows, 135.78 MiB in 0.196279627 sec., 21311636 rows/sec., 691.75 MiB/sec. ``` This is especially useful for queries that read data in order and terminate early to return "last X things" matching a query. See #11564 for more thoughts on this.
2020-07-07 18:10:44 +00:00
const Settings & settings) const;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
MarkRanges filterMarksUsingIndex(
2020-05-28 13:45:08 +00:00
MergeTreeIndexPtr index_helper,
2019-06-19 15:30:48 +00:00
MergeTreeIndexConditionPtr condition,
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
MergeTreeData::DataPartPtr part,
const MarkRanges & ranges,
const Settings & settings,
const MergeTreeReaderSettings & reader_settings) const;
2014-03-13 12:48:07 +00:00
};
}