2014-03-13 12:48:07 +00:00
|
|
|
#pragma once
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
#include <Core/QueryProcessingStage.h>
|
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#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
|
|
|
|
{
|
|
|
|
|
2018-04-20 00:20:36 +00:00
|
|
|
class KeyCondition;
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-09-17 12:28:29 +00:00
|
|
|
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
2018-10-23 08:19:47 +00:00
|
|
|
|
2019-09-13 15:41:09 +00:00
|
|
|
Pipes read(
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
|
2019-09-13 15:41:09 +00:00
|
|
|
Pipes readFromParts(
|
2018-07-18 12:17:48 +00:00
|
|
|
MergeTreeData::DataPartsVector parts,
|
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2018-07-18 12:17:48 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
|
|
|
const Context & context,
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size,
|
2018-07-18 12:17:48 +00:00
|
|
|
unsigned num_streams,
|
2018-10-10 16:20:15 +00:00
|
|
|
const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const;
|
2018-07-18 12:17:48 +00:00
|
|
|
|
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
|
|
|
|
2019-09-13 15:41:09 +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,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool use_uncompressed_cache,
|
2019-04-17 21:20:51 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & virt_columns,
|
2019-10-10 16:30:30 +00:00
|
|
|
const Settings & settings,
|
2019-12-18 15:54:45 +00:00
|
|
|
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
|
2019-09-13 15:41:09 +00:00
|
|
|
Pipes spreadMarkRangesAmongStreamsWithOrder(
|
2019-05-18 12:21:40 +00:00
|
|
|
RangesInDataParts && parts,
|
2019-07-18 14:41:11 +00:00
|
|
|
size_t num_streams,
|
2019-05-18 12:21:40 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
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,
|
2017-04-01 07:20:54 +00:00
|
|
|
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
|
|
|
|
2019-09-13 15:41:09 +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,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Names & column_names,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size,
|
2017-04-01 07:20:54 +00:00
|
|
|
bool use_uncompressed_cache,
|
2019-04-17 21:20:51 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
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;
|
2015-11-18 21:37:28 +00:00
|
|
|
|
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.
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t getApproximateTotalRowsToRead(
|
|
|
|
const MergeTreeData::DataPartsVector & parts,
|
2020-06-17 12:39:20 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2018-04-20 00:20:36 +00:00
|
|
|
const KeyCondition & key_condition,
|
2017-04-01 07:20:54 +00:00
|
|
|
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,
|
2018-04-20 00:20:36 +00:00
|
|
|
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;
|
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,
|
2019-02-05 14:50:25 +00:00
|
|
|
MergeTreeData::DataPartPtr part,
|
|
|
|
const MarkRanges & ranges,
|
2020-06-25 19:31:54 +00:00
|
|
|
const Settings & settings,
|
|
|
|
const MergeTreeReaderSettings & reader_settings) const;
|
2014-03-13 12:48:07 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|