Correctly check constant expr

This commit is contained in:
Amos Bird 2021-03-10 02:48:46 +08:00
parent b936619fa9
commit 02604185f2
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4

View File

@ -31,33 +31,30 @@ namespace
{
/// Verifying that the function depends only on the specified columns
bool isValidFunction(const ASTPtr & expression, const Block & block)
bool isValidFunction(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant)
{
const auto * function = expression->as<ASTFunction>();
if (function && functionIsInOrGlobalInOperator(function->name))
{
// Second argument of IN can be a scalar subquery
return isValidFunction(function->arguments->children[0], block);
return isValidFunction(function->arguments->children[0], is_constant);
}
else
{
auto column_name = expression->getColumnName();
return block.has(column_name) && isColumnConst(*block.getByName(column_name).column);
}
return is_constant(expression);
}
/// Extract all subfunctions of the main conjunction, but depending only on the specified columns
bool extractFunctions(const ASTPtr & expression, const Block & block, std::vector<ASTPtr> & result)
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, std::vector<ASTPtr> & result)
{
const auto * function = expression->as<ASTFunction>();
if (function && (function->name == "and" || function->name == "indexHint"))
{
bool ret = true;
for (const auto & child : function->arguments->children)
ret &= extractFunctions(child, block, result);
ret &= extractFunctions(child, is_constant, result);
return ret;
}
else if (isValidFunction(expression, block))
else if (isValidFunction(expression, is_constant))
{
result.push_back(expression->clone());
return true;
@ -142,23 +139,29 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context,
for (size_t i = 0; i < block.columns(); ++i)
block.getByPosition(i).column = block.getByPosition(i).type->createColumnConstWithDefaultValue(1);
// Collect all expression columns in expression_ast. Constant expressions will have constant columns.
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
PreparedSets prepared_sets;
SubqueriesForSets subqueries_for_sets;
ActionsVisitor::Data visitor_data(
context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false);
ActionsVisitor(visitor_data).visit(condition_ast);
actions = visitor_data.getActions();
auto expression_actions = std::make_shared<ExpressionActions>(actions);
expression_actions->execute(block);
// Provide input columns as constant columns to check if an expression is constant.
std::function<bool(const ASTPtr &)> is_constant = [&block, &context](const ASTPtr & node)
{
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
PreparedSets prepared_sets;
SubqueriesForSets subqueries_for_sets;
ActionsVisitor::Data visitor_data(
context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false);
ActionsVisitor(visitor_data).visit(node);
actions = visitor_data.getActions();
auto expression_actions = std::make_shared<ExpressionActions>(actions);
auto block_with_constants = block;
expression_actions->execute(block_with_constants);
auto column_name = node->getColumnName();
return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column);
};
/// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
std::vector<ASTPtr> functions;
if (select.where())
unmodified &= extractFunctions(select.where(), block, functions);
unmodified &= extractFunctions(select.where(), is_constant, functions);
if (select.prewhere())
unmodified &= extractFunctions(select.prewhere(), block, functions);
unmodified &= extractFunctions(select.prewhere(), is_constant, functions);
expression_ast = buildWhereExpression(functions);
return unmodified;