ClickHouse/src/Storages/MergeTree/RPNBuilder.h

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

292 lines
9.2 KiB
C++
Raw Normal View History

2019-03-07 09:15:58 +00:00
#pragma once
#include <Core/Block.h>
2022-10-26 10:43:29 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/Set.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/ActionsDAG.h>
2019-03-07 09:15:58 +00:00
2019-03-12 13:36:15 +00:00
namespace DB
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
/** Context of RPNBuilderTree.
*
2022-10-31 11:30:00 +00:00
* For AST tree context, precalculated block with constants and prepared sets are required for index analysis.
2022-10-26 10:43:29 +00:00
* For DAG tree precalculated block with constants and prepared sets are not required, because constants and sets already
* calculated inside COLUMN actions dag node.
*/
class RPNBuilderTreeContext
{
public:
/// Construct RPNBuilderTreeContext for ActionsDAG tree
explicit RPNBuilderTreeContext(ContextPtr query_context_);
/// Construct RPNBuilderTreeContext for AST tree
explicit RPNBuilderTreeContext(ContextPtr query_context_, Block block_with_constants_, PreparedSetsPtr prepared_sets_);
/// Get query context
const ContextPtr & getQueryContext() const
{
return query_context;
}
/// Get query context settings
const Settings & getSettings() const
{
return query_context->getSettingsRef();
}
2022-10-26 10:43:29 +00:00
/** Get block with constants.
* Valid only for AST tree.
*/
const Block & getBlockWithConstants() const
{
return block_with_constants;
}
/** Get prepared sets.
* Valid only for AST tree.
*/
const PreparedSetsPtr & getPreparedSets() const
{
return prepared_sets;
}
private:
/// Valid for both AST and ActionDAG tree
ContextPtr query_context;
/// Valid only for AST tree
Block block_with_constants;
/// Valid only for AST tree
PreparedSetsPtr prepared_sets;
};
class RPNBuilderFunctionTreeNode;
/** RPNBuilderTreeNode is wrapper around DAG or AST node.
* It defines unified interface for index analysis.
*/
class RPNBuilderTreeNode
{
public:
/// Construct RPNBuilderTreeNode with non null dag node and tree context
explicit RPNBuilderTreeNode(const ActionsDAG::Node * dag_node_, RPNBuilderTreeContext & tree_context_);
/// Construct RPNBuilderTreeNode with non null ast node and tree context
explicit RPNBuilderTreeNode(const IAST * ast_node_, RPNBuilderTreeContext & tree_context_);
/// Get AST node
const IAST * getASTNode() const { return ast_node; }
/// Get DAG node
const ActionsDAG::Node * getDAGNode() const { return dag_node; }
2022-10-26 10:43:29 +00:00
/// Get column name
std::string getColumnName() const;
/** Get column name.
* Function `modulo` is replaced with `moduloLegacy`.
*/
std::string getColumnNameWithModuloLegacy() const;
/// Is node function
bool isFunction() const;
/// Is node constant
bool isConstant() const;
bool isSubqueryOrSet() const;
2022-10-26 10:43:29 +00:00
/** Get constant as constant column.
* Node must be constant before calling these method, otherwise logical exception is thrown.
*/
ColumnWithTypeAndName getConstantColumn() const;
/** Try get constant from node. If node is constant returns true, and constant value and constant type output parameters are set.
* Otherwise false is returned.
*/
bool tryGetConstant(Field & output_value, DataTypePtr & output_type) const;
/// Try get prepared set from node
2023-05-04 17:54:08 +00:00
FutureSetPtr tryGetPreparedSet() const;
2022-10-26 10:43:29 +00:00
/// Try get prepared set from node that match data types
2023-05-04 17:54:08 +00:00
FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const;
2022-10-26 10:43:29 +00:00
/// Try get prepared set from node that match indexes mapping and data types
2023-05-04 17:54:08 +00:00
FutureSetPtr tryGetPreparedSet(
2022-10-26 10:43:29 +00:00
const std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const DataTypes & data_types) const;
/** Convert node to function node.
* Node must be function before calling these method, otherwise exception is thrown.
*/
RPNBuilderFunctionTreeNode toFunctionNode() const;
2022-10-31 11:30:00 +00:00
/// Convert node to function node or null optional
std::optional<RPNBuilderFunctionTreeNode> toFunctionNodeOrNull() const;
2022-10-26 10:43:29 +00:00
/// Get tree context
const RPNBuilderTreeContext & getTreeContext() const
{
return tree_context;
}
/// Get tree context
RPNBuilderTreeContext & getTreeContext()
{
return tree_context;
}
protected:
const IAST * ast_node = nullptr;
const ActionsDAG::Node * dag_node = nullptr;
RPNBuilderTreeContext & tree_context;
};
/** RPNBuilderFunctionTreeNode is wrapper around RPNBuilderTreeNode with function type.
* It provide additional functionality that is specific for function.
*/
class RPNBuilderFunctionTreeNode : public RPNBuilderTreeNode
{
public:
2022-10-31 11:30:00 +00:00
using RPNBuilderTreeNode::RPNBuilderTreeNode;
2022-10-26 10:43:29 +00:00
/// Get function name
std::string getFunctionName() const;
/// Get function arguments size
size_t getArgumentsSize() const;
2022-10-31 11:30:00 +00:00
/// Get function argument at index
2022-10-26 10:43:29 +00:00
RPNBuilderTreeNode getArgumentAt(size_t index) const;
};
/** RPN Builder build stack of reverse polish notation elements (RPNElements) required for index analysis.
*
* RPNBuilder client must provide RPNElement type that has following interface:
*
* struct RPNElementInterface
* {
* enum Function
* {
* FUNCTION_UNKNOWN, /// Can take any value.
* /// Operators of the logical expression.
* FUNCTION_NOT,
* FUNCTION_AND,
* FUNCTION_OR,
* ...
* };
*
* RPNElementInterface();
*
* Function function = FUNCTION_UNKNOWN;
*
* }
*
* RPNBuilder take care of building stack of RPNElements with `NOT`, `AND`, `OR` types.
* In addition client must provide ExtractAtomFromTreeFunction that returns true and RPNElement as output parameter,
* if it can convert RPNBuilderTree node to RPNElement, false otherwise.
*/
2019-03-07 09:15:58 +00:00
template <typename RPNElement>
2022-10-26 10:43:29 +00:00
class RPNBuilder
2019-03-07 09:15:58 +00:00
{
public:
2022-10-26 10:43:29 +00:00
using RPNElements = std::vector<RPNElement>;
using ExtractAtomFromTreeFunction = std::function<bool (const RPNBuilderTreeNode & node, RPNElement & out)>;
2019-03-07 09:15:58 +00:00
2022-10-26 13:54:37 +00:00
explicit RPNBuilder(const ActionsDAG::Node * filter_actions_dag_node,
2022-10-26 10:43:29 +00:00
ContextPtr query_context_,
const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_)
: tree_context(std::move(query_context_))
, extract_atom_from_tree_function(extract_atom_from_tree_function_)
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context));
}
2019-03-07 09:15:58 +00:00
2022-10-26 10:43:29 +00:00
RPNBuilder(const ASTPtr & filter_node,
ContextPtr query_context_,
Block block_with_constants_,
PreparedSetsPtr prepared_sets_,
const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_)
: tree_context(std::move(query_context_), std::move(block_with_constants_), std::move(prepared_sets_))
, extract_atom_from_tree_function(extract_atom_from_tree_function_)
{
traverseTree(RPNBuilderTreeNode(filter_node.get(), tree_context));
2019-03-07 09:15:58 +00:00
}
2022-10-26 10:43:29 +00:00
RPNElements && extractRPN() && { return std::move(rpn_elements); }
2019-03-07 09:15:58 +00:00
private:
2022-10-26 10:43:29 +00:00
void traverseTree(const RPNBuilderTreeNode & node)
2019-03-07 09:15:58 +00:00
{
RPNElement element;
2022-10-26 10:43:29 +00:00
if (node.isFunction())
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
auto function_node = node.toFunctionNode();
if (extractLogicalOperatorFromTree(function_node, element))
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
size_t arguments_size = function_node.getArgumentsSize();
for (size_t argument_index = 0; argument_index < arguments_size; ++argument_index)
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
auto function_node_argument = function_node.getArgumentAt(argument_index);
traverseTree(function_node_argument);
2019-03-07 09:15:58 +00:00
/** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity
* - in this case `n - 1` elements are added (where `n` is the number of arguments).
*/
2022-10-26 10:43:29 +00:00
if (argument_index != 0 || element.function == RPNElement::FUNCTION_NOT)
rpn_elements.emplace_back(std::move(element));
2019-03-07 09:15:58 +00:00
}
return;
}
}
2022-10-26 10:43:29 +00:00
if (!extract_atom_from_tree_function(node, element))
2019-03-07 09:15:58 +00:00
element.function = RPNElement::FUNCTION_UNKNOWN;
2022-10-26 10:43:29 +00:00
rpn_elements.emplace_back(std::move(element));
2019-03-07 09:15:58 +00:00
}
2022-10-26 10:43:29 +00:00
bool extractLogicalOperatorFromTree(const RPNBuilderFunctionTreeNode & function_node, RPNElement & out)
2019-03-07 09:15:58 +00:00
{
2022-10-31 11:30:00 +00:00
/** Functions AND, OR, NOT.
* Also a special function `indexHint` - works as if instead of calling a function there are just parentheses
* (or, the same thing - calling the function `and` from one argument).
*/
2019-03-07 09:15:58 +00:00
2022-10-26 10:43:29 +00:00
auto function_name = function_node.getFunctionName();
if (function_name == "not")
2019-03-07 09:15:58 +00:00
{
2022-10-26 10:43:29 +00:00
if (function_node.getArgumentsSize() != 1)
2019-03-07 09:15:58 +00:00
return false;
out.function = RPNElement::FUNCTION_NOT;
}
else
{
2022-10-26 10:43:29 +00:00
if (function_name == "and" || function_name == "indexHint")
2019-03-07 09:15:58 +00:00
out.function = RPNElement::FUNCTION_AND;
2022-10-26 10:43:29 +00:00
else if (function_name == "or")
2019-03-07 09:15:58 +00:00
out.function = RPNElement::FUNCTION_OR;
else
return false;
}
return true;
}
2022-10-26 10:43:29 +00:00
RPNBuilderTreeContext tree_context;
const ExtractAtomFromTreeFunction & extract_atom_from_tree_function;
RPNElements rpn_elements;
2019-03-07 09:15:58 +00:00
};
2022-05-16 18:59:27 +00:00
}