Merge pull request #52653 from azat/virtual-columns-or

RFC: Fix filtering by virtual columns with OR expression
This commit is contained in:
robot-ch-test-poll3 2023-07-28 09:18:36 +02:00 committed by GitHub
commit d6e2e8b92c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 103 additions and 9 deletions

View File

@ -795,6 +795,10 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(
bool filter_by_database_virtual_column /* = false */,
bool filter_by_table_virtual_column /* = false */) const
{
/// FIXME: filtering does not work with allow_experimental_analyzer due to
/// different column names there (it has "table_name._table" not just
/// "_table")
assert(!filter_by_database_virtual_column || !filter_by_table_virtual_column || query);
const Settings & settings = query_context->getSettingsRef();

View File

@ -30,6 +30,7 @@
#include <Storages/VirtualColumnUtils.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
#include <Parsers/makeASTForLogicalFunction.h>
#include <Columns/ColumnSet.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/ActionsVisitor.h>
@ -63,14 +64,31 @@ bool isValidFunction(const ASTPtr & expression, const std::function<bool(const A
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, ASTs & result)
{
const auto * function = expression->as<ASTFunction>();
if (function && (function->name == "and" || function->name == "indexHint"))
if (function)
{
bool ret = true;
for (const auto & child : function->arguments->children)
ret &= extractFunctions(child, is_constant, result);
return ret;
if (function->name == "and" || function->name == "indexHint")
{
bool ret = true;
for (const auto & child : function->arguments->children)
ret &= extractFunctions(child, is_constant, result);
return ret;
}
else if (function->name == "or")
{
bool ret = true;
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)
result.push_back(makeASTForLogicalOr(std::move(or_args)));
return ret;
}
}
else if (isValidFunction(expression, is_constant))
if (isValidFunction(expression, is_constant))
{
result.push_back(expression->clone());
return true;
@ -80,13 +98,13 @@ bool extractFunctions(const ASTPtr & expression, const std::function<bool(const
}
/// Construct a conjunction from given functions
ASTPtr buildWhereExpression(const ASTs & functions)
ASTPtr buildWhereExpression(ASTs && functions)
{
if (functions.empty())
return nullptr;
if (functions.size() == 1)
return functions[0];
return makeASTFunction("and", functions);
return makeASTForLogicalAnd(std::move(functions));
}
}
@ -171,7 +189,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
if (select.prewhere())
unmodified &= extractFunctions(select.prewhere(), is_constant, functions);
expression_ast = buildWhereExpression(functions);
expression_ast = buildWhereExpression(std::move(functions));
return unmodified;
}

View File

@ -0,0 +1,38 @@
-- { echoOn }
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=0, convert_query_to_cnf=0;
v1 1
v1 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=0, convert_query_to_cnf=0;
v1 1
v2 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=0;
v1 1
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=0;
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=0, convert_query_to_cnf=1;
v1 1
v1 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=0, convert_query_to_cnf=1;
v1 1
v2 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=1;
v1 1
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=0, convert_query_to_cnf=1;
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=1, convert_query_to_cnf=0;
v1 1
v1 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=1, convert_query_to_cnf=0;
v1 1
v2 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=0;
v1 1
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=0;
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings allow_experimental_analyzer=1, convert_query_to_cnf=1;
v1 1
v1 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings allow_experimental_analyzer=1, convert_query_to_cnf=1;
v1 1
v2 2
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=1;
v1 1
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings allow_experimental_analyzer=1, convert_query_to_cnf=1;

View File

@ -0,0 +1,34 @@
drop table if exists m;
drop view if exists v1;
drop view if exists v2;
drop table if exists d1;
drop table if exists d2;
create table d1 (key Int, value Int) engine=Memory();
create table d2 (key Int, value Int) engine=Memory();
insert into d1 values (1, 10);
insert into d1 values (2, 20);
insert into d2 values (1, 10);
insert into d2 values (2, 20);
create view v1 as select * from d1;
create view v2 as select * from d2;
create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$');
-- avoid reorder
set max_threads=1;
-- { echoOn }
{% for settings in [
'allow_experimental_analyzer=0, convert_query_to_cnf=0',
'allow_experimental_analyzer=0, convert_query_to_cnf=1',
'allow_experimental_analyzer=1, convert_query_to_cnf=0',
'allow_experimental_analyzer=1, convert_query_to_cnf=1'
] %}
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') settings {{ settings }};
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v2') settings {{ settings }};
select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v3') settings {{ settings }};
select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') settings {{ settings }};
{% endfor %}