MergeTree indexes use ActionsDAG

This commit is contained in:
Maksim Kita 2022-10-26 12:43:29 +02:00
parent b4db75d7cd
commit 1b6293f6db
11 changed files with 1063 additions and 831 deletions

View File

@ -906,8 +906,11 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
for (const auto & node : added_filter_nodes.nodes) for (const auto & node : added_filter_nodes.nodes)
nodes.nodes.push_back(node); nodes.nodes.push_back(node);
key_condition.emplace( key_condition.emplace(std::move(nodes),
std::move(nodes), query_info.syntax_analyzer_result, query_info.prepared_sets, context, primary_key_columns, primary_key.expression); context,
primary_key_columns,
primary_key.expression,
query_info.syntax_analyzer_result->getArrayJoinSourceNameSet());
} }
else else
{ {

View File

@ -27,6 +27,7 @@
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Storages/KeyDescription.h> #include <Storages/KeyDescription.h>
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
#include <cassert> #include <cassert>
#include <stack> #include <stack>
@ -112,289 +113,6 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr
return res; return res;
} }
static void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false)
{
switch (node.type)
{
case (ActionsDAG::ActionType::INPUT):
writeString(node.result_name, out);
break;
case (ActionsDAG::ActionType::COLUMN):
{
/// If it was created from ASTLiteral, then result_name can be an alias.
/// We need to convert value back to string here.
if (const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get()))
writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out);
/// It may be possible that column is ColumnSet
else
writeString(node.result_name, out);
break;
}
case (ActionsDAG::ActionType::ALIAS):
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
break;
case (ActionsDAG::ActionType::ARRAY_JOIN):
writeCString("arrayJoin(", out);
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
writeChar(')', out);
break;
case (ActionsDAG::ActionType::FUNCTION):
{
auto name = node.function_base->getName();
if (legacy && name == "modulo")
writeCString("moduleLegacy", out);
else
writeString(name, out);
writeChar('(', out);
bool first = true;
for (const auto * arg : node.children)
{
if (!first)
writeCString(", ", out);
first = false;
appendColumnNameWithoutAlias(*arg, out, legacy);
}
writeChar(')', out);
}
}
}
static std::string getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false)
{
WriteBufferFromOwnString out;
appendColumnNameWithoutAlias(node, out, legacy);
return std::move(out.str());
}
class KeyCondition::Tree
{
public:
explicit Tree(const IAST * ast_) : ast(ast_) { assert(ast); }
explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) { assert(dag); }
std::string getColumnName() const
{
if (ast)
return ast->getColumnNameWithoutAlias();
else
return getColumnNameWithoutAlias(*dag);
}
std::string getColumnNameLegacy() const
{
if (ast)
{
auto adjusted_ast = ast->clone();
KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast);
return adjusted_ast->getColumnNameWithoutAlias();
}
else
return getColumnNameWithoutAlias(*dag, true);
}
bool isFunction() const
{
if (ast)
return typeid_cast<const ASTFunction *>(ast);
else
return dag->type == ActionsDAG::ActionType::FUNCTION;
}
bool isConstant() const
{
if (ast)
return typeid_cast<const ASTLiteral *>(ast);
else
return dag->column && isColumnConst(*dag->column);
}
ColumnWithTypeAndName getConstant() const
{
if (!isConstant())
throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a constant");
ColumnWithTypeAndName res;
if (ast)
{
const auto * literal = assert_cast<const ASTLiteral *>(ast);
res.type = applyVisitor(FieldToDataType(), literal->value);
res.column = res.type->createColumnConst(0, literal->value);
}
else
{
res.type = dag->result_type;
res.column = dag->column;
}
return res;
}
bool tryGetConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const
{
if (ast)
{
// Constant expr should use alias names if any
String column_name = ast->getColumnName();
if (const auto * lit = ast->as<ASTLiteral>())
{
/// By default block_with_constants has only one column named "_dummy".
/// If block contains only constants it's may not be preprocessed by
// ExpressionAnalyzer, so try to look up in the default column.
if (!block_with_constants.has(column_name))
column_name = "_dummy";
/// Simple literal
out_value = lit->value;
out_type = block_with_constants.getByName(column_name).type;
/// If constant is not Null, we can assume it's type is not Nullable as well.
if (!out_value.isNull())
out_type = removeNullable(out_type);
return true;
}
else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column))
{
/// An expression which is dependent on constants only
const auto & expr_info = block_with_constants.getByName(column_name);
out_value = (*expr_info.column)[0];
out_type = expr_info.type;
if (!out_value.isNull())
out_type = removeNullable(out_type);
return true;
}
}
else
{
if (dag->column && isColumnConst(*dag->column))
{
out_value = (*dag->column)[0];
out_type = dag->result_type;
if (!out_value.isNull())
out_type = removeNullable(out_type);
return true;
}
}
return false;
}
ConstSetPtr tryGetPreparedSet(
const PreparedSetsPtr & sets,
const std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const DataTypes & data_types) const
{
if (sets && ast)
{
if (ast->as<ASTSubquery>() || ast->as<ASTTableIdentifier>())
return sets->get(PreparedSetKey::forSubquery(*ast));
/// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information
/// about types in left argument of the IN operator. Instead, we manually iterate through all the sets
/// and find the one for the right arg based on the AST structure (getTreeHash), after that we check
/// that the types it was prepared with are compatible with the types of the primary key.
auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set)
{
assert(indexes_mapping.size() == data_types.size());
for (size_t i = 0; i < indexes_mapping.size(); ++i)
{
if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i]))
return false;
}
return true;
};
for (const auto & set : sets->getByTreeHash(ast->getTreeHash()))
{
if (types_match(set))
return set;
}
}
else if (dag->column)
{
const IColumn * col = dag->column.get();
if (const auto * col_const = typeid_cast<const ColumnConst *>(col))
col = &col_const->getDataColumn();
if (const auto * col_set = typeid_cast<const ColumnSet *>(col))
{
auto set = col_set->getData();
if (set->isCreated())
return set;
}
}
return nullptr;
}
FunctionTree asFunction() const;
protected:
const IAST * ast = nullptr;
const ActionsDAG::Node * dag = nullptr;
};
class KeyCondition::FunctionTree : public KeyCondition::Tree
{
public:
std::string getFunctionName() const
{
if (ast)
return assert_cast<const ASTFunction *>(ast)->name;
else
return dag->function_base->getName();
}
size_t numArguments() const
{
if (ast)
{
const auto * func = assert_cast<const ASTFunction *>(ast);
return func->arguments ? func->arguments->children.size() : 0;
}
else
return dag->children.size();
}
Tree getArgumentAt(size_t idx) const
{
if (ast)
return Tree(assert_cast<const ASTFunction *>(ast)->arguments->children[idx].get());
else
return Tree(dag->children[idx]);
}
private:
using Tree::Tree;
friend class Tree;
};
KeyCondition::FunctionTree KeyCondition::Tree::asFunction() const
{
if (!isFunction())
throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a function");
if (ast)
return KeyCondition::FunctionTree(ast);
else
return KeyCondition::FunctionTree(dag);
}
/// A dictionary containing actions to the corresponding functions to turn them into `RPNElement`
const KeyCondition::AtomMap KeyCondition::atom_map const KeyCondition::AtomMap KeyCondition::atom_map
{ {
{ {
@ -869,16 +587,17 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr)
KeyCondition::KeyCondition( KeyCondition::KeyCondition(
const ASTPtr & query, const ASTPtr & query,
const ASTs & additional_filter_asts, const ASTs & additional_filter_asts,
TreeRewriterResultPtr syntax_analyzer_result, Block block_with_constants,
PreparedSetsPtr prepared_sets_, PreparedSetsPtr prepared_sets,
ContextPtr context, ContextPtr context,
const Names & key_column_names, const Names & key_column_names,
const ExpressionActionsPtr & key_expr_, const ExpressionActionsPtr & key_expr_,
NameSet array_joined_column_names_,
bool single_point_, bool single_point_,
bool strict_) bool strict_)
: key_expr(key_expr_) : key_expr(key_expr_)
, key_subexpr_names(getAllSubexpressionNames(*key_expr)) , key_subexpr_names(getAllSubexpressionNames(*key_expr))
, prepared_sets(prepared_sets_) , array_joined_column_names(std::move(array_joined_column_names_))
, single_point(single_point_) , single_point(single_point_)
, strict(strict_) , strict(strict_)
{ {
@ -889,82 +608,64 @@ KeyCondition::KeyCondition(
key_columns[name] = i; key_columns[name] = i;
} }
if (!syntax_analyzer_result) auto filter_node = buildFilterNode(query, additional_filter_asts);
if (!filter_node)
{ {
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
return; return;
} }
/** Evaluation of expressions that depend only on constants. /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. * the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
* when parsing the AST into internal RPN representation.
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
* are pushed down and applied (when possible) to leaf nodes.
*/ */
Block block_with_constants = getBlockWithConstants(query, syntax_analyzer_result, context); auto inverted_filter_node = cloneASTWithInversionPushDown(filter_node);
if (syntax_analyzer_result) RPNBuilder<RPNElement> builder(
{ inverted_filter_node,
for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) std::move(context),
array_joined_columns.insert(name); std::move(block_with_constants),
} std::move(prepared_sets),
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
}
const ASTSelectQuery & select = query->as<ASTSelectQuery &>(); KeyCondition::KeyCondition(
const SelectQueryInfo & query_info,
ASTs filters; ContextPtr context,
if (select.where()) const Names & key_column_names,
filters.push_back(select.where()); const ExpressionActionsPtr & key_expr_,
bool single_point_,
if (select.prewhere()) bool strict_)
filters.push_back(select.prewhere()); : KeyCondition(
query_info.query,
for (const auto & filter_ast : additional_filter_asts) query_info.filter_asts,
filters.push_back(filter_ast); KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context),
query_info.prepared_sets,
if (!filters.empty()) context,
{ key_column_names,
ASTPtr filter_query; key_expr_,
if (filters.size() == 1) query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(),
{ single_point_,
filter_query = filters.front(); strict_)
} {
else
{
auto function = std::make_shared<ASTFunction>();
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = std::move(filters);
filter_query = function;
}
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
* when parsing the AST into internal RPN representation.
* To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's
* are pushed down and applied (when possible) to leaf nodes.
*/
auto ast = cloneASTWithInversionPushDown(filter_query);
traverseAST(Tree(ast.get()), context, block_with_constants);
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
} }
KeyCondition::KeyCondition( KeyCondition::KeyCondition(
ActionDAGNodes dag_nodes, ActionDAGNodes dag_nodes,
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
ContextPtr context, ContextPtr context,
const Names & key_column_names, const Names & key_column_names,
const ExpressionActionsPtr & key_expr_, const ExpressionActionsPtr & key_expr_,
NameSet array_joined_column_names_,
bool single_point_, bool single_point_,
bool strict_) bool strict_)
: key_expr(key_expr_) : key_expr(key_expr_)
, key_subexpr_names(getAllSubexpressionNames(*key_expr)) , key_subexpr_names(getAllSubexpressionNames(*key_expr))
, prepared_sets(prepared_sets_) , array_joined_column_names(std::move(array_joined_column_names_))
, single_point(single_point_) , single_point(single_point_)
, strict(strict_) , strict(strict_)
{ {
@ -975,29 +676,23 @@ KeyCondition::KeyCondition(
key_columns[name] = i; key_columns[name] = i;
} }
if (!syntax_analyzer_result) if (dag_nodes.nodes.empty())
{ {
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
return; return;
} }
for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context);
array_joined_columns.insert(name); assert(inverted_dag->getOutputs().size() == 1);
if (!dag_nodes.nodes.empty()) const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0];
RPNBuilder<RPNElement> builder(inverted_dag_filter_node, context, [&](const RPNBuilderTreeNode & node, RPNElement & out)
{ {
auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); return extractAtomFromTree(node, out);
});
// std::cerr << "========== inverted dag: " << inverted_dag->dumpDAG() << std::endl; rpn = std::move(builder).extractRPN();
Block empty;
for (const auto * node : inverted_dag->getOutputs())
traverseAST(Tree(node), context, empty);
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
} }
bool KeyCondition::addCondition(const String & column, const Range & range) bool KeyCondition::addCondition(const String & column, const Range & range)
@ -1009,12 +704,12 @@ bool KeyCondition::addCondition(const String & column, const Range & range)
return true; return true;
} }
/** Computes value of constant expression and its data type.
* Returns false, if expression isn't constant.
*/
bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type)
{ {
return Tree(expr.get()).tryGetConstant(block_with_constants, out_value, out_type); RPNBuilderTreeContext tree_context(nullptr, block_with_constants, nullptr);
RPNBuilderTreeNode node(expr.get(), tree_context);
return node.tryGetConstant(out_value, out_type);
} }
@ -1098,39 +793,6 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr &
return {field.columns, field.row_idx, result_idx}; return {field.columns, field.row_idx, result_idx};
} }
void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants)
{
RPNElement element;
if (node.isFunction())
{
auto func = node.asFunction();
if (tryParseLogicalOperatorFromAST(func, element))
{
size_t num_args = func.numArguments();
for (size_t i = 0; i < num_args; ++i)
{
traverseAST(func.getArgumentAt(i), context, block_with_constants);
/** 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).
*/
if (i != 0 || element.function == RPNElement::FUNCTION_NOT)
rpn.emplace_back(element);
}
return;
}
}
if (!tryParseAtomFromAST(node, context, block_with_constants, element))
{
element.function = RPNElement::FUNCTION_UNKNOWN;
}
rpn.emplace_back(std::move(element));
}
/** The key functional expression constraint may be inferred from a plain column in the expression. /** The key functional expression constraint may be inferred from a plain column in the expression.
* For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`,
* it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())`
@ -1251,7 +913,7 @@ bool KeyCondition::transformConstantWithValidFunctions(
} }
bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
const Tree & node, const RPNBuilderTreeNode & node,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_column_type, DataTypePtr & out_key_column_type,
Field & out_value, Field & out_value,
@ -1259,7 +921,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
{ {
String expr_name = node.getColumnName(); String expr_name = node.getColumnName();
if (array_joined_columns.contains(expr_name)) if (array_joined_column_names.contains(expr_name))
return false; return false;
if (!key_subexpr_names.contains(expr_name)) if (!key_subexpr_names.contains(expr_name))
@ -1286,11 +948,15 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
/// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)`
bool KeyCondition::canConstantBeWrappedByFunctions( bool KeyCondition::canConstantBeWrappedByFunctions(
const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) const RPNBuilderTreeNode & node,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type)
{ {
String expr_name = node.getColumnName(); String expr_name = node.getColumnName();
if (array_joined_columns.contains(expr_name)) if (array_joined_column_names.contains(expr_name))
return false; return false;
if (!key_subexpr_names.contains(expr_name)) if (!key_subexpr_names.contains(expr_name))
@ -1304,7 +970,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions(
/// The case `f(modulo(...))` for totally monotonic `f ` is considered to be rare. /// The case `f(modulo(...))` for totally monotonic `f ` is considered to be rare.
/// ///
/// Note: for negative values, we can filter more partitions then needed. /// Note: for negative values, we can filter more partitions then needed.
expr_name = node.getColumnNameLegacy(); expr_name = node.getColumnNameWithModuloLegacy();
if (!key_subexpr_names.contains(expr_name)) if (!key_subexpr_names.contains(expr_name))
return false; return false;
@ -1321,8 +987,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions(
} }
bool KeyCondition::tryPrepareSetIndex( bool KeyCondition::tryPrepareSetIndex(
const FunctionTree & func, const RPNBuilderFunctionTreeNode & func,
ContextPtr context,
RPNElement & out, RPNElement & out,
size_t & out_key_column_num) size_t & out_key_column_num)
{ {
@ -1332,13 +997,12 @@ bool KeyCondition::tryPrepareSetIndex(
std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping; std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> indexes_mapping;
DataTypes data_types; DataTypes data_types;
auto get_key_tuple_position_mapping = [&](const Tree & node, size_t tuple_index) auto get_key_tuple_position_mapping = [&](const RPNBuilderTreeNode & node, size_t tuple_index)
{ {
MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; MergeTreeSetIndex::KeyTuplePositionMapping index_mapping;
index_mapping.tuple_index = tuple_index; index_mapping.tuple_index = tuple_index;
DataTypePtr data_type; DataTypePtr data_type;
if (isKeyPossiblyWrappedByMonotonicFunctions( if (isKeyPossiblyWrappedByMonotonicFunctions(node, index_mapping.key_index, data_type, index_mapping.functions))
node, context, index_mapping.key_index, data_type, index_mapping.functions))
{ {
indexes_mapping.push_back(index_mapping); indexes_mapping.push_back(index_mapping);
data_types.push_back(data_type); data_types.push_back(data_type);
@ -1352,25 +1016,29 @@ bool KeyCondition::tryPrepareSetIndex(
{ {
/// Note: in case of ActionsDAG, tuple may be a constant. /// Note: in case of ActionsDAG, tuple may be a constant.
/// In this case, there is no keys in tuple. So, we don't have to check it. /// In this case, there is no keys in tuple. So, we don't have to check it.
auto left_arg_tuple = left_arg.asFunction(); auto left_arg_tuple = left_arg.toFunctionNode();
if (left_arg_tuple.getFunctionName() == "tuple") if (left_arg_tuple.getFunctionName() == "tuple")
{ {
left_args_count = left_arg_tuple.numArguments(); left_args_count = left_arg_tuple.getArgumentsSize();
for (size_t i = 0; i < left_args_count; ++i) for (size_t i = 0; i < left_args_count; ++i)
get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i); get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i);
} }
else else
{
get_key_tuple_position_mapping(left_arg, 0); get_key_tuple_position_mapping(left_arg, 0);
}
} }
else else
{
get_key_tuple_position_mapping(left_arg, 0); get_key_tuple_position_mapping(left_arg, 0);
}
if (indexes_mapping.empty()) if (indexes_mapping.empty())
return false; return false;
const auto right_arg = func.getArgumentAt(1); const auto right_arg = func.getArgumentAt(1);
auto prepared_set = right_arg.tryGetPreparedSet(prepared_sets, indexes_mapping, data_types); auto prepared_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types);
if (!prepared_set) if (!prepared_set)
return false; return false;
@ -1462,13 +1130,12 @@ private:
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
const Tree & node, const RPNBuilderTreeNode & node,
ContextPtr context,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type, DataTypePtr & out_key_res_column_type,
MonotonicFunctionsChain & out_functions_chain) MonotonicFunctionsChain & out_functions_chain)
{ {
std::vector<FunctionTree> chain_not_tested_for_monotonicity; std::vector<RPNBuilderFunctionTreeNode> chain_not_tested_for_monotonicity;
DataTypePtr key_column_type; DataTypePtr key_column_type;
if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity)) if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity))
@ -1477,17 +1144,17 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it) for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
{ {
auto function = *it; auto function = *it;
auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), context); auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), node.getTreeContext().getQueryContext());
if (!func_builder) if (!func_builder)
return false; return false;
ColumnsWithTypeAndName arguments; ColumnsWithTypeAndName arguments;
ColumnWithTypeAndName const_arg; ColumnWithTypeAndName const_arg;
FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST;
if (function.numArguments() == 2) if (function.getArgumentsSize() == 2)
{ {
if (function.getArgumentAt(0).isConstant()) if (function.getArgumentAt(0).isConstant())
{ {
const_arg = function.getArgumentAt(0).getConstant(); const_arg = function.getArgumentAt(0).getConstantColumn();
arguments.push_back(const_arg); arguments.push_back(const_arg);
arguments.push_back({ nullptr, key_column_type, "" }); arguments.push_back({ nullptr, key_column_type, "" });
kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST;
@ -1495,7 +1162,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
else if (function.getArgumentAt(1).isConstant()) else if (function.getArgumentAt(1).isConstant())
{ {
arguments.push_back({ nullptr, key_column_type, "" }); arguments.push_back({ nullptr, key_column_type, "" });
const_arg = function.getArgumentAt(1).getConstant(); const_arg = function.getArgumentAt(1).getConstantColumn();
arguments.push_back(const_arg); arguments.push_back(const_arg);
kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST;
} }
@ -1521,10 +1188,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
} }
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const Tree & node, const RPNBuilderTreeNode & node,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_column_type, DataTypePtr & out_key_column_type,
std::vector<FunctionTree> & out_functions_chain) std::vector<RPNBuilderFunctionTreeNode> & out_functions_chain)
{ {
/** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`. /** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`.
* Therefore, use the full name of the expression for search. * Therefore, use the full name of the expression for search.
@ -1534,7 +1201,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
// Key columns should use canonical names for index analysis // Key columns should use canonical names for index analysis
String name = node.getColumnName(); String name = node.getColumnName();
if (array_joined_columns.contains(name)) if (array_joined_column_names.contains(name))
return false; return false;
auto it = key_columns.find(name); auto it = key_columns.find(name);
@ -1547,37 +1214,39 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
if (node.isFunction()) if (node.isFunction())
{ {
auto func = node.asFunction(); auto function_node = node.toFunctionNode();
size_t num_args = func.numArguments(); size_t arguments_size = function_node.getArgumentsSize();
if (num_args > 2 || num_args == 0) if (arguments_size > 2 || arguments_size == 0)
return false; return false;
out_functions_chain.push_back(func); out_functions_chain.push_back(function_node);
bool ret = false;
if (num_args == 2) bool result = false;
if (arguments_size == 2)
{ {
if (func.getArgumentAt(0).isConstant()) if (function_node.getArgumentAt(0).isConstant())
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain); result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain);
} }
else if (func.getArgumentAt(1).isConstant()) else if (function_node.getArgumentAt(1).isConstant())
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
} }
} }
else else
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
} }
return ret;
return result;
} }
return false; return false;
} }
static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const KeyCondition::Tree & node) static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name)
{ {
try try
{ {
@ -1587,13 +1256,13 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
{ {
throw Exception("Key expression contains comparison between inconvertible types: " + throw Exception("Key expression contains comparison between inconvertible types: " +
desired_type->getName() + " and " + src_type->getName() + desired_type->getName() + " and " + src_type->getName() +
" inside " + node.getColumnName(), " inside " + node_column_name,
ErrorCodes::BAD_TYPE_OF_FIELD); ErrorCodes::BAD_TYPE_OF_FIELD);
} }
} }
bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out) bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
{ {
/** Functions < > = != <= >= in `notIn` isNull isNotNull, where one argument is a constant, and the other is one of columns of key, /** Functions < > = != <= >= in `notIn` isNull isNotNull, where one argument is a constant, and the other is one of columns of key,
* or itself, wrapped in a chain of possibly-monotonic functions, * or itself, wrapped in a chain of possibly-monotonic functions,
@ -1603,8 +1272,8 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
DataTypePtr const_type; DataTypePtr const_type;
if (node.isFunction()) if (node.isFunction())
{ {
auto func = node.asFunction(); auto func = node.toFunctionNode();
size_t num_args = func.numArguments(); size_t num_args = func.getArgumentsSize();
DataTypePtr key_expr_type; /// Type of expression containing key column DataTypePtr key_expr_type; /// Type of expression containing key column
size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) size_t key_column_num = -1; /// Number of a key column (inside key_column_names array)
@ -1616,7 +1285,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
if (num_args == 1) if (num_args == 1)
{ {
if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain))) if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain)))
return false; return false;
if (key_column_num == static_cast<size_t>(-1)) if (key_column_num == static_cast<size_t>(-1))
@ -1647,7 +1316,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
if (functionIsInOrGlobalInOperator(func_name)) if (functionIsInOrGlobalInOperator(func_name))
{ {
if (tryPrepareSetIndex(func, context, out, key_column_num)) if (tryPrepareSetIndex(func, out, key_column_num))
{ {
key_arg_pos = 0; key_arg_pos = 0;
is_set_const = true; is_set_const = true;
@ -1655,7 +1324,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
else else
return false; return false;
} }
else if (func.getArgumentAt(1).tryGetConstant(block_with_constants, const_value, const_type)) else if (func.getArgumentAt(1).tryGetConstant(const_value, const_type))
{ {
/// If the const operand is null, the atom will be always false /// If the const operand is null, the atom will be always false
if (const_value.isNull()) if (const_value.isNull())
@ -1664,7 +1333,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
return true; return true;
} }
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain)) if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain))
{ {
key_arg_pos = 0; key_arg_pos = 0;
} }
@ -1685,7 +1354,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
else else
return false; return false;
} }
else if (func.getArgumentAt(0).tryGetConstant(block_with_constants, const_value, const_type)) else if (func.getArgumentAt(0).tryGetConstant(const_value, const_type))
{ {
/// If the const operand is null, the atom will be always false /// If the const operand is null, the atom will be always false
if (const_value.isNull()) if (const_value.isNull())
@ -1694,7 +1363,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
return true; return true;
} }
if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), context, key_column_num, key_expr_type, chain)) if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, chain))
{ {
key_arg_pos = 1; key_arg_pos = 1;
} }
@ -1774,7 +1443,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
if (!const_type->equals(*common_type)) if (!const_type->equals(*common_type))
{ {
castValueToType(common_type, const_value, const_type, node); castValueToType(common_type, const_value, const_type, node.getColumnName());
// Need to set is_constant_transformed unless we're doing exact conversion // Need to set is_constant_transformed unless we're doing exact conversion
if (!key_expr_type_not_null->equals(*common_type)) if (!key_expr_type_not_null->equals(*common_type))
@ -1819,7 +1488,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
return atom_it->second(out, const_value); return atom_it->second(out, const_value);
} }
else if (node.tryGetConstant(block_with_constants, const_value, const_type)) else if (node.tryGetConstant(const_value, const_type))
{ {
/// For cases where it says, for example, `WHERE 0 AND something` /// For cases where it says, for example, `WHERE 0 AND something`
@ -1842,32 +1511,6 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl
return false; return false;
} }
bool KeyCondition::tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out)
{
/// 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).
if (func.getFunctionName() == "not")
{
if (func.numArguments() != 1)
return false;
out.function = RPNElement::FUNCTION_NOT;
}
else
{
if (func.getFunctionName() == "and" || func.getFunctionName() == "indexHint")
out.function = RPNElement::FUNCTION_AND;
else if (func.getFunctionName() == "or")
out.function = RPNElement::FUNCTION_OR;
else
return false;
}
return true;
}
String KeyCondition::toString() const String KeyCondition::toString() const
{ {
String res; String res;

View File

@ -2,11 +2,16 @@
#include <optional> #include <optional>
#include <Interpreters/Set.h>
#include <Core/SortDescription.h> #include <Core/SortDescription.h>
#include <Parsers/ASTExpressionList.h>
#include <Storages/SelectQueryInfo.h>
#include <Parsers/ASTExpressionList.h>
#include <Interpreters/Set.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/TreeRewriter.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/RPNBuilder.h>
namespace DB namespace DB
{ {
@ -205,45 +210,37 @@ public:
class KeyCondition class KeyCondition
{ {
public: public:
/// Does not take into account the SAMPLE section. all_columns - the set of all columns of the table. /// Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters
KeyCondition( KeyCondition(
const ASTPtr & query, const ASTPtr & query,
const ASTs & additional_filter_asts, const ASTs & additional_filter_asts,
TreeRewriterResultPtr syntax_analyzer_result, Block block_with_constants,
PreparedSetsPtr prepared_sets_, PreparedSetsPtr prepared_sets_,
ContextPtr context, ContextPtr context,
const Names & key_column_names, const Names & key_column_names,
const ExpressionActionsPtr & key_expr, const ExpressionActionsPtr & key_expr,
NameSet array_joined_column_names,
bool single_point_ = false, bool single_point_ = false,
bool strict_ = false); bool strict_ = false);
/** Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters.
* Select query, additional filters, prepared sets are initialized using query info.
*/
KeyCondition( KeyCondition(
const SelectQueryInfo & query_info, const SelectQueryInfo & query_info,
ContextPtr context, ContextPtr context,
const Names & key_column_names, const Names & key_column_names,
const ExpressionActionsPtr & key_expr_, const ExpressionActionsPtr & key_expr_,
bool single_point_ = false, bool single_point_ = false,
bool strict_ = false) bool strict_ = false);
: KeyCondition(
query_info.query,
query_info.filter_asts,
query_info.syntax_analyzer_result,
query_info.prepared_sets,
context,
key_column_names,
key_expr_,
single_point_,
strict_)
{
}
/// Construct key condition from ActionsDAG nodes
KeyCondition( KeyCondition(
ActionDAGNodes dag_nodes, ActionDAGNodes dag_nodes,
TreeRewriterResultPtr syntax_analyzer_result,
PreparedSetsPtr prepared_sets_,
ContextPtr context, ContextPtr context,
const Names & key_column_names, const Names & key_column_names,
const ExpressionActionsPtr & key_expr, const ExpressionActionsPtr & key_expr,
NameSet array_joined_column_names,
bool single_point_ = false, bool single_point_ = false,
bool strict_ = false); bool strict_ = false);
@ -275,6 +272,7 @@ public:
/// Checks that the index can not be used /// Checks that the index can not be used
/// FUNCTION_UNKNOWN will be AND'ed (if any). /// FUNCTION_UNKNOWN will be AND'ed (if any).
bool alwaysUnknownOrTrue() const; bool alwaysUnknownOrTrue() const;
/// Checks that the index can not be used /// Checks that the index can not be used
/// Does not allow any FUNCTION_UNKNOWN (will instantly return true). /// Does not allow any FUNCTION_UNKNOWN (will instantly return true).
bool anyUnknownOrAlwaysTrue() const; bool anyUnknownOrAlwaysTrue() const;
@ -313,10 +311,18 @@ public:
* Returns false, if expression isn't constant. * Returns false, if expression isn't constant.
*/ */
static bool getConstant( static bool getConstant(
const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type); const ASTPtr & expr,
Block & block_with_constants,
Field & out_value,
DataTypePtr & out_type);
/** Calculate expressions, that depend only on constants.
* For index to work when something like "WHERE Date = toDate(now())" is written.
*/
static Block getBlockWithConstants( static Block getBlockWithConstants(
const ASTPtr & query, const TreeRewriterResultPtr & syntax_analyzer_result, ContextPtr context); const ASTPtr & query,
const TreeRewriterResultPtr & syntax_analyzer_result,
ContextPtr context);
static std::optional<Range> applyMonotonicFunctionsChainToRange( static std::optional<Range> applyMonotonicFunctionsChainToRange(
Range key_range, Range key_range,
@ -373,14 +379,11 @@ private:
using RPN = std::vector<RPNElement>; using RPN = std::vector<RPNElement>;
using ColumnIndices = std::map<String, size_t>; using ColumnIndices = std::map<String, size_t>;
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value)>;
public: public:
using AtomMap = std::unordered_map<std::string, bool(*)(RPNElement & out, const Field & value)>;
static const AtomMap atom_map; static const AtomMap atom_map;
class Tree;
class FunctionTree;
private: private:
BoolMask checkInRange( BoolMask checkInRange(
size_t used_key_size, size_t used_key_size,
@ -390,9 +393,7 @@ private:
bool right_bounded, bool right_bounded,
BoolMask initial_mask) const; BoolMask initial_mask) const;
void traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants); bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out);
bool tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out);
static bool tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out);
/** Is node the key column /** Is node the key column
* or expression in which column of key is wrapped by chain of functions, * or expression in which column of key is wrapped by chain of functions,
@ -401,17 +402,16 @@ private:
* and fills chain of possibly-monotonic functions. * and fills chain of possibly-monotonic functions.
*/ */
bool isKeyPossiblyWrappedByMonotonicFunctions( bool isKeyPossiblyWrappedByMonotonicFunctions(
const Tree & node, const RPNBuilderTreeNode & node,
ContextPtr context,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_res_column_type, DataTypePtr & out_key_res_column_type,
MonotonicFunctionsChain & out_functions_chain); MonotonicFunctionsChain & out_functions_chain);
bool isKeyPossiblyWrappedByMonotonicFunctionsImpl( bool isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const Tree & node, const RPNBuilderTreeNode & node,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_column_type, DataTypePtr & out_key_column_type,
std::vector<FunctionTree> & out_functions_chain); std::vector<RPNBuilderFunctionTreeNode> & out_functions_chain);
bool transformConstantWithValidFunctions( bool transformConstantWithValidFunctions(
const String & expr_name, const String & expr_name,
@ -422,21 +422,24 @@ private:
std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const; std::function<bool(IFunctionBase &, const IDataType &)> always_monotonic) const;
bool canConstantBeWrappedByMonotonicFunctions( bool canConstantBeWrappedByMonotonicFunctions(
const Tree & node, const RPNBuilderTreeNode & node,
size_t & out_key_column_num, size_t & out_key_column_num,
DataTypePtr & out_key_column_type, DataTypePtr & out_key_column_type,
Field & out_value, Field & out_value,
DataTypePtr & out_type); DataTypePtr & out_type);
bool canConstantBeWrappedByFunctions( bool canConstantBeWrappedByFunctions(
const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); const RPNBuilderTreeNode & node,
size_t & out_key_column_num,
DataTypePtr & out_key_column_type,
Field & out_value,
DataTypePtr & out_type);
/// If it's possible to make an RPNElement /// If it's possible to make an RPNElement
/// that will filter values (possibly tuples) by the content of 'prepared_set', /// that will filter values (possibly tuples) by the content of 'prepared_set',
/// do it and return true. /// do it and return true.
bool tryPrepareSetIndex( bool tryPrepareSetIndex(
const FunctionTree & func, const RPNBuilderFunctionTreeNode & func,
ContextPtr context,
RPNElement & out, RPNElement & out,
size_t & out_key_column_num); size_t & out_key_column_num);
@ -472,11 +475,12 @@ private:
/// All intermediate columns are used to calculate key_expr. /// All intermediate columns are used to calculate key_expr.
const NameSet key_subexpr_names; const NameSet key_subexpr_names;
NameSet array_joined_columns; /// Array joined column names
PreparedSetsPtr prepared_sets; NameSet array_joined_column_names;
// If true, always allow key_expr to be wrapped by function // If true, always allow key_expr to be wrapped by function
bool single_point; bool single_point;
// If true, do not use always_monotonic information to transform constants // If true, do not use always_monotonic information to transform constants
bool strict; bool strict;
}; };

