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
|
|
|
|
2021-05-02 13:38:19 +00:00
|
|
|
struct MergeTreeDataSelectSamplingData
|
|
|
|
{
|
2021-05-23 23:56:03 +00:00
|
|
|
bool use_sampling = false;
|
2021-05-25 16:34:43 +00:00
|
|
|
bool read_nothing = false;
|
2021-05-27 13:40:33 +00:00
|
|
|
Float64 used_sample_factor = 1.0;
|
2021-05-02 13:38:19 +00:00
|
|
|
std::shared_ptr<ASTFunction> filter_function;
|
|
|
|
ActionsDAGPtr filter_expression;
|
|
|
|
};
|
|
|
|
|
2021-05-27 13:40:33 +00:00
|
|
|
using PartitionIdToMaxBlock = std::unordered_map<String, Int64>;
|
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-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,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size,
|
2017-06-02 15:54:39 +00:00
|
|
|
unsigned num_streams,
|
2021-02-10 14:12:49 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2021-05-27 16:53:58 +00:00
|
|
|
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read = nullptr) const;
|
2014-03-13 12:48:07 +00:00
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// The same as read, but with specified set of parts.
|
|
|
|
QueryPlanPtr readFromParts(
|
2021-05-27 13:40:33 +00:00
|
|
|
MergeTreeData::DataPartsVector parts,
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_base,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const SelectQueryInfo & query_info,
|
|
|
|
ContextPtr context,
|
2021-05-28 17:16:09 +00:00
|
|
|
UInt64 max_block_size,
|
2021-05-27 13:40:33 +00:00
|
|
|
unsigned num_streams,
|
2021-05-27 16:53:58 +00:00
|
|
|
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read = nullptr) const;
|
2021-05-27 13:40:33 +00:00
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// Get an estimation for the number of marks we are going to read.
|
|
|
|
/// Reads nothing. Secondary indexes are not used.
|
|
|
|
/// This method is used to select best projection for table.
|
|
|
|
size_t estimateNumMarksToRead(
|
2018-07-18 12:17:48 +00:00
|
|
|
MergeTreeData::DataPartsVector parts,
|
|
|
|
const Names & column_names,
|
2021-02-10 14:12:49 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_base,
|
2020-06-16 14:25:08 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2018-07-18 12:17:48 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2018-07-18 12:17:48 +00:00
|
|
|
unsigned num_streams,
|
2021-05-27 16:53:58 +00:00
|
|
|
std::shared_ptr<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;
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log;
|
2014-03-13 12:48:07 +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.
|
2021-05-25 16:34:43 +00:00
|
|
|
static size_t getApproximateTotalRowsToRead(
|
2017-04-01 07:20:54 +00:00
|
|
|
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,
|
2021-05-25 16:34:43 +00:00
|
|
|
const Settings & settings,
|
|
|
|
Poco::Logger * log);
|
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,
|
2021-05-25 16:34:43 +00:00
|
|
|
const std::optional<std::unordered_set<String>> & part_values,
|
2021-02-05 09:54:34 +00:00
|
|
|
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.
|
2021-05-25 16:34:43 +00:00
|
|
|
static void selectPartsToReadWithUUIDFilter(
|
2021-02-05 09:54:34 +00:00
|
|
|
MergeTreeData::DataPartsVector & parts,
|
2021-05-25 16:34:43 +00:00
|
|
|
const std::optional<std::unordered_set<String>> & part_values,
|
2020-11-24 14:24:48 +00:00
|
|
|
MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids,
|
2021-02-05 09:54:34 +00:00
|
|
|
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-12 12:23:32 +00:00
|
|
|
ContextPtr query_context,
|
2021-05-25 16:34:43 +00:00
|
|
|
PartFilterCounters & counters,
|
|
|
|
Poco::Logger * log);
|
|
|
|
|
2021-05-27 13:40:33 +00:00
|
|
|
public:
|
2021-05-28 17:16:09 +00:00
|
|
|
/// For given number rows and bytes, get the number of marks to read.
|
|
|
|
/// It is a minimal number of marks which contain so many rows and bytes.
|
2021-05-28 09:41:07 +00:00
|
|
|
static size_t roundRowsOrBytesToMarks(
|
|
|
|
size_t rows_setting,
|
|
|
|
size_t bytes_setting,
|
|
|
|
size_t rows_granularity,
|
|
|
|
size_t bytes_granularity);
|
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// The same as roundRowsOrBytesToMarks, but return no more than max_marks.
|
2021-05-28 09:41:07 +00:00
|
|
|
static size_t minMarksForConcurrentRead(
|
|
|
|
size_t rows_setting,
|
|
|
|
size_t bytes_setting,
|
|
|
|
size_t rows_granularity,
|
|
|
|
size_t bytes_granularity,
|
|
|
|
size_t max_marks);
|
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// If possible, filter using expression on virtual columns.
|
|
|
|
/// Example: SELECT count() FROM table WHERE _part = 'part_name'
|
|
|
|
/// If expression found, return a set with allowed part names (std::nullopt otherwise).
|
2021-05-27 13:40:33 +00:00
|
|
|
static std::optional<std::unordered_set<String>> filterPartsByVirtualColumns(
|
|
|
|
const MergeTreeData & data,
|
2021-05-28 17:16:09 +00:00
|
|
|
const MergeTreeData::DataPartsVector & parts,
|
2021-05-27 13:40:33 +00:00
|
|
|
const ASTPtr & query,
|
|
|
|
ContextPtr context);
|
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// Filter parts using minmax index and partition key.
|
2021-05-27 13:40:33 +00:00
|
|
|
static void filterPartsByPartition(
|
2021-05-28 17:16:09 +00:00
|
|
|
MergeTreeData::DataPartsVector & parts,
|
|
|
|
const std::optional<std::unordered_set<String>> & part_values,
|
2021-05-27 13:40:33 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
const MergeTreeData & data,
|
|
|
|
const SelectQueryInfo & query_info,
|
2021-05-27 16:53:58 +00:00
|
|
|
const ContextPtr & context,
|
2021-05-27 13:40:33 +00:00
|
|
|
const PartitionIdToMaxBlock * max_block_numbers_to_read,
|
|
|
|
Poco::Logger * log,
|
|
|
|
ReadFromMergeTree::IndexStats & index_stats);
|
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// Filter parts using primary key and secondary indexes.
|
|
|
|
/// For every part, select mark ranges to read.
|
2021-07-14 15:09:17 +00:00
|
|
|
/// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings.
|
2021-05-27 13:40:33 +00:00
|
|
|
static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes(
|
|
|
|
MergeTreeData::DataPartsVector && parts,
|
2021-05-26 18:14:43 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot,
|
2021-05-27 13:40:33 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2021-05-27 16:53:58 +00:00
|
|
|
const ContextPtr & context,
|
2021-05-28 17:16:09 +00:00
|
|
|
const KeyCondition & key_condition,
|
2021-05-26 18:14:43 +00:00
|
|
|
const MergeTreeReaderSettings & reader_settings,
|
|
|
|
Poco::Logger * log,
|
|
|
|
size_t num_streams,
|
2021-05-27 13:40:33 +00:00
|
|
|
ReadFromMergeTree::IndexStats & index_stats,
|
2021-06-30 10:39:58 +00:00
|
|
|
bool use_skip_indexes,
|
2021-07-14 15:09:17 +00:00
|
|
|
bool check_limits);
|
2021-05-26 18:14:43 +00:00
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// Create expression for sampling.
|
|
|
|
/// Also, calculate _sample_factor if needed.
|
|
|
|
/// Also, update key condition with selected sampling range.
|
2021-05-25 16:34:43 +00:00
|
|
|
static MergeTreeDataSelectSamplingData getSampling(
|
|
|
|
const ASTSelectQuery & select,
|
2021-05-28 17:16:09 +00:00
|
|
|
NamesAndTypesList available_real_columns,
|
2021-05-28 14:34:02 +00:00
|
|
|
const MergeTreeData::DataPartsVector & parts,
|
2021-05-25 16:34:43 +00:00
|
|
|
KeyCondition & key_condition,
|
|
|
|
const MergeTreeData & data,
|
2021-05-28 17:16:09 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
ContextPtr context,
|
2021-05-25 16:34:43 +00:00
|
|
|
bool sample_factor_column_queried,
|
2021-05-28 17:16:09 +00:00
|
|
|
Poco::Logger * log);
|
2021-05-26 18:14:43 +00:00
|
|
|
|
2021-05-28 17:16:09 +00:00
|
|
|
/// Check query limits: max_partitions_to_read, max_concurrent_queries.
|
|
|
|
/// Also, return QueryIdHolder. If not null, we should keep it until query finishes.
|
|
|
|
static std::shared_ptr<QueryIdHolder> checkLimits(
|
2021-05-28 09:41:07 +00:00
|
|
|
const MergeTreeData & data,
|
|
|
|
const RangesInDataParts & parts_with_ranges,
|
|
|
|
const ContextPtr & context);
|
2014-03-13 12:48:07 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|