ClickHouse/src/Storages/SelectQueryInfo.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

233 lines
7.1 KiB
C++
Raw Normal View History

2017-07-15 04:06:51 +00:00
#pragma once
2022-09-07 17:09:37 +00:00
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/TableExpressionModifiers.h>
2023-02-03 13:34:18 +00:00
#include <Core/Names.h>
#include <Core/SortDescription.h>
#include <Interpreters/AggregateDescription.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/PreparedSets.h>
2022-09-07 17:09:37 +00:00
#include <Planner/PlannerContext.h>
2023-02-03 13:34:18 +00:00
#include <QueryPipeline/StreamLocalLimits.h>
2017-07-15 04:06:51 +00:00
#include <memory>
namespace DB
{
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
2020-11-03 11:28:28 +00:00
class ActionsDAG;
using ActionsDAGPtr = std::shared_ptr<ActionsDAG>;
struct PrewhereInfo;
using PrewhereInfoPtr = std::shared_ptr<PrewhereInfo>;
struct FilterInfo;
using FilterInfoPtr = std::shared_ptr<FilterInfo>;
struct FilterDAGInfo;
using FilterDAGInfoPtr = std::shared_ptr<FilterDAGInfo>;
struct InputOrderInfo;
using InputOrderInfoPtr = std::shared_ptr<const InputOrderInfo>;
struct TreeRewriterResult;
using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
class ReadInOrderOptimizer;
using ReadInOrderOptimizerPtr = std::shared_ptr<const ReadInOrderOptimizer>;
class Cluster;
using ClusterPtr = std::shared_ptr<Cluster>;
struct PrewhereInfo
{
2021-02-15 19:48:06 +00:00
/// Actions for row level security filter. Applied separately before prewhere_actions.
/// This actions are separate because prewhere condition should not be executed over filtered rows.
2021-06-25 14:49:28 +00:00
ActionsDAGPtr row_level_filter;
/// Actions which are executed on block in order to get filter column for prewhere step.
2020-11-03 19:05:47 +00:00
ActionsDAGPtr prewhere_actions;
2021-02-15 19:48:06 +00:00
String row_level_column_name;
2020-11-03 19:05:47 +00:00
String prewhere_column_name;
bool remove_prewhere_column = false;
bool need_filter = false;
2021-06-25 14:49:28 +00:00
PrewhereInfo() = default;
explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_)
2020-11-03 19:05:47 +00:00
: prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {}
2020-12-02 18:16:31 +00:00
std::string dump() const;
2022-07-21 15:52:33 +00:00
PrewhereInfoPtr clone() const
{
PrewhereInfoPtr prewhere_info = std::make_shared<PrewhereInfo>();
if (row_level_filter)
prewhere_info->row_level_filter = row_level_filter->clone();
if (prewhere_actions)
prewhere_info->prewhere_actions = prewhere_actions->clone();
prewhere_info->row_level_column_name = row_level_column_name;
prewhere_info->prewhere_column_name = prewhere_column_name;
prewhere_info->remove_prewhere_column = remove_prewhere_column;
prewhere_info->need_filter = need_filter;
return prewhere_info;
}
2020-11-03 19:05:47 +00:00
};
/// Helper struct to store all the information about the filter expression.
struct FilterInfo
{
ExpressionActionsPtr alias_actions;
ExpressionActionsPtr actions;
String column_name;
bool do_remove_column = false;
};
/// Same as FilterInfo, but with ActionsDAG.
struct FilterDAGInfo
{
ActionsDAGPtr actions;
String column_name;
bool do_remove_column = false;
2020-12-02 18:16:31 +00:00
std::string dump() const;
};
2020-05-13 13:49:10 +00:00
struct InputOrderInfo
{
2022-07-04 15:05:52 +00:00
/// Sort description for merging of already sorted streams.
/// Always a prefix of ORDER BY or GROUP BY description specified in query.
SortDescription sort_description_for_merging;
2022-07-04 15:05:52 +00:00
/** Size of prefix of sorting key that is already
* sorted before execution of sorting or aggreagation.
*
* Contains both columns that scpecified in
* ORDER BY or GROUP BY clause of query
* and columns that turned out to be already sorted.
*
* E.g. if we have sorting key ORDER BY (a, b, c, d)
* and query with `WHERE a = 'x' AND b = 'y' ORDER BY c, d` clauses.
* sort_description_for_merging will be equal to (c, d) and
* used_prefix_of_sorting_key_size will be equal to 4.
*/
const size_t used_prefix_of_sorting_key_size;
const int direction;
const UInt64 limit;
2019-07-19 10:14:27 +00:00
InputOrderInfo(
const SortDescription & sort_description_for_merging_,
size_t used_prefix_of_sorting_key_size_,
int direction_, UInt64 limit_)
: sort_description_for_merging(sort_description_for_merging_)
, used_prefix_of_sorting_key_size(used_prefix_of_sorting_key_size_)
, direction(direction_), limit(limit_)
2019-12-10 23:18:24 +00:00
{
}
2019-12-11 01:34:39 +00:00
bool operator==(const InputOrderInfo &) const = default;
};
class IMergeTreeDataPart;
using ManyExpressionActions = std::vector<ExpressionActionsPtr>;
2017-07-15 04:06:51 +00:00
/** Query along with some additional data,
* that can be used during query processing
* inside storage engines.
*/
struct SelectQueryInfo
2017-07-15 04:06:51 +00:00
{
SelectQueryInfo()
: prepared_sets(std::make_shared<PreparedSets>())
{}
2017-07-15 04:06:51 +00:00
ASTPtr query;
ASTPtr view_query; /// Optimized VIEW query
2017-07-15 04:06:51 +00:00
/// Query tree
QueryTreeNodePtr query_tree;
2022-09-07 17:09:37 +00:00
/// Planner context
PlannerContextPtr planner_context;
/// Storage table expression
/// It's guaranteed to be present in JOIN TREE of `query_tree`
2022-09-07 17:09:37 +00:00
QueryTreeNodePtr table_expression;
/// Table expression modifiers for storage
std::optional<TableExpressionModifiers> table_expression_modifiers;
2022-05-31 14:43:38 +00:00
std::shared_ptr<const StorageLimitsList> storage_limits;
/// Local storage limits
StorageLimits local_storage_limits;
/// Cluster for the query.
ClusterPtr cluster;
/// Optimized cluster for the query.
/// In case of optimize_skip_unused_shards it may differs from original cluster.
///
/// Configured in StorageDistributed::getQueryProcessingStage()
ClusterPtr optimized_cluster;
TreeRewriterResultPtr syntax_analyzer_result;
2022-06-21 11:24:46 +00:00
/// This is an additional filer applied to current table.
ASTPtr additional_filter_ast;
2022-07-28 09:40:09 +00:00
/// It is needed for PK analysis based on row_level_policy and additional_filters.
ASTs filter_asts;
ASTPtr parallel_replica_custom_key_ast;
2022-11-17 18:44:26 +00:00
/// Filter actions dag for current storage
ActionsDAGPtr filter_actions_dag;
2020-05-13 13:49:10 +00:00
ReadInOrderOptimizerPtr order_optimizer;
/// Can be modified while reading from storage
InputOrderInfoPtr input_order_info;
2019-12-10 23:18:24 +00:00
2017-07-15 04:06:51 +00:00
/// Prepared sets are used for indices by storage engine.
/// Example: x IN (1, 2, 3)
PreparedSetsPtr prepared_sets;
/// Cached value of ExpressionAnalysisResult
bool has_window = false;
bool has_order_by = false;
bool need_aggregate = false;
PrewhereInfoPtr prewhere_info;
/// If query has aggregate functions
bool has_aggregates = false;
/// If query has any filter and no arrayJoin before filter. Used by skipping FINAL
/// Skipping FINAL algorithm will output the original chunk and a column indices of
/// selected rows. If query has filter and doesn't have array join before any filter,
/// we can merge the indices with the first filter in FilterTransform later.
bool has_filters_and_no_array_join_before_filter = false;
ClusterPtr getCluster() const { return !optimized_cluster ? cluster : optimized_cluster; }
bool settings_limit_offset_done = false;
bool is_internal = false;
bool parallel_replicas_disabled = false;
bool is_parameterized_view = false;
bool optimize_trivial_count = false;
2022-10-23 21:22:52 +00:00
// If limit is not 0, that means it's a trivial limit query.
UInt64 limit = 0;
/// For IStorageSystemOneBlock
std::vector<UInt8> columns_mask;
2023-05-23 18:36:02 +00:00
bool isFinal() const;
2022-03-30 03:04:31 +00:00
};
2017-07-15 04:06:51 +00:00
}