View File

@ -6,11 +6,13 @@
#include <Columns/ColumnConst.h> #include <Columns/ColumnConst.h>
#include <Columns/ColumnTuple.h> #include <Columns/ColumnTuple.h>
#include <Storages/MergeTree/RPNBuilder.h> #include <Storages/MergeTree/RPNBuilder.h>
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
#include <Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h> #include <Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.h>
#include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h> #include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Interpreters/misc.h> #include <Interpreters/misc.h>
#include <Interpreters/BloomFilterHash.h> #include <Interpreters/BloomFilterHash.h>
#include <Interpreters/castColumn.h> #include <Interpreters/castColumn.h>
@ -28,19 +30,7 @@ namespace ErrorCodes
namespace namespace
{ {
PreparedSetKey getPreparedSetKey(const ASTPtr & node, const DataTypePtr & data_type) ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set)
{
/// If the data type is tuple, let's try unbox once
if (node->as<ASTSubquery>() || node->as<ASTIdentifier>())
return PreparedSetKey::forSubquery(*node);
if (const auto * date_type_tuple = typeid_cast<const DataTypeTuple *>(&*data_type))
return PreparedSetKey::forLiteral(*node, date_type_tuple->getElements());
return PreparedSetKey::forLiteral(*node, DataTypes(1, data_type));
}
ColumnWithTypeAndName getPreparedSetInfo(const SetPtr & prepared_set)
{ {
if (prepared_set->getDataTypes().size() == 1) if (prepared_set->getDataTypes().size() == 1)
return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"};
@ -110,8 +100,22 @@ MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter(
const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_) const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_)
: WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_) : WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_)
{ {
auto atom_from_ast = [this](auto & node, auto, auto & constants, auto & out) { return traverseAtomAST(node, constants, out); }; ASTPtr filter_node = buildFilterNode(query_info.query);
rpn = std::move(RPNBuilder<RPNElement>(info_, getContext(), atom_from_ast).extractRPN());
if (!filter_node)
{
rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return;
}
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_);
RPNBuilder<RPNElement> builder(
filter_node,
context_,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
} }
bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const
@ -235,12 +239,13 @@ bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndex
return rpn_stack[0].can_be_true; return rpn_stack[0].can_be_true;
} }
bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out) bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
{ {
{ {
Field const_value; Field const_value;
DataTypePtr const_type; DataTypePtr const_type;
if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type))
if (node.tryGetConstant(const_value, const_type))
{ {
if (const_value.getType() == Field::Types::UInt64) if (const_value.getType() == Field::Types::UInt64)
{ {
@ -262,56 +267,57 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl
} }
} }
return traverseFunction(node, block_with_constants, out, nullptr); return traverseFunction(node, out, nullptr /*parent*/);
} }
bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent) bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent)
{ {
bool maybe_useful = false; bool maybe_useful = false;
if (const auto * function = node->as<ASTFunction>()) if (node.isFunction())
{ {
if (!function->arguments) const auto function = node.toFunctionNode();
auto arguments_size = function.getArgumentsSize();
auto function_name = function.getFunctionName();
if (arguments_size != 2)
return false; return false;
const ASTs & arguments = function->arguments->children; auto lhs_argument = function.getArgumentAt(0);
for (const auto & arg : arguments) auto rhs_argument = function.getArgumentAt(1);
auto lhs_argument_column_name = lhs_argument.getColumnName();
if (functionIsInOrGlobalInOperator(function_name))
{ {
if (traverseFunction(arg, block_with_constants, out, node)) ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet();
maybe_useful = true;
}
if (arguments.size() != 2) if (prepared_set && prepared_set->hasExplicitSetElements())
return false;
if (functionIsInOrGlobalInOperator(function->name))
{
auto prepared_set = getPreparedSet(arguments[1]);
if (prepared_set)
{ {
if (traverseASTIn(function->name, arguments[0], prepared_set, out)) const auto prepared_info = getPreparedSetInfo(prepared_set);
if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out))
maybe_useful = true; maybe_useful = true;
} }
} }
else if (function->name == "equals" || else if (function_name == "equals" ||
function->name == "notEquals" || function_name == "notEquals" ||
function->name == "has" || function_name == "has" ||
function->name == "mapContains" || function_name == "mapContains" ||
function->name == "indexOf" || function_name == "indexOf" ||
function->name == "hasAny" || function_name == "hasAny" ||
function->name == "hasAll") function_name == "hasAll")
{ {
Field const_value; Field const_value;
DataTypePtr const_type; DataTypePtr const_type;
if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type))
if (rhs_argument.tryGetConstant(const_value, const_type))
{ {
if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out, parent)) if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent))
maybe_useful = true; maybe_useful = true;
} }
else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type)) else if (lhs_argument.tryGetConstant(const_value, const_type))
{ {
if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out, parent)) if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent))
maybe_useful = true; maybe_useful = true;
} }
} }
@ -320,28 +326,20 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
return maybe_useful; return maybe_useful;
} }
bool MergeTreeIndexConditionBloomFilter::traverseASTIn( bool MergeTreeIndexConditionBloomFilter::traverseTreeIn(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const SetPtr & prepared_set, const ConstSetPtr & prepared_set,
RPNElement & out)
{
const auto prepared_info = getPreparedSetInfo(prepared_set);
return traverseASTIn(function_name, key_ast, prepared_set, prepared_info.type, prepared_info.column, out);
}
bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
const DataTypePtr & type, const DataTypePtr & type,
const ColumnPtr & column, const ColumnPtr & column,
RPNElement & out) RPNElement & out)
{ {
if (header.has(key_ast->getColumnName())) auto key_node_column_name = key_node.getColumnName();
if (header.has(key_node_column_name))
{ {
size_t row_size = column->size(); size_t row_size = column->size();
size_t position = header.getPositionByName(key_ast->getColumnName()); size_t position = header.getPositionByName(key_node_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type; const DataTypePtr & index_type = header.getByPosition(position).type;
const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size)));
@ -355,30 +353,33 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
return true; return true;
} }
if (const auto * function = key_ast->as<ASTFunction>()) if (key_node.isFunction())
{ {
auto key_node_function = key_node.toFunctionNode();
auto key_node_function_name = key_node_function.getFunctionName();
size_t key_node_function_arguments_size = key_node_function.getArgumentsSize();
WhichDataType which(type); WhichDataType which(type);
if (which.isTuple() && function->name == "tuple") if (which.isTuple() && function_name == "tuple")
{ {
const auto & tuple_column = typeid_cast<const ColumnTuple *>(column.get()); const auto & tuple_column = typeid_cast<const ColumnTuple *>(column.get());
const auto & tuple_data_type = typeid_cast<const DataTypeTuple *>(type.get()); const auto & tuple_data_type = typeid_cast<const DataTypeTuple *>(type.get());
const ASTs & arguments = typeid_cast<const ASTExpressionList &>(*function->arguments).children;
if (tuple_data_type->getElements().size() != arguments.size() || tuple_column->getColumns().size() != arguments.size()) if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size)
throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
bool match_with_subtype = false; bool match_with_subtype = false;
const auto & sub_columns = tuple_column->getColumns(); const auto & sub_columns = tuple_column->getColumns();
const auto & sub_data_types = tuple_data_type->getElements(); const auto & sub_data_types = tuple_data_type->getElements();
for (size_t index = 0; index < arguments.size(); ++index) for (size_t index = 0; index < key_node_function_arguments_size; ++index)
match_with_subtype |= traverseASTIn(function_name, arguments[index], nullptr, sub_data_types[index], sub_columns[index], out); match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out);
return match_with_subtype; return match_with_subtype;
} }
if (function->name == "arrayElement") if (key_node_function_name == "arrayElement")
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map
@ -388,9 +389,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
* that way we skip necessary granules where map key does not exists. * that way we skip necessary granules where map key does not exists.
*/ */
if (!prepared_set)
return false;
auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst();
ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" };
ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check};
@ -400,11 +398,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
if (set_contain_default_value) if (set_contain_default_value)
return false; return false;
const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as<ASTIdentifier>(); const auto & col_name = key_node_function.getArgumentAt(0).getColumnName();
if (!column_ast_identifier)
return false;
const auto & col_name = column_ast_identifier->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
auto map_values_index_column_name = fmt::format("mapValues({})", col_name); auto map_values_index_column_name = fmt::format("mapValues({})", col_name);
@ -412,16 +406,17 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
{ {
/// For mapKeys we serialize key argument with bloom filter /// For mapKeys we serialize key argument with bloom filter
auto & argument = function->arguments.get()->children[1]; auto first_argument = key_node_function.getArgumentAt(1);
if (const auto * literal = argument->as<ASTLiteral>()) Field constant_value;
DataTypePtr constant_type;
if (first_argument.tryGetConstant(constant_value, constant_type))
{ {
size_t position = header.getPositionByName(map_keys_index_column_name); size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type; const DataTypePtr & index_type = header.getByPosition(position).type;
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key))); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value)));
} }
else else
{ {
@ -459,74 +454,97 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
} }
static bool indexOfCanUseBloomFilter(const ASTPtr & parent) static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent)
{ {
if (!parent) if (!parent)
return true; return true;
if (!parent->isFunction())
return false;
auto function = parent->toFunctionNode();
auto function_name = function.getFunctionName();
/// `parent` is a function where `indexOf` is located. /// `parent` is a function where `indexOf` is located.
/// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`.
if (const auto * function = parent->as<ASTFunction>()) if (function_name == "and")
{ {
if (function->name == "and") return true;
}
else if (function_name == "equals" /// notEquals is not applicable
|| function_name == "greater" || function_name == "greaterOrEquals"
|| function_name == "less" || function_name == "lessOrEquals")
{
size_t function_arguments_size = function.getArgumentsSize();
if (function_arguments_size != 2)
return false;
/// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible.
/// We should return true when the corresponding expression implies that the array contains the element.
/// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element
/// (at least at 11th position but it does not matter).
bool reversed = false;
Field constant_value;
DataTypePtr constant_type;
if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type))
{ {
reversed = true;
}
else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type))
{
}
else
{
return false;
}
Field zero(0);
bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero);
if (function_name == "equals" && !constant_equal_zero)
{
/// indexOf(...) = c, c != 0
return true; return true;
} }
else if (function->name == "equals" /// notEquals is not applicable else if (function_name == "notEquals" && constant_equal_zero)
|| function->name == "greater" || function->name == "greaterOrEquals"
|| function->name == "less" || function->name == "lessOrEquals")
{ {
if (function->arguments->children.size() != 2) /// indexOf(...) != c, c = 0
return false; return true;
/// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible.
/// We should return true when the corresponding expression implies that the array contains the element.
/// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element
/// (at least at 11th position but it does not matter).
bool reversed = false;
const ASTLiteral * constant = nullptr;
if (const ASTLiteral * left = function->arguments->children[0]->as<ASTLiteral>())
{
constant = left;
reversed = true;
}
else if (const ASTLiteral * right = function->arguments->children[1]->as<ASTLiteral>())
{
constant = right;
}
else
return false;
Field zero(0);
return (function->name == "equals" /// indexOf(...) = c, c != 0
&& !applyVisitor(FieldVisitorAccurateEquals(), constant->value, zero))
|| (function->name == "notEquals" /// indexOf(...) != c, c = 0
&& applyVisitor(FieldVisitorAccurateEquals(), constant->value, zero))
|| (function->name == (reversed ? "less" : "greater") /// indexOf(...) > c, c >= 0
&& !applyVisitor(FieldVisitorAccurateLess(), constant->value, zero))
|| (function->name == (reversed ? "lessOrEquals" : "greaterOrEquals") /// indexOf(...) >= c, c > 0
&& applyVisitor(FieldVisitorAccurateLess(), zero, constant->value));
} }
else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero))
{
/// indexOf(...) > c, c >= 0
return true;
}
else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value))
{
/// indexOf(...) >= c, c > 0
return true;
}
return false;
} }
return false; return false;
} }
bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const DataTypePtr & value_type, const DataTypePtr & value_type,
const Field & value_field, const Field & value_field,
RPNElement & out, RPNElement & out,
const ASTPtr & parent) const RPNBuilderTreeNode * parent)
{ {
if (header.has(key_ast->getColumnName())) auto key_column_name = key_node.getColumnName();
if (header.has(key_column_name))
{ {
size_t position = header.getPositionByName(key_ast->getColumnName()); size_t position = header.getPositionByName(key_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type; const DataTypePtr & index_type = header.getByPosition(position).type;
const auto * array_type = typeid_cast<const DataTypeArray *>(index_type.get()); const auto * array_type = typeid_cast<const DataTypeArray *>(index_type.get());
@ -602,13 +620,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
if (function_name == "mapContains" || function_name == "has") if (function_name == "mapContains" || function_name == "has")
{ {
const auto * key_ast_identifier = key_ast.get()->as<const ASTIdentifier>(); auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name);
if (!key_ast_identifier)
return false;
const auto & col_name = key_ast_identifier->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
if (!header.has(map_keys_index_column_name)) if (!header.has(map_keys_index_column_name))
return false; return false;
@ -629,29 +641,32 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
return true; return true;
} }
if (const auto * function = key_ast->as<ASTFunction>()) if (key_node.isFunction())
{ {
WhichDataType which(value_type); WhichDataType which(value_type);
if (which.isTuple() && function->name == "tuple") auto key_node_function = key_node.toFunctionNode();
auto key_node_function_name = key_node_function.getFunctionName();
size_t key_node_function_arguments_size = key_node_function.getArgumentsSize();
if (which.isTuple() && key_node_function_name == "tuple")
{ {
const Tuple & tuple = value_field.get<const Tuple &>(); const Tuple & tuple = value_field.get<const Tuple &>();
const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get()); const auto * value_tuple_data_type = typeid_cast<const DataTypeTuple *>(value_type.get());
const ASTs & arguments = typeid_cast<const ASTExpressionList &>(*function->arguments).children;
if (tuple.size() != arguments.size()) if (tuple.size() != key_node_function_arguments_size)
throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
bool match_with_subtype = false; bool match_with_subtype = false;
const DataTypes & subtypes = value_tuple_data_type->getElements(); const DataTypes & subtypes = value_tuple_data_type->getElements();
for (size_t index = 0; index < tuple.size(); ++index) for (size_t index = 0; index < tuple.size(); ++index)
match_with_subtype |= traverseASTEquals(function_name, arguments[index], subtypes[index], tuple[index], out, key_ast); match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node);
return match_with_subtype; return match_with_subtype;
} }
if (function->name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals"))
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
@ -663,27 +678,22 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
if (value_field == value_type->getDefault()) if (value_field == value_type->getDefault())
return false; return false;
const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as<ASTIdentifier>(); auto first_argument = key_node_function.getArgumentAt(0);
if (!column_ast_identifier) const auto col_name = first_argument.getColumnName();
return false;
const auto & col_name = column_ast_identifier->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
auto map_values_index_column_name = fmt::format("mapValues({})", col_name); auto map_values_index_column_name = fmt::format("mapValues({})", col_name);
size_t position = 0; size_t position = 0;
Field const_value = value_field; Field const_value = value_field;
DataTypePtr const_type;
if (header.has(map_keys_index_column_name)) if (header.has(map_keys_index_column_name))
{ {
position = header.getPositionByName(map_keys_index_column_name); position = header.getPositionByName(map_keys_index_column_name);
auto second_argument = key_node_function.getArgumentAt(1);
auto & argument = function->arguments.get()->children[1]; if (!second_argument.tryGetConstant(const_value, const_type))
if (const auto * literal = argument->as<ASTLiteral>())
const_value = literal->value;
else
return false; return false;
} }
else if (header.has(map_values_index_column_name)) else if (header.has(map_values_index_column_name))
@ -708,23 +718,4 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
return false; return false;
} }
SetPtr MergeTreeIndexConditionBloomFilter::getPreparedSet(const ASTPtr & node)
{
if (header.has(node->getColumnName()))
{
const auto & column_and_type = header.getByName(node->getColumnName());
auto set_key = getPreparedSetKey(node, column_and_type.type);
if (auto prepared_set = query_info.prepared_sets->get(set_key))
return prepared_set;
}
else
{
for (const auto & set : query_info.prepared_sets->getByTreeHash(node->getTreeHash()))
if (set->hasExplicitSetElements())
return set;
}
return DB::SetPtr();
}
} }

View File

@ -62,35 +62,27 @@ private:
const size_t hash_functions; const size_t hash_functions;
std::vector<RPNElement> rpn; std::vector<RPNElement> rpn;
SetPtr getPreparedSet(const ASTPtr & node);
bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const;
bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out);
bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent); bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent);
bool traverseASTIn( bool traverseTreeIn(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const SetPtr & prepared_set, const ConstSetPtr & prepared_set,
RPNElement & out);
bool traverseASTIn(
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
const DataTypePtr & type, const DataTypePtr & type,
const ColumnPtr & column, const ColumnPtr & column,
RPNElement & out); RPNElement & out);
bool traverseASTEquals( bool traverseTreeEquals(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const DataTypePtr & value_type, const DataTypePtr & value_type,
const Field & value_field, const Field & value_field,
RPNElement & out, RPNElement & out,
const ASTPtr & parent); const RPNBuilderTreeNode * parent);
}; };
} }

