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>
|
2021-02-05 09:54:34 +00:00
|
|
|
#include <Storages/MergeTree/PartitionPruner.h>
|
2021-04-08 08:19:04 +00:00
|
|
|
#include <Processors/QueryPlan/ReadFromMergeTree.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
|
|
|
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryPlanPtr 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
|
|
|
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryPlanPtr 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
|
|
|
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryPlanPtr spreadMarkRangesAmongStreams(
|
2017-11-24 23:03:58 +00:00
|
|
|
RangesInDataParts && parts,
|
2021-04-08 08:19:04 +00:00
|
|
|
ReadFromMergeTree::IndexStatPtr index_stats,
|
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,
|
2021-01-25 05:01:39 +00:00
|
|
|
const MergeTreeReaderSettings & reader_settings,
|
|
|
|
const String & query_id) 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
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder(
|
2019-05-18 12:21:40 +00:00
|
|
|
RangesInDataParts && parts,
|
2021-04-08 08:19:04 +00:00
|
|
|
ReadFromMergeTree::IndexStatPtr index_stats,
|
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,
|
2020-11-11 10:08:57 +00:00
|
|
|
const ActionsDAGPtr & 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,
|
2021-01-25 05:01:39 +00:00
|
|
|
ActionsDAGPtr & out_projection,
|
|
|
|
const String & query_id) const;
|
2014-03-13 12:48:07 +00:00
|
|
|
|
2020-10-01 17:34:22 +00:00
|
|
|
QueryPlanPtr spreadMarkRangesAmongStreamsFinal(
|
2017-11-24 23:03:58 +00:00
|
|
|
RangesInDataParts && parts,
|
2021-04-08 08:19:04 +00:00
|
|
|
ReadFromMergeTree::IndexStatPtr index_stats,
|
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,
|
2021-01-25 05:01:39 +00:00
|
|
|
ActionsDAGPtr & out_projection,
|
|
|
|
const String & query_id) 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
|
|
|
|
2020-07-20 15:09:00 +00:00
|
|
|
static 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,
|
2020-07-20 15:09:00 +00:00
|
|
|
const Settings & settings,
|
|
|
|
Poco::Logger * log);
|
2019-02-05 14:50:25 +00:00
|
|
|
|
2020-07-20 15:09:00 +00:00
|
|
|
static 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,
|
2020-07-20 15:09:00 +00:00
|
|
|
const MergeTreeReaderSettings & reader_settings,
|
2021-02-13 10:59:09 +00:00
|
|
|
size_t & total_granules,
|
|
|
|
size_t & granules_dropped,
|
2020-07-20 15:09:00 +00:00
|
|
|
Poco::Logger * log);
|
2021-02-05 09:54:34 +00:00
|
|
|
|
2021-04-06 12:39:55 +00:00
|
|
|
struct PartFilterCounters
|
|
|
|
{
|
|
|
|
size_t num_initial_selected_parts = 0;
|
|
|
|
size_t num_initial_selected_granules = 0;
|
|
|
|
size_t num_parts_after_minmax = 0;
|
|
|
|
size_t num_granules_after_minmax = 0;
|
2021-04-08 11:48:54 +00:00
|
|
|
size_t num_parts_after_partition_pruner = 0;
|
|
|
|
size_t num_granules_after_partition_pruner = 0;
|
2021-04-06 12:39:55 +00:00
|
|
|
};
|
|
|
|
|
2021-02-05 09:54:34 +00:00
|
|
|
/// Select the parts in which there can be data that satisfy `minmax_idx_condition` and that match the condition on `_part`,
|
|
|
|
/// as well as `max_block_number_to_read`.
|
2021-03-02 16:13:36 +00:00
|
|
|
static void selectPartsToRead(
|
2021-02-05 09:54:34 +00:00
|
|
|
MergeTreeData::DataPartsVector & parts,
|
|
|
|
const std::unordered_set<String> & part_values,
|
|
|
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
2021-03-02 10:33:54 +00:00
|
|
|
const DataTypes & minmax_columns_types,
|
2021-02-05 09:54:34 +00:00
|
|
|
std::optional<PartitionPruner> & partition_pruner,
|
2021-04-06 12:39:55 +00:00
|
|
|
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
|
|
|
PartFilterCounters & counters);
|
2021-02-05 09:54:34 +00:00
|
|
|
|
|
|
|
/// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded.
|
|
|
|
void selectPartsToReadWithUUIDFilter(
|
|
|
|
MergeTreeData::DataPartsVector & parts,
|
|
|
|
const std::unordered_set<String> & part_values,
|
|
|
|
const std::optional<KeyCondition> & minmax_idx_condition,
|
2021-03-02 10:33:54 +00:00
|
|
|
const DataTypes & minmax_columns_types,
|
2021-02-05 09:54:34 +00:00
|
|
|
std::optional<PartitionPruner> & partition_pruner,
|
|
|
|
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
2021-04-06 12:39:55 +00:00
|
|
|
const Context & query_context,
|
|
|
|
PartFilterCounters & counters) const;
|
2014-03-13 12:48:07 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|