2014-09-17 13:05:18 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/Block.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2017-07-15 03:48:36 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2023-03-20 10:29:27 +00:00
|
|
|
#include <Storages/MergeTree/RPNBuilder.h>
|
2014-09-17 13:05:18 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <boost/noncopyable.hpp>
|
|
|
|
|
|
|
|
#include <memory>
|
|
|
|
#include <set>
|
|
|
|
#include <unordered_map>
|
|
|
|
|
2015-06-05 17:30:24 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
namespace Poco { class Logger; }
|
|
|
|
|
2014-09-17 13:05:18 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
class ASTSelectQuery;
|
|
|
|
class ASTFunction;
|
|
|
|
class MergeTreeData;
|
2020-06-17 12:39:20 +00:00
|
|
|
struct StorageInMemoryMetadata;
|
2020-06-18 11:02:31 +00:00
|
|
|
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2014-09-23 11:35:27 +00:00
|
|
|
/** Identifies WHERE expressions that can be placed in PREWHERE by calculating respective
|
|
|
|
* sizes of columns used in particular expression and identifying "good" conditions of
|
|
|
|
* form "column_name = constant", where "constant" is outside some `threshold` specified in advance.
|
|
|
|
*
|
2019-02-14 19:59:51 +00:00
|
|
|
* If there are "good" conditions present in WHERE, the one with minimal summary column size is transferred to PREWHERE.
|
|
|
|
* Otherwise any condition with minimal summary column size can be transferred to PREWHERE.
|
2020-11-27 12:47:27 +00:00
|
|
|
* If column sizes are unknown (in compact parts), the number of columns, participating in condition is used instead.
|
2014-09-23 11:35:27 +00:00
|
|
|
*/
|
2017-05-25 01:12:41 +00:00
|
|
|
class MergeTreeWhereOptimizer : private boost::noncopyable
|
2014-09-17 13:05:18 +00:00
|
|
|
{
|
|
|
|
public:
|
2014-10-01 16:38:04 +00:00
|
|
|
MergeTreeWhereOptimizer(
|
2021-02-06 15:08:42 +00:00
|
|
|
std::unordered_map<std::string, UInt64> column_sizes_,
|
2020-06-17 12:39:20 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-03-09 00:28:05 +00:00
|
|
|
const Names & queried_columns_,
|
2023-03-02 17:45:59 +00:00
|
|
|
const std::optional<NameSet> & supported_columns_,
|
2019-08-03 11:02:40 +00:00
|
|
|
Poco::Logger * log_);
|
2014-09-17 13:05:18 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
void optimize(SelectQueryInfo & select_query_info, const ContextPtr & context) const;
|
|
|
|
|
|
|
|
struct FilterActionsOptimizeResult
|
|
|
|
{
|
|
|
|
ActionsDAGPtr filter_actions;
|
|
|
|
ActionsDAGPtr prewhere_filter_actions;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::optional<FilterActionsOptimizeResult> optimize(const ActionsDAGPtr & filter_dag,
|
|
|
|
const std::string & filter_column_name,
|
|
|
|
const ContextPtr & context,
|
|
|
|
bool is_final);
|
2023-03-19 16:25:37 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
private:
|
2019-02-14 23:51:17 +00:00
|
|
|
struct Condition
|
|
|
|
{
|
2023-03-20 10:29:27 +00:00
|
|
|
explicit Condition(RPNBuilderTreeNode node_)
|
|
|
|
: node(std::move(node_))
|
|
|
|
{}
|
|
|
|
|
|
|
|
RPNBuilderTreeNode node;
|
|
|
|
|
2019-02-15 19:25:29 +00:00
|
|
|
UInt64 columns_size = 0;
|
2023-03-20 10:29:27 +00:00
|
|
|
NameSet table_columns;
|
2020-11-27 12:47:27 +00:00
|
|
|
|
|
|
|
/// Can condition be moved to prewhere?
|
2019-02-14 23:51:17 +00:00
|
|
|
bool viable = false;
|
2020-11-27 12:47:27 +00:00
|
|
|
|
|
|
|
/// Does the condition presumably have good selectivity?
|
2019-02-14 23:51:17 +00:00
|
|
|
bool good = false;
|
|
|
|
|
|
|
|
auto tuple() const
|
|
|
|
{
|
2023-03-20 10:29:27 +00:00
|
|
|
return std::make_tuple(!viable, !good, columns_size, table_columns.size());
|
2019-02-14 23:51:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Is condition a better candidate for moving to PREWHERE?
|
|
|
|
bool operator< (const Condition & rhs) const
|
|
|
|
{
|
|
|
|
return tuple() < rhs.tuple();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
using Conditions = std::list<Condition>;
|
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
struct WhereOptimizerContext
|
|
|
|
{
|
|
|
|
ContextPtr context;
|
|
|
|
NameSet array_joined_names;
|
|
|
|
bool move_all_conditions_to_prewhere = false;
|
|
|
|
bool is_final = false;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct OptimizeResult
|
|
|
|
{
|
|
|
|
Conditions where_conditions;
|
|
|
|
Conditions prewhere_conditions;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::optional<OptimizeResult> optimizeImpl(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const;
|
|
|
|
|
|
|
|
void analyzeImpl(Conditions & res, const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const;
|
2019-02-14 23:51:17 +00:00
|
|
|
|
|
|
|
/// Transform conjunctions chain in WHERE expression to Conditions list.
|
2023-03-20 10:29:27 +00:00
|
|
|
Conditions analyze(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const;
|
|
|
|
|
|
|
|
/// Reconstruct AST from conditions
|
|
|
|
static ASTPtr reconstructAST(const Conditions & conditions);
|
2019-02-14 23:51:17 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
/// Reconstruct DAG from conditions
|
|
|
|
static ActionsDAGPtr reconstructDAG(const Conditions & conditions, const ContextPtr & context);
|
2019-02-14 23:51:17 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
void optimizeArbitrary(ASTSelectQuery & select) const;
|
2014-09-18 16:06:56 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
UInt64 getColumnsSize(const NameSet & columns) const;
|
2023-03-14 17:46:26 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
bool columnsSupportPrewhere(const NameSet & columns) const;
|
|
|
|
|
|
|
|
bool isExpressionOverSortingKey(const RPNBuilderTreeNode & node) const;
|
2014-09-17 13:05:18 +00:00
|
|
|
|
2021-03-24 08:35:20 +00:00
|
|
|
bool isSortingKey(const String & column_name) const;
|
2021-03-17 10:49:24 +00:00
|
|
|
|
2017-01-14 09:00:19 +00:00
|
|
|
bool isConstant(const ASTPtr & expr) const;
|
2014-09-17 13:05:18 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
bool isSubsetOfTableColumns(const NameSet & columns) const;
|
2014-10-01 16:38:04 +00:00
|
|
|
|
2015-06-08 16:05:44 +00:00
|
|
|
/** ARRAY JOIN'ed columns as well as arrayJoin() result cannot be used in PREWHERE, therefore expressions
|
2015-07-15 05:07:00 +00:00
|
|
|
* containing said columns should not be moved to PREWHERE at all.
|
|
|
|
* We assume all AS aliases have been expanded prior to using this class
|
|
|
|
*
|
|
|
|
* Also, disallow moving expressions with GLOBAL [NOT] IN.
|
|
|
|
*/
|
2023-03-20 10:29:27 +00:00
|
|
|
bool cannotBeMoved(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const;
|
2015-06-08 16:05:44 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
static NameSet determineArrayJoinedNames(const ASTSelectQuery & select);
|
2015-09-21 12:13:05 +00:00
|
|
|
|
2023-03-20 10:29:27 +00:00
|
|
|
const NameSet table_columns;
|
2019-02-15 19:00:19 +00:00
|
|
|
const Names queried_columns;
|
2023-03-02 17:45:59 +00:00
|
|
|
const std::optional<NameSet> supported_columns;
|
2021-03-24 08:35:20 +00:00
|
|
|
const NameSet sorting_key_names;
|
2017-01-14 09:00:19 +00:00
|
|
|
Poco::Logger * log;
|
2019-02-15 19:25:29 +00:00
|
|
|
std::unordered_map<std::string, UInt64> column_sizes;
|
|
|
|
UInt64 total_size_of_queried_columns = 0;
|
2014-09-17 13:05:18 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
}
|