View File

@ -11,9 +11,11 @@
#include <Interpreters/misc.h> #include <Interpreters/misc.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/RPNBuilder.h> #include <Storages/MergeTree/RPNBuilder.h>
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
@ -148,13 +150,22 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
, token_extractor(token_extactor_) , token_extractor(token_extactor_)
, prepared_sets(query_info.prepared_sets) , prepared_sets(query_info.prepared_sets)
{ {
rpn = std::move( ASTPtr filter_node = buildFilterNode(query_info.query);
RPNBuilder<RPNElement>(
query_info, context, if (!filter_node)
[this] (const ASTPtr & node, ContextPtr /* context */, Block & block_with_constants, RPNElement & out) -> bool {
{ rpn.push_back(RPNElement::FUNCTION_UNKNOWN);
return this->traverseAtomAST(node, block_with_constants, out); return;
}).extractRPN()); }
auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context);
RPNBuilder<RPNElement> builder(
filter_node,
context,
std::move(block_with_constants),
query_info.prepared_sets,
[&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); });
rpn = std::move(builder).extractRPN();
} }
bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
@ -306,13 +317,13 @@ bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, siz
return true; return true;
} }
bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out) bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out)
{ {
{ {
Field const_value; Field const_value;
DataTypePtr const_type; DataTypePtr const_type;
if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) if (node.tryGetConstant(const_value, const_type))
{ {
/// Check constant like in KeyCondition /// Check constant like in KeyCondition
if (const_value.getType() == Field::Types::UInt64 if (const_value.getType() == Field::Types::UInt64
@ -329,53 +340,56 @@ bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & bl
} }
} }
if (const auto * function = node->as<ASTFunction>()) if (node.isFunction())
{ {
if (!function->arguments) auto function_node = node.toFunctionNode();
auto function_name = function_node.getFunctionName();
size_t arguments_size = function_node.getArgumentsSize();
if (arguments_size != 2)
return false; return false;
const ASTs & arguments = function->arguments->children; auto left_argument = function_node.getArgumentAt(0);
auto right_argument = function_node.getArgumentAt(1);
if (arguments.size() != 2) if (functionIsInOrGlobalInOperator(function_name))
return false;
if (functionIsInOrGlobalInOperator(function->name))
{ {
if (tryPrepareSetBloomFilter(arguments, out)) if (tryPrepareSetBloomFilter(left_argument, right_argument, out))
{ {
if (function->name == "notIn") if (function_name == "notIn")
{ {
out.function = RPNElement::FUNCTION_NOT_IN; out.function = RPNElement::FUNCTION_NOT_IN;
return true; return true;
} }
else if (function->name == "in") else if (function_name == "in")
{ {
out.function = RPNElement::FUNCTION_IN; out.function = RPNElement::FUNCTION_IN;
return true; return true;
} }
} }
} }
else if (function->name == "equals" || else if (function_name == "equals" ||
function->name == "notEquals" || function_name == "notEquals" ||
function->name == "has" || function_name == "has" ||
function->name == "mapContains" || function_name == "mapContains" ||
function->name == "like" || function_name == "like" ||
function->name == "notLike" || function_name == "notLike" ||
function->name == "hasToken" || function_name == "hasToken" ||
function->name == "startsWith" || function_name == "startsWith" ||
function->name == "endsWith" || function_name == "endsWith" ||
function->name == "multiSearchAny") function_name == "multiSearchAny")
{ {
Field const_value; Field const_value;
DataTypePtr const_type; DataTypePtr const_type;
if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type))
if (right_argument.tryGetConstant(const_value, const_type))
{ {
if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out)) if (traverseTreeEquals(function_name, left_argument, const_type, const_value, out))
return true; return true;
} }
else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type) && (function->name == "equals" || function->name == "notEquals")) else if (left_argument.tryGetConstant(const_value, const_type) && (function_name == "equals" || function_name == "notEquals"))
{ {
if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out)) if (traverseTreeEquals(function_name, right_argument, const_type, const_value, out))
return true; return true;
} }
} }
@ -384,9 +398,9 @@ bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & bl
return false; return false;
} }
bool MergeTreeConditionFullText::traverseASTEquals( bool MergeTreeConditionFullText::traverseTreeEquals(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const DataTypePtr & value_type, const DataTypePtr & value_type,
const Field & value_field, const Field & value_field,
RPNElement & out) RPNElement & out)
@ -397,13 +411,17 @@ bool MergeTreeConditionFullText::traverseASTEquals(
Field const_value = value_field; Field const_value = value_field;
auto column_name = key_node.getColumnName();
size_t key_column_num = 0; size_t key_column_num = 0;
bool key_exists = getKey(key_ast->getColumnName(), key_column_num); bool key_exists = getKey(column_name, key_column_num);
bool map_key_exists = getKey(fmt::format("mapKeys({})", key_ast->getColumnName()), key_column_num); bool map_key_exists = getKey(fmt::format("mapKeys({})", column_name), key_column_num);
if (const auto * function = key_ast->as<ASTFunction>()) if (key_node.isFunction())
{ {
if (function->name == "arrayElement") auto key_function_node = key_node.toFunctionNode();
auto key_function_node_function_name = key_function_node.getFunctionName();
if (key_function_node_function_name == "arrayElement")
{ {
/** Try to parse arrayElement for mapKeys index. /** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
@ -415,11 +433,8 @@ bool MergeTreeConditionFullText::traverseASTEquals(
if (value_field == value_type->getDefault()) if (value_field == value_type->getDefault())
return false; return false;
const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as<ASTIdentifier>(); auto first_argument = key_function_node.getArgumentAt(0);
if (!column_ast_identifier) const auto & map_column_name = first_argument.getColumnName();
return false;
const auto & map_column_name = column_ast_identifier->name();
size_t map_keys_key_column_num = 0; size_t map_keys_key_column_num = 0;
auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name); auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name);
@ -431,12 +446,11 @@ bool MergeTreeConditionFullText::traverseASTEquals(
if (map_keys_exists) if (map_keys_exists)
{ {
auto & argument = function->arguments.get()->children[1]; auto second_argument = key_function_node.getArgumentAt(1);
DataTypePtr const_type;
if (const auto * literal = argument->as<ASTLiteral>()) if (second_argument.tryGetConstant(const_value, const_type))
{ {
auto element_key = literal->value;
const_value = element_key;
key_column_num = map_keys_key_column_num; key_column_num = map_keys_key_column_num;
key_exists = true; key_exists = true;
} }
@ -567,33 +581,37 @@ bool MergeTreeConditionFullText::traverseASTEquals(
} }
bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
const ASTs & args, const RPNBuilderTreeNode & left_argument,
const RPNBuilderTreeNode & right_argument,
RPNElement & out) RPNElement & out)
{ {
const ASTPtr & left_arg = args[0];
const ASTPtr & right_arg = args[1];
std::vector<KeyTuplePositionMapping> key_tuple_mapping; std::vector<KeyTuplePositionMapping> key_tuple_mapping;
DataTypes data_types; DataTypes data_types;
const auto * left_arg_tuple = typeid_cast<const ASTFunction *>(left_arg.get()); if (left_argument.isFunction())
if (left_arg_tuple && left_arg_tuple->name == "tuple")
{ {
const auto & tuple_elements = left_arg_tuple->arguments->children; auto left_argument_function_node = left_argument.toFunctionNode();
for (size_t i = 0; i < tuple_elements.size(); ++i) auto left_argument_function_node_name = left_argument_function_node.getFunctionName();
if (left_argument_function_node_name == "tuple")
{ {
size_t key = 0; size_t left_argument_function_node_arguments_size = left_argument_function_node.getArgumentsSize();
if (getKey(tuple_elements[i]->getColumnName(), key))
for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i)
{ {
key_tuple_mapping.emplace_back(i, key); size_t key = 0;
data_types.push_back(index_data_types[key]); if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key))
{
key_tuple_mapping.emplace_back(i, key);
data_types.push_back(index_data_types[key]);
}
} }
} }
} }
else else
{ {
size_t key = 0; size_t key = 0;
if (getKey(left_arg->getColumnName(), key)) if (getKey(left_argument.getColumnName(), key))
{ {
key_tuple_mapping.emplace_back(0, key); key_tuple_mapping.emplace_back(0, key);
data_types.push_back(index_data_types[key]); data_types.push_back(index_data_types[key]);
@ -603,19 +621,10 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
if (key_tuple_mapping.empty()) if (key_tuple_mapping.empty())
return false; return false;
PreparedSetKey set_key; auto prepared_set = right_argument.tryGetPreparedSet(data_types);
if (typeid_cast<const ASTSubquery *>(right_arg.get()) || typeid_cast<const ASTIdentifier *>(right_arg.get()))
set_key = PreparedSetKey::forSubquery(*right_arg);
else
set_key = PreparedSetKey::forLiteral(*right_arg, data_types);
auto prepared_set = prepared_sets->get(set_key);
if (!prepared_set) if (!prepared_set)
return false; return false;
if (!prepared_set->hasExplicitSetElements())
return false;
for (const auto & data_type : prepared_set->getDataTypes()) for (const auto & data_type : prepared_set->getDataTypes())
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString) if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
return false; return false;

View File

@ -122,17 +122,17 @@ private:
using RPN = std::vector<RPNElement>; using RPN = std::vector<RPNElement>;
bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out);
bool traverseASTEquals( bool traverseTreeEquals(
const String & function_name, const String & function_name,
const ASTPtr & key_ast, const RPNBuilderTreeNode & key_node,
const DataTypePtr & value_type, const DataTypePtr & value_type,
const Field & value_field, const Field & value_field,
RPNElement & out); RPNElement & out);
bool getKey(const std::string & key_column_name, size_t & key_column_num); bool getKey(const std::string & key_column_name, size_t & key_column_num);
bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); bool tryPrepareSetBloomFilter(const RPNBuilderTreeNode & left_argument, const RPNBuilderTreeNode & right_argument, RPNElement & out);
static bool createFunctionEqualsCondition( static bool createFunctionEqualsCondition(
RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor); RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor);

View File

@ -0,0 +1,47 @@
#include <Storages/MergeTree/MergeTreeIndexUtils.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
namespace DB
{
ASTPtr buildFilterNode(const ASTPtr & select_query, ASTs additional_filters)
{
auto & select_query_typed = select_query->as<ASTSelectQuery &>();
ASTs filters;
if (select_query_typed.where())
filters.push_back(select_query_typed.where());
if (select_query_typed.prewhere())
filters.push_back(select_query_typed.prewhere());
filters.insert(filters.end(), additional_filters.begin(), additional_filters.end());
if (filters.empty())
return nullptr;
ASTPtr filter_node;
if (filters.size() == 1)
{
filter_node = filters.front();
}
else
{
auto function = std::make_shared<ASTFunction>();
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = std::move(filters);
filter_node = std::move(function);
}
return filter_node;
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
/** Build AST filter node for index analysis from WHERE and PREWHERE sections of select query and additional filters.
* If select query does not have WHERE or PREWHERE and additional filters are empty null is returned.
*/
ASTPtr buildFilterNode(const ASTPtr & select_query, ASTs additional_filters = {});
}

View File

@ -0,0 +1,380 @@
#include <Storages/MergeTree/RPNBuilder.h>
#include <Common/FieldVisitorToString.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSubquery.h>
#include <DataTypes/FieldToDataType.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnSet.h>
#include <Functions/IFunction.h>
#include <Storages/KeyDescription.h>
namespace DB
{
namespace
{
void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false)
{
switch (node.type)
{
case (ActionsDAG::ActionType::INPUT):
writeString(node.result_name, out);
break;
case (ActionsDAG::ActionType::COLUMN):
{
/// If it was created from ASTLiteral, then result_name can be an alias.
/// We need to convert value back to string here.
if (const auto * column_const = typeid_cast<const ColumnConst *>(node.column.get()))
writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out);
/// It may be possible that column is ColumnSet
else
writeString(node.result_name, out);
break;
}
case (ActionsDAG::ActionType::ALIAS):
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
break;
case (ActionsDAG::ActionType::ARRAY_JOIN):
writeCString("arrayJoin(", out);
appendColumnNameWithoutAlias(*node.children.front(), out, legacy);
writeChar(')', out);
break;
case (ActionsDAG::ActionType::FUNCTION):
{
auto name = node.function_base->getName();
if (legacy && name == "modulo")
writeCString("moduleLegacy", out);
else
writeString(name, out);
writeChar('(', out);
bool first = true;
for (const auto * arg : node.children)
{
if (!first)
writeCString(", ", out);
first = false;
appendColumnNameWithoutAlias(*arg, out, legacy);
}
writeChar(')', out);
}
}
}
String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false)
{
WriteBufferFromOwnString out;
appendColumnNameWithoutAlias(node, out, legacy);
return std::move(out.str());
}
}
RPNBuilderTreeContext::RPNBuilderTreeContext(ContextPtr query_context_)
: query_context(std::move(query_context_))
{}
RPNBuilderTreeContext::RPNBuilderTreeContext(ContextPtr query_context_, Block block_with_constants_, PreparedSetsPtr prepared_sets_)
: query_context(std::move(query_context_))
, block_with_constants(std::move(block_with_constants_))
, prepared_sets(std::move(prepared_sets_))
{}
RPNBuilderTreeNode::RPNBuilderTreeNode(const ActionsDAG::Node * dag_node_, RPNBuilderTreeContext & tree_context_)
: dag_node(dag_node_)
, tree_context(tree_context_)
{
assert(dag_node);
}
RPNBuilderTreeNode::RPNBuilderTreeNode(const IAST * ast_node_, RPNBuilderTreeContext & tree_context_)
: ast_node(ast_node_)
, tree_context(tree_context_)
{
assert(ast_node);
}
std::string RPNBuilderTreeNode::getColumnName() const
{
if (ast_node)
return ast_node->getColumnNameWithoutAlias();
else
return getColumnNameWithoutAlias(*dag_node);
}
std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const
{
if (ast_node)
{
auto adjusted_ast = ast_node->clone();
KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast);
return adjusted_ast->getColumnNameWithoutAlias();
}
else
{
return getColumnNameWithoutAlias(*dag_node, true);
}
}
bool RPNBuilderTreeNode::isFunction() const
{
if (ast_node)
return typeid_cast<const ASTFunction *>(ast_node);
else
return dag_node->type == ActionsDAG::ActionType::FUNCTION;
}
bool RPNBuilderTreeNode::isConstant() const
{
if (ast_node)
return typeid_cast<const ASTLiteral *>(ast_node);
else
return dag_node->column && isColumnConst(*dag_node->column);
}
ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const
{
if (!isConstant())
throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant");
ColumnWithTypeAndName res;
if (ast_node)
{
const auto * literal = assert_cast<const ASTLiteral *>(ast_node);
res.type = applyVisitor(FieldToDataType(), literal->value);
res.column = res.type->createColumnConst(0, literal->value);
}
else
{
res.type = dag_node->result_type;
res.column = dag_node->column;
}
return res;
}
bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & output_type) const
{
if (!isConstant())
return false;
if (ast_node)
{
// Constant expr should use alias names if any
String column_name = ast_node->getColumnName();
const auto & block_with_constants = tree_context.getBlockWithConstants();
if (const auto * lit = ast_node->as<ASTLiteral>())
{
/// By default block_with_constants has only one column named "_dummy".
/// If block contains only constants it's may not be preprocessed by
// ExpressionAnalyzer, so try to look up in the default column.
if (!block_with_constants.has(column_name))
column_name = "_dummy";
/// Simple literal
output_value = lit->value;
output_type = block_with_constants.getByName(column_name).type;
/// If constant is not Null, we can assume it's type is not Nullable as well.
if (!output_value.isNull())
output_type = removeNullable(output_type);
return true;
}
else if (block_with_constants.has(column_name) &&
isColumnConst(*block_with_constants.getByName(column_name).column))
{
/// An expression which is dependent on constants only
const auto & expr_info = block_with_constants.getByName(column_name);
output_value = (*expr_info.column)[0];
output_type = expr_info.type;
if (!output_value.isNull())
output_type = removeNullable(output_type);
return true;
}
}
else
{
if (dag_node->column && isColumnConst(*dag_node->column))
{
output_value = (*dag_node->column)[0];
output_type = dag_node->result_type;
if (!output_value.isNull())
output_type = removeNullable(output_type);
return true;
}
}
return false;
}
namespace
{
ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node)
{
if (!dag_node->column)
return {};
const IColumn * col = dag_node->column.get();
if (const auto * col_const = typeid_cast<const ColumnConst *>(col))
col = &col_const->getDataColumn();
if (const auto * col_set = typeid_cast<const ColumnSet *>(col))
{
auto set = col_set->getData();
if (set->isCreated())
return set;
}
return {};
}
}
ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const
{
const auto & prepared_sets = getTreeContext().getPreparedSets();
if (ast_node && prepared_sets)
{
auto prepared_sets_with_same_hash = prepared_sets->getByTreeHash(ast_node->getTreeHash());
for (auto & set : prepared_sets_with_same_hash)
if (set->isCreated())
return set;
}
else if (dag_node)
{
return tryGetSetFromDAGNode(dag_node);
}
return {};
}
ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const
{
const auto & prepared_sets = getTreeContext().getPreparedSets();
if (prepared_sets && ast_node)
{
if (ast_node->as<ASTSubquery>() || ast_node->as<ASTTableIdentifier>())
return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node));
return prepared_sets->get(PreparedSetKey::forLiteral(*ast_node, data_types));
}
else if (dag_node)
{
return tryGetSetFromDAGNode(dag_node);
}
return nullptr;
}
ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(
const std::vector<MergeTreeSetIndex::KeyTuplePositionMapping> & indexes_mapping,
const DataTypes & data_types) const
{
const auto & prepared_sets = getTreeContext().getPreparedSets();
if (prepared_sets && ast_node)
{
if (ast_node->as<ASTSubquery>() || ast_node->as<ASTTableIdentifier>())
return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node));
/// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information
/// about types in left argument of the IN operator. Instead, we manually iterate through all the sets
/// and find the one for the right arg based on the AST structure (getTreeHash), after that we check
/// that the types it was prepared with are compatible with the types of the primary key.
auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set)
{
assert(indexes_mapping.size() == data_types.size());
for (size_t i = 0; i < indexes_mapping.size(); ++i)
{
if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i]))
return false;
}
return true;
};
auto tree_hash = ast_node->getTreeHash();
for (const auto & set : prepared_sets->getByTreeHash(tree_hash))
{
if (types_match(set))
return set;
}
}
else if (dag_node->column)
{
return tryGetSetFromDAGNode(dag_node);
}
return nullptr;
}
RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const
{
if (!isFunction())
throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant");
if (this->ast_node)
return RPNBuilderFunctionTreeNode(this->ast_node, tree_context);
else
return RPNBuilderFunctionTreeNode(this->dag_node, tree_context);
}
std::string RPNBuilderFunctionTreeNode::getFunctionName() const
{
if (ast_node)
return assert_cast<const ASTFunction *>(ast_node)->name;
else
return dag_node->function_base->getName();
}
size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const
{
if (ast_node)
{
const auto * ast_function = assert_cast<const ASTFunction *>(ast_node);
return ast_function->arguments ? ast_function->arguments->children.size() : 0;
}
else
{
return dag_node->children.size();
}
}
RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const
{
if (ast_node)
{
const auto * ast_function = assert_cast<const ASTFunction *>(ast_node);
return RPNBuilderTreeNode(ast_function->arguments->children[index].get(), tree_context);
}
else
{
return RPNBuilderTreeNode(dag_node->children[index], tree_context);
}
}
}

