This commit is contained in:
Nikolai Kochetov 2021-06-02 19:56:24 +03:00
parent 9a652e6aba
commit c855cf7057
2 changed files with 171 additions and 43 deletions

View File

@ -114,6 +114,130 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr
return res; return res;
} }
class KeyCondition::Tree
{
public:
explicit Tree(const IAST * ast_) : ast(ast_) {}
explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) {}
std::string getColumnName() const
{
if (ast)
return ast->getColumnNameWithoutAlias();
else
getColumnNameWithoutAlias(dag);
}
// size_t numChildren() const
// {
// if (ast)
// return ast->children.size();
// else
// return dag->children.size();
// }
// Tree getChildrenAt(size_t idx) const
// {
// if (ast)
// return Tree(ast->children[idx].get());
// else
// return Tree(dag->children[idx]);
// }
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->type == ActionsDAG::ActionType::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;
}
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 ? 0 : func->arguments->size();
}
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` /// 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
@ -560,18 +684,19 @@ 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 ASTPtr & node, ContextPtr context, Block & block_with_constants) void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants)
{ {
RPNElement element; RPNElement element;
if (const auto * func = node->as<ASTFunction>()) if (node.isFunction())
{ {
auto func = node.asFunction();
if (tryParseLogicalOperatorFromAST(func, element)) if (tryParseLogicalOperatorFromAST(func, element))
{ {
auto & args = func->arguments->children; size_t num_args = func.numArguments();
for (size_t i = 0, size = args.size(); i < size; ++i) for (size_t i = 0; i < num_args; ++i)
{ {
traverseAST(args[i], context, block_with_constants); 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 /** 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).
@ -968,13 +1093,13 @@ private:
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node, const Tree & node,
ContextPtr context, 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<const ASTFunction *> chain_not_tested_for_monotonicity; std::vector<FunctionTree> 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))
@ -982,14 +1107,14 @@ 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)
{ {
const auto & args = (*it)->arguments->children; auto func = *it;
auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context); auto func_builder = FunctionFactory::instance().tryGet(func.getFunctionName(), context);
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 (args.size() == 2) if (func.numArguments() == 2)
{ {
if (const auto * arg_left = args[0]->as<ASTLiteral>()) if (const auto * arg_left = args[0]->as<ASTLiteral>())
{ {
@ -1029,10 +1154,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions(
} }
bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node, const Tree & 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<const ASTFunction *> & out_functions_chain) std::vector<FunctionTree> & 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.
@ -1040,7 +1165,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
const auto & sample_block = key_expr->getSampleBlock(); const auto & sample_block = key_expr->getSampleBlock();
// Key columns should use canonical names for index analysis // Key columns should use canonical names for index analysis
String name = node->getColumnNameWithoutAlias(); String name = node.getColumnName();
auto it = key_columns.find(name); auto it = key_columns.find(name);
if (key_columns.end() != it) if (key_columns.end() != it)
@ -1050,31 +1175,30 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl(
return true; return true;
} }
if (const auto * func = node->as<ASTFunction>()) if (node.isFunction())
{ {
if (!func->arguments) auto func = node.asFunction();
return false;
const auto & args = func->arguments->children; size_t num_args = func.numArguments();
if (args.size() > 2 || args.empty()) if (num_args > 2 || num_args == 0)
return false; return false;
out_functions_chain.push_back(func); out_functions_chain.push_back(func);
bool ret = false; bool ret = false;
if (args.size() == 2) if (num_args == 2)
{ {
if (args[0]->as<ASTLiteral>()) if (func.getArgumentAt(0).isConstant())
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[1], out_key_column_num, out_key_column_type, out_functions_chain); ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain);
} }
else if (args[1]->as<ASTLiteral>()) else if (func.getArgumentAt(1).isConstant())
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain); ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
} }
} }
else else
{ {
ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain); ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain);
} }
return ret; return ret;
} }
@ -1099,7 +1223,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value,
} }
bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out) bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out)
{ {
/** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of key, /** Functions < > = != <= >= in `notIn`, 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,
@ -1107,27 +1231,28 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context,
*/ */
Field const_value; Field const_value;
DataTypePtr const_type; DataTypePtr const_type;
if (const auto * func = node->as<ASTFunction>()) if (node.isFunction())
{ {
const ASTs & args = func->arguments->children; auto func = node.asFunction();
size_t num_args = func.numArguments();
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)
MonotonicFunctionsChain chain; MonotonicFunctionsChain chain;
std::string func_name = func->name; std::string func_name = func.getFunctionName();
if (atom_map.find(func_name) == std::end(atom_map)) if (atom_map.find(func_name) == std::end(atom_map))
return false; return false;
if (args.size() == 1) if (num_args == 1)
{ {
if (!(isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))) if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, 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))
throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR);
} }
else if (args.size() == 2) else if (num_args == 2)
{ {
size_t key_arg_pos; /// Position of argument with key column (non-const argument) size_t key_arg_pos; /// Position of argument with key column (non-const argument)
bool is_set_const = false; bool is_set_const = false;
@ -1315,25 +1440,24 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context,
return false; return false;
} }
bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out) bool KeyCondition::tryParseLogicalOperatorFromAST(const FunctionTree & func, 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 = func->arguments->children;
if (func->name == "not") if (func.getFunctionName() == "not")
{ {
if (args.size() != 1) if (func.numArguments() != 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 (func.getFunctionName() == "and" || func.getFunctionName() == "indexHint")
out.function = RPNElement::FUNCTION_AND; out.function = RPNElement::FUNCTION_AND;
else if (func->name == "or") else if (func.getFunctionName() == "or")
out.function = RPNElement::FUNCTION_OR; out.function = RPNElement::FUNCTION_OR;
else else
return false; return false;

View File

@ -378,6 +378,10 @@ public:
static const AtomMap atom_map; static const AtomMap atom_map;
private: private:
class Tree;
class FunctionTree;
BoolMask checkInRange( BoolMask checkInRange(
size_t used_key_size, size_t used_key_size,
const FieldRef * left_key, const FieldRef * left_key,
@ -386,9 +390,9 @@ private:
bool right_bounded, bool right_bounded,
BoolMask initial_mask) const; BoolMask initial_mask) const;
void traverseAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants); void traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants);
bool tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out); bool tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out);
static bool tryParseLogicalOperatorFromAST(const ASTFunction * func, 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,
@ -397,17 +401,17 @@ private:
* and fills chain of possibly-monotonic functions. * and fills chain of possibly-monotonic functions.
*/ */
bool isKeyPossiblyWrappedByMonotonicFunctions( bool isKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node, const Tree & node,
ContextPtr context, 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 ASTPtr & node, const Tree & 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<const ASTFunction *> & out_functions_chain); std::vector<FunctionTree> & out_functions_chain);
bool canConstantBeWrappedByMonotonicFunctions( bool canConstantBeWrappedByMonotonicFunctions(
const ASTPtr & node, const ASTPtr & node,