Merge pull request #55657 from ClickHouse/revert-55418-fix-OR

Revert "Fix filtering by virtual columns with OR filter in query"
This commit is contained in:
Antonio Andelic 2023-10-16 10:51:06 +02:00 committed by GitHub
commit 6c543c730b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 10 additions and 92 deletions

View File

@ -1,4 +1,3 @@
#include <algorithm>
#include <memory>
#include <Core/NamesAndTypes.h>
#include <Core/TypeId.h>
@ -82,33 +81,14 @@ bool extractFunctions(const ASTPtr & expression, const std::function<bool(const
}
else if (function->name == "or")
{
bool ret = false;
bool ret = true;
ASTs or_args;
for (const auto & child : function->arguments->children)
ret |= extractFunctions(child, is_constant, or_args);
if (!or_args.empty())
{
/// In case of there are less number of arguments for which
/// is_constant() == true, we need to add always-true
/// implicitly to avoid breaking AND invariant.
///
/// Consider the following:
///
/// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20))
///
/// Without implicit always-true:
///
/// (_table = 'v2') AND (_table = 'v1')
///
/// With:
///
/// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1')
///
if (or_args.size() != function->arguments->children.size())
or_args.push_back(std::make_shared<ASTLiteral>(Field(1)));
ret &= extractFunctions(child, is_constant, or_args);
/// We can keep condition only if it still OR condition (i.e. we
/// have dependent conditions for columns at both sides)
if (or_args.size() == 2)
result.push_back(makeASTForLogicalOr(std::move(or_args)));
}
return ret;
}
}
@ -185,10 +165,8 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
if (!select.where() && !select.prewhere())
return unmodified;
// Provide input columns as constant columns to check if an expression is
// constant and depends on the columns from provided block (the last is
// required to allow skipping some conditions for handling OR).
std::function<bool(const ASTPtr &)> is_constant = [&block, &context](const ASTPtr & expr)
// 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());
PreparedSetsPtr prepared_sets = std::make_shared<PreparedSets>();
@ -200,26 +178,13 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true,
{ aggregation_keys, grouping_set_keys, GroupByKind::NONE });
ActionsVisitor(visitor_data).visit(expr);
ActionsVisitor(visitor_data).visit(node);
actions = visitor_data.getActions();
auto expr_column_name = expr->getColumnName();
const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name);
if (!expr_const_node)
return false;
auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context);
const auto & nodes = filter_actions->getNodes();
bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node)
{
return block.has(node.result_name);
});
if (!has_dependent_columns)
return false;
auto expression_actions = std::make_shared<ExpressionActions>(actions);
auto block_with_constants = block;
expression_actions->execute(block_with_constants);
return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column);
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.

View File

@ -18,11 +18,6 @@ create view v2 as select * from d2;
create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$');
{# -- FIXME:
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }};
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }};
#}
-- avoid reorder
set max_threads=1;
-- { echoOn }

View File

@ -1,14 +0,0 @@
-- { echoOn }
SELECT * FROM or_bug WHERE (key = 1) OR false OR false;
1
SELECT * FROM or_bug WHERE (key = 1) OR false;
1
SELECT * FROM or_bug WHERE (key = 1);
1
-- { echoOn }
select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1;
5840ead423829c1eab29fa97 this is a test
select * from forms where text_field like '%this%' or 0 = 1;
5840ead423829c1eab29fa97 this is a test
select * from forms where text_field like '%this%';
5840ead423829c1eab29fa97 this is a test

View File

@ -1,28 +0,0 @@
-- https://github.com/ClickHouse/ClickHouse/pull/52653
DROP TABLE IF EXISTS or_bug;
CREATE TABLE or_bug (key UInt8) ENGINE=MergeTree ORDER BY key;
INSERT INTO TABLE or_bug VALUES (0), (1);
-- { echoOn }
SELECT * FROM or_bug WHERE (key = 1) OR false OR false;
SELECT * FROM or_bug WHERE (key = 1) OR false;
SELECT * FROM or_bug WHERE (key = 1);
-- { echoOff }
-- https://github.com/ClickHouse/ClickHouse/issues/55288
DROP TABLE IF EXISTS forms;
CREATE TABLE forms
(
`form_id` FixedString(24),
`text_field` String
)
ENGINE = MergeTree
PRIMARY KEY form_id
ORDER BY form_id;
insert into forms values ('5840ead423829c1eab29fa97','this is a test');
-- { echoOn }
select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1;
select * from forms where text_field like '%this%' or 0 = 1;
select * from forms where text_field like '%this%';
-- { echoOff }