View File

@ -1,111 +1,262 @@
#pragma once #pragma once
#include <Core/Block.h> #include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/SelectQueryInfo.h>
#include <Common/typeid_cast.h>
#include <Interpreters/Context.h>
#include <Interpreters/Set.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/ActionsDAG.h>
namespace DB namespace DB
{ {
/// Builds reverse polish notation /** Context of RPNBuilderTree.
template <typename RPNElement> *
class RPNBuilder : WithContext * For AST tree context, precalculated block with constansts and prepared sets are required for index analysis.
* 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: public:
using RPN = std::vector<RPNElement>; /// Construct RPNBuilderTreeContext for ActionsDAG tree
using AtomFromASTFunc = std::function< explicit RPNBuilderTreeContext(ContextPtr query_context_);
bool(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out)>;
RPNBuilder(const SelectQueryInfo & query_info, ContextPtr context_, const AtomFromASTFunc & atom_from_ast_) /// Construct RPNBuilderTreeContext for AST tree
: WithContext(context_), atom_from_ast(atom_from_ast_) explicit RPNBuilderTreeContext(ContextPtr query_context_, Block block_with_constants_, PreparedSetsPtr prepared_sets_);
/// Get query context
const ContextPtr & getQueryContext() const
{ {
/** Evaluation of expressions that depend only on constants. return query_context;
* For the index to be used, if it is written, for example `WHERE Date = toDate(now())`.
*/
block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, getContext());
/// Transform WHERE section to Reverse Polish notation
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query_info.query);
if (select.where())
{
traverseAST(select.where());
if (select.prewhere())
{
traverseAST(select.prewhere());
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere())
{
traverseAST(select.prewhere());
}
else
{
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
}
} }
RPN && extractRPN() { return std::move(rpn); } /** 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: private:
void traverseAST(const ASTPtr & node) /// 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 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;
/** 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
ConstSetPtr tryGetPreparedSet() const;
/// Try get prepared set from node that match data types
ConstSetPtr tryGetPreparedSet(const DataTypes & data_types) const;
/// Try get prepared set from node that match indexes mapping and data types
ConstSetPtr tryGetPreparedSet(
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;
/// 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:
/// Get function name
std::string getFunctionName() const;
/// Get function arguments size
size_t getArgumentsSize() const;
/// Get argument at index
RPNBuilderTreeNode getArgumentAt(size_t index) const;
using RPNBuilderTreeNode::RPNBuilderTreeNode;
};
/** 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.
*/
template <typename RPNElement>
class RPNBuilder
{
public:
using RPNElements = std::vector<RPNElement>;
using ExtractAtomFromTreeFunction = std::function<bool (const RPNBuilderTreeNode & node, RPNElement & out)>;
explicit RPNBuilder(const ActionsDAG::Node * & filter_actions_dag_node,
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_)
{
traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context));
}
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));
}
RPNElements && extractRPN() && { return std::move(rpn_elements); }
private:
void traverseTree(const RPNBuilderTreeNode & node)
{ {
RPNElement element; RPNElement element;
if (ASTFunction * func = typeid_cast<ASTFunction *>(&*node)) if (node.isFunction())
{ {
if (operatorFromAST(func, element)) auto function_node = node.toFunctionNode();
if (extractLogicalOperatorFromTree(function_node, element))
{ {
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children; size_t arguments_size = function_node.getArgumentsSize();
for (size_t i = 0, size = args.size(); i < size; ++i)
for (size_t argument_index = 0; argument_index < arguments_size; ++argument_index)
{ {
traverseAST(args[i]); auto function_node_argument = function_node.getArgumentAt(argument_index);
traverseTree(function_node_argument);
/** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity /** 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). * - in this case `n - 1` elements are added (where `n` is the number of arguments).
*/ */
if (i != 0 || element.function == RPNElement::FUNCTION_NOT) if (argument_index != 0 || element.function == RPNElement::FUNCTION_NOT)
rpn.emplace_back(std::move(element)); rpn_elements.emplace_back(std::move(element));
} }
return; return;
} }
} }
if (!atom_from_ast(node, getContext(), block_with_constants, element)) if (!extract_atom_from_tree_function(node, element))
{
element.function = RPNElement::FUNCTION_UNKNOWN; element.function = RPNElement::FUNCTION_UNKNOWN;
}
rpn.emplace_back(std::move(element)); rpn_elements.emplace_back(std::move(element));
} }
bool operatorFromAST(const ASTFunction * func, RPNElement & out) bool extractLogicalOperatorFromTree(const RPNBuilderFunctionTreeNode & function_node, RPNElement & out)
{ {
/// Functions AND, OR, NOT. /// Functions AND, OR, NOT.
/// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses /// 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). /// (or, the same thing - calling the function `and` from one argument).
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
if (func->name == "not") auto function_name = function_node.getFunctionName();
if (function_name == "not")
{ {
if (args.size() != 1) if (function_node.getArgumentsSize() != 1)
return false; return false;
out.function = RPNElement::FUNCTION_NOT; out.function = RPNElement::FUNCTION_NOT;
} }
else else
{ {
if (func->name == "and" || func->name == "indexHint") if (function_name == "and" || function_name == "indexHint")
out.function = RPNElement::FUNCTION_AND; out.function = RPNElement::FUNCTION_AND;
else if (func->name == "or") else if (function_name == "or")
out.function = RPNElement::FUNCTION_OR; out.function = RPNElement::FUNCTION_OR;
else else
return false; return false;
@ -114,10 +265,9 @@ private:
return true; return true;
} }
const AtomFromASTFunc & atom_from_ast; RPNBuilderTreeContext tree_context;
Block block_with_constants; const ExtractAtomFromTreeFunction & extract_atom_from_tree_function;
RPN rpn; RPNElements rpn_elements;
}; };
} }