mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Revert "Fix filtering by virtual columns with OR filter in query"
This commit is contained in:
parent
f7cf512824
commit
fb901c24a1
@ -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.
|
||||
|
@ -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 }
|
||||
|
@ -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
|
@ -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 }
|
Loading…
Reference in New Issue
Block a user