mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Fix filtering by virtual columns with OR filter in query
The problem with the initial implementation #52653 was:
- OR can have multiple arguments
- It simply not correct to assume that if there are two arguments this is OK.
Consider the following example:
"WHERE (column_not_from_partition_by = 1) OR false OR false"
Will be converted to:
"WHERE false OR false"
And it will simply read nothing.
Yes, we could apply some optimization for bool, but this will not always
work, since to optimize things like "0 = 1" we need to execute it.
And the only way to make handle this correctly (with ability to ignore
some commands during filtering) is to make is_constant() function return
has it use something from the input block, so that we can be sure, that
we have some sensible, and not just "false".
Plus we cannot simply ignore the difference of the input and output
arguments of handling OR, we need to add always-true (1/true) if the
size is different, since otherwise it could break invariants (see
comment in the code).
This includes (but not limited to):
- _part* filtering for MergeTree
- _path/_file for various File/HDFS/... engines
- _table for Merge
- ...
P.S. analyzer does not have this bug, since it execute expression as
whole, and this is what filterBlockWithQuery() should do actually
instead, but this will be a more complex patch.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
(cherry picked from commit b107712e0c
)
This commit is contained in:
parent
6c543c730b
commit
96f2a46a66
@ -1,3 +1,4 @@
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/TypeId.h>
|
||||
@ -81,14 +82,33 @@ bool extractFunctions(const ASTPtr & expression, const std::function<bool(const
|
||||
}
|
||||
else if (function->name == "or")
|
||||
{
|
||||
bool ret = true;
|
||||
bool ret = false;
|
||||
ASTs or_args;
|
||||
for (const auto & child : function->arguments->children)
|
||||
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)
|
||||
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)));
|
||||
result.push_back(makeASTForLogicalOr(std::move(or_args)));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
@ -165,8 +185,10 @@ 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.
|
||||
std::function<bool(const ASTPtr &)> is_constant = [&block, &context](const ASTPtr & node)
|
||||
// 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)
|
||||
{
|
||||
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
|
||||
PreparedSetsPtr prepared_sets = std::make_shared<PreparedSets>();
|
||||
@ -178,13 +200,26 @@ 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(node);
|
||||
ActionsVisitor(visitor_data).visit(expr);
|
||||
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);
|
||||
auto column_name = node->getColumnName();
|
||||
return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column);
|
||||
return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column);
|
||||
};
|
||||
|
||||
/// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
|
||||
|
@ -18,6 +18,11 @@ 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 }
|
||||
|
14
tests/queries/0_stateless/02896_multiple_OR.reference
Normal file
14
tests/queries/0_stateless/02896_multiple_OR.reference
Normal file
@ -0,0 +1,14 @@
|
||||
-- { 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
|
28
tests/queries/0_stateless/02896_multiple_OR.sql
Normal file
28
tests/queries/0_stateless/02896_multiple_OR.sql
Normal file
@ -0,0 +1,28 @@
|
||||
-- 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 }
|
Loading…
Reference in New Issue
Block a user