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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
#include <algorithm>
|
2022-07-18 15:53:30 +00:00
|
|
|
#include <memory>
|
2017-05-23 18:01:50 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
2023-09-20 11:32:07 +00:00
|
|
|
#include <Core/TypeId.h>
|
2016-05-23 00:40:28 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-07-22 17:13:05 +00:00
|
|
|
#include <Interpreters/TreeRewriter.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Interpreters/IdentifierSemantic.h>
|
2021-03-03 08:36:20 +00:00
|
|
|
#include <Interpreters/misc.h>
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2021-03-03 08:36:20 +00:00
|
|
|
#include <Parsers/ASTSubquery.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
2021-04-20 14:53:17 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
2017-07-21 06:35:58 +00:00
|
|
|
#include <Columns/ColumnsCommon.h>
|
2020-07-08 23:52:57 +00:00
|
|
|
#include <Columns/FilterDescription.h>
|
2017-01-14 09:00:19 +00:00
|
|
|
|
2023-08-17 16:54:43 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
|
|
|
|
2023-06-16 19:38:50 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
|
|
|
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
|
|
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
|
|
|
#include <Processors/Sinks/EmptySink.h>
|
|
|
|
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
|
|
|
|
2017-11-20 05:22:54 +00:00
|
|
|
#include <Storages/VirtualColumnUtils.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2023-07-26 18:25:48 +00:00
|
|
|
#include <Parsers/makeASTForLogicalFunction.h>
|
2023-05-24 17:53:37 +00:00
|
|
|
#include <Columns/ColumnSet.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
2021-03-04 14:27:07 +00:00
|
|
|
#include <Interpreters/ActionsVisitor.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
2014-02-26 10:59:56 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-04-20 14:53:17 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2019-05-20 12:16:51 +00:00
|
|
|
namespace
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
|
|
|
|
2017-03-25 20:12:56 +00:00
|
|
|
/// Verifying that the function depends only on the specified columns
|
2021-03-09 18:48:46 +00:00
|
|
|
bool isValidFunction(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant)
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
2021-03-03 08:36:20 +00:00
|
|
|
const auto * function = expression->as<ASTFunction>();
|
2021-03-04 05:59:57 +00:00
|
|
|
if (function && functionIsInOrGlobalInOperator(function->name))
|
2021-03-03 08:36:20 +00:00
|
|
|
{
|
2021-03-04 05:59:57 +00:00
|
|
|
// Second argument of IN can be a scalar subquery
|
2021-03-09 18:48:46 +00:00
|
|
|
return isValidFunction(function->arguments->children[0], is_constant);
|
2021-03-03 08:36:20 +00:00
|
|
|
}
|
|
|
|
else
|
2021-03-09 18:48:46 +00:00
|
|
|
return is_constant(expression);
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
|
|
|
|
2017-03-25 20:12:56 +00:00
|
|
|
/// Extract all subfunctions of the main conjunction, but depending only on the specified columns
|
2022-10-13 19:50:11 +00:00
|
|
|
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, ASTs & result)
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
const auto * function = expression->as<ASTFunction>();
|
2023-07-26 18:25:48 +00:00
|
|
|
|
|
|
|
if (function)
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
2023-07-26 18:25:48 +00:00
|
|
|
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")
|
|
|
|
{
|
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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
bool ret = false;
|
2023-07-26 18:25:48 +00:00
|
|
|
ASTs or_args;
|
|
|
|
for (const auto & child : function->arguments->children)
|
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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
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)));
|
2023-07-26 18:25:48 +00:00
|
|
|
result.push_back(makeASTForLogicalOr(std::move(or_args)));
|
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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
}
|
2023-07-26 18:25:48 +00:00
|
|
|
return ret;
|
|
|
|
}
|
2014-08-08 09:04:06 +00:00
|
|
|
}
|
2023-07-26 18:25:48 +00:00
|
|
|
|
|
|
|
if (isValidFunction(expression, is_constant))
|
2014-08-08 09:04:06 +00:00
|
|
|
{
|
|
|
|
result.push_back(expression->clone());
|
2021-03-03 08:36:20 +00:00
|
|
|
return true;
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
2021-03-03 08:36:20 +00:00
|
|
|
else
|
|
|
|
return false;
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
|
|
|
|
2017-03-25 20:12:56 +00:00
|
|
|
/// Construct a conjunction from given functions
|
2023-07-26 18:25:48 +00:00
|
|
|
ASTPtr buildWhereExpression(ASTs && functions)
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
2020-03-08 21:40:00 +00:00
|
|
|
if (functions.empty())
|
2017-12-15 03:47:43 +00:00
|
|
|
return nullptr;
|
|
|
|
if (functions.size() == 1)
|
|
|
|
return functions[0];
|
2023-07-26 18:25:48 +00:00
|
|
|
return makeASTForLogicalAnd(std::move(functions));
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
|
|
|
|
2019-05-20 12:16:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
namespace VirtualColumnUtils
|
|
|
|
{
|
|
|
|
|
2019-11-06 17:37:30 +00:00
|
|
|
void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func)
|
|
|
|
{
|
|
|
|
auto & select = ast->as<ASTSelectQuery &>();
|
|
|
|
if (!select.with())
|
|
|
|
select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared<ASTExpressionList>());
|
|
|
|
|
|
|
|
if (func.empty())
|
|
|
|
{
|
|
|
|
auto literal = std::make_shared<ASTLiteral>(value);
|
|
|
|
literal->alias = column_name;
|
|
|
|
literal->prefer_alias_to_column_name = true;
|
|
|
|
select.with()->children.push_back(literal);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto literal = std::make_shared<ASTLiteral>(value);
|
|
|
|
literal->prefer_alias_to_column_name = true;
|
|
|
|
|
|
|
|
auto function = makeASTFunction(func, literal);
|
|
|
|
function->alias = column_name;
|
|
|
|
function->prefer_alias_to_column_name = true;
|
|
|
|
select.with()->children.push_back(function);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block block, ASTPtr & expression_ast)
|
2014-02-26 10:59:56 +00:00
|
|
|
{
|
2021-04-20 14:53:17 +00:00
|
|
|
if (block.rows() == 0)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot prepare filter with empty block");
|
2021-04-20 14:53:17 +00:00
|
|
|
|
|
|
|
/// Take the first row of the input block to build a constant block
|
|
|
|
auto columns = block.getColumns();
|
|
|
|
Columns const_columns(columns.size());
|
|
|
|
for (size_t i = 0; i < columns.size(); ++i)
|
|
|
|
{
|
|
|
|
if (isColumnConst(*columns[i]))
|
|
|
|
const_columns[i] = columns[i]->cloneResized(1);
|
|
|
|
else
|
|
|
|
const_columns[i] = ColumnConst::create(columns[i]->cloneResized(1), 1);
|
|
|
|
}
|
2022-11-26 03:24:11 +00:00
|
|
|
|
2021-04-20 14:53:17 +00:00
|
|
|
block.setColumns(const_columns);
|
|
|
|
|
2021-03-08 03:09:06 +00:00
|
|
|
bool unmodified = true;
|
2021-03-04 14:27:07 +00:00
|
|
|
const auto & select = query->as<ASTSelectQuery &>();
|
2019-04-09 14:22:35 +00:00
|
|
|
if (!select.where() && !select.prewhere())
|
2021-03-08 03:09:06 +00:00
|
|
|
return unmodified;
|
2017-04-01 07:20:54 +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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
// 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)
|
2021-03-09 18:48:46 +00:00
|
|
|
{
|
|
|
|
auto actions = std::make_shared<ActionsDAG>(block.getColumnsWithTypeAndName());
|
2022-07-18 15:53:30 +00:00
|
|
|
PreparedSetsPtr prepared_sets = std::make_shared<PreparedSets>();
|
2022-05-25 23:22:29 +00:00
|
|
|
const NamesAndTypesList source_columns;
|
|
|
|
const NamesAndTypesList aggregation_keys;
|
|
|
|
const ColumnNumbersList grouping_set_keys;
|
|
|
|
|
2021-03-09 18:48:46 +00:00
|
|
|
ActionsVisitor::Data visitor_data(
|
2023-05-02 18:04:47 +00:00
|
|
|
context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true,
|
2022-05-25 23:22:29 +00:00
|
|
|
{ aggregation_keys, grouping_set_keys, GroupByKind::NONE });
|
2022-11-28 16:13:01 +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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
ActionsVisitor(visitor_data).visit(expr);
|
2021-03-09 18:48:46 +00:00
|
|
|
actions = visitor_data.getActions();
|
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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
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;
|
|
|
|
|
2021-03-09 18:48:46 +00:00
|
|
|
auto expression_actions = std::make_shared<ExpressionActions>(actions);
|
|
|
|
auto block_with_constants = block;
|
|
|
|
expression_actions->execute(block_with_constants);
|
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 b107712e0cf8e2671da7c1641b268df57e54bd7a)
2023-10-09 14:08:22 +00:00
|
|
|
return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column);
|
2021-03-09 18:48:46 +00:00
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-03-08 03:09:06 +00:00
|
|
|
/// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
|
2022-10-13 19:50:11 +00:00
|
|
|
ASTs functions;
|
2019-04-09 14:22:35 +00:00
|
|
|
if (select.where())
|
2021-03-09 18:48:46 +00:00
|
|
|
unmodified &= extractFunctions(select.where(), is_constant, functions);
|
2019-04-09 14:22:35 +00:00
|
|
|
if (select.prewhere())
|
2021-03-09 18:48:46 +00:00
|
|
|
unmodified &= extractFunctions(select.prewhere(), is_constant, functions);
|
2021-03-03 08:36:20 +00:00
|
|
|
|
2023-07-26 18:25:48 +00:00
|
|
|
expression_ast = buildWhereExpression(std::move(functions));
|
2021-03-08 03:09:06 +00:00
|
|
|
return unmodified;
|
2021-03-03 08:36:20 +00:00
|
|
|
}
|
|
|
|
|
2023-09-01 14:29:31 +00:00
|
|
|
static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & context)
|
2021-03-03 08:36:20 +00:00
|
|
|
{
|
2023-05-24 17:53:37 +00:00
|
|
|
for (const auto & node : actions->getNodes())
|
|
|
|
{
|
|
|
|
if (node.type == ActionsDAG::ActionType::COLUMN)
|
|
|
|
{
|
|
|
|
const ColumnSet * column_set = checkAndGetColumnConstData<const ColumnSet>(node.column.get());
|
|
|
|
if (!column_set)
|
|
|
|
column_set = checkAndGetColumn<const ColumnSet>(node.column.get());
|
|
|
|
|
|
|
|
if (column_set)
|
|
|
|
{
|
|
|
|
auto future_set = column_set->getData();
|
2023-06-19 12:56:24 +00:00
|
|
|
if (!future_set->get())
|
2023-05-24 17:53:37 +00:00
|
|
|
{
|
2023-06-16 19:38:50 +00:00
|
|
|
if (auto * set_from_subquery = typeid_cast<FutureSetFromSubquery *>(future_set.get()))
|
|
|
|
{
|
|
|
|
auto plan = set_from_subquery->build(context);
|
2023-09-01 14:29:31 +00:00
|
|
|
|
|
|
|
if (!plan)
|
|
|
|
continue;
|
|
|
|
|
2023-06-16 19:38:50 +00:00
|
|
|
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
|
|
|
|
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
|
|
|
|
pipeline.complete(std::make_shared<EmptySink>(Block()));
|
|
|
|
|
|
|
|
CompletedPipelineExecutor executor(pipeline);
|
|
|
|
executor.execute();
|
|
|
|
}
|
2023-05-24 17:53:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2023-09-01 14:29:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void filterBlockWithQuery(ActionsDAGPtr dag, Block & block, ContextPtr context)
|
|
|
|
{
|
|
|
|
auto actions = std::make_shared<ExpressionActions>(dag);
|
|
|
|
makeSets(actions, context);
|
|
|
|
Block block_with_filter = block;
|
|
|
|
actions->execute(block_with_filter);
|
|
|
|
|
|
|
|
/// Filter the block.
|
|
|
|
String filter_column_name = dag->getOutputs().at(0)->result_name;
|
|
|
|
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst();
|
|
|
|
|
|
|
|
ConstantFilterDescription constant_filter(*filter_column);
|
|
|
|
|
|
|
|
if (constant_filter.always_true)
|
|
|
|
{
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (constant_filter.always_false)
|
|
|
|
{
|
|
|
|
block = block.cloneEmpty();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
FilterDescription filter(*filter_column);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
|
|
|
ColumnPtr & column = block.safeGetByPosition(i).column;
|
|
|
|
column = column->filter(*filter.data, -1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr context, ASTPtr expression_ast)
|
|
|
|
{
|
|
|
|
if (block.rows() == 0)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (!expression_ast)
|
|
|
|
prepareFilterBlockWithQuery(query, context, block, expression_ast);
|
|
|
|
|
|
|
|
if (!expression_ast)
|
|
|
|
return;
|
|
|
|
|
|
|
|
/// Let's analyze and calculate the prepared expression.
|
|
|
|
auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList());
|
|
|
|
ExpressionAnalyzer analyzer(expression_ast, syntax_result, context);
|
|
|
|
ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes);
|
|
|
|
|
|
|
|
makeSets(actions, context);
|
2023-05-24 17:53:37 +00:00
|
|
|
|
2018-03-06 16:11:29 +00:00
|
|
|
Block block_with_filter = block;
|
|
|
|
actions->execute(block_with_filter);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-25 20:12:56 +00:00
|
|
|
/// Filter the block.
|
2014-07-29 14:05:15 +00:00
|
|
|
String filter_column_name = expression_ast->getColumnName();
|
2023-09-20 11:32:07 +00:00
|
|
|
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullIfNeeded();
|
|
|
|
if (filter_column->getDataType() != TypeIndex::UInt8)
|
|
|
|
return;
|
2020-07-08 23:52:57 +00:00
|
|
|
|
|
|
|
ConstantFilterDescription constant_filter(*filter_column);
|
|
|
|
|
|
|
|
if (constant_filter.always_true)
|
2021-04-11 19:39:22 +00:00
|
|
|
{
|
2020-07-08 23:52:57 +00:00
|
|
|
return;
|
2021-04-11 19:39:22 +00:00
|
|
|
}
|
2020-07-08 23:52:57 +00:00
|
|
|
|
|
|
|
if (constant_filter.always_false)
|
2021-04-11 19:39:22 +00:00
|
|
|
{
|
2020-07-08 23:52:57 +00:00
|
|
|
block = block.cloneEmpty();
|
2021-04-11 19:39:22 +00:00
|
|
|
return;
|
|
|
|
}
|
2020-07-08 23:52:57 +00:00
|
|
|
|
|
|
|
FilterDescription filter(*filter_column);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2014-07-29 14:05:15 +00:00
|
|
|
for (size_t i = 0; i < block.columns(); ++i)
|
|
|
|
{
|
2017-01-02 20:12:12 +00:00
|
|
|
ColumnPtr & column = block.safeGetByPosition(i).column;
|
2020-07-08 23:52:57 +00:00
|
|
|
column = column->filter(*filter.data, -1);
|
2014-07-29 14:05:15 +00:00
|
|
|
}
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
|
|
|
|
2023-08-17 16:54:43 +00:00
|
|
|
NamesAndTypesList getPathAndFileVirtualsForStorage(NamesAndTypesList storage_columns)
|
|
|
|
{
|
|
|
|
auto default_virtuals = NamesAndTypesList{
|
|
|
|
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
|
|
|
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
|
|
|
|
|
|
|
default_virtuals.sort();
|
|
|
|
storage_columns.sort();
|
|
|
|
|
|
|
|
NamesAndTypesList result_virtuals;
|
|
|
|
std::set_difference(
|
|
|
|
default_virtuals.begin(), default_virtuals.end(), storage_columns.begin(), storage_columns.end(),
|
|
|
|
std::back_inserter(result_virtuals),
|
|
|
|
[](const NameAndTypePair & lhs, const NameAndTypePair & rhs){ return lhs.name < rhs.name; });
|
|
|
|
|
|
|
|
return result_virtuals;
|
|
|
|
}
|
|
|
|
|
|
|
|
static void addPathAndFileToVirtualColumns(Block & block, const String & path, size_t idx)
|
|
|
|
{
|
|
|
|
if (block.has("_path"))
|
|
|
|
block.getByName("_path").column->assumeMutableRef().insert(path);
|
|
|
|
|
|
|
|
if (block.has("_file"))
|
|
|
|
{
|
|
|
|
auto pos = path.find_last_of('/');
|
2023-08-22 13:01:07 +00:00
|
|
|
String file;
|
|
|
|
if (pos != std::string::npos)
|
|
|
|
file = path.substr(pos + 1);
|
|
|
|
else
|
|
|
|
file = path;
|
2023-08-17 16:54:43 +00:00
|
|
|
|
|
|
|
block.getByName("_file").column->assumeMutableRef().insert(file);
|
|
|
|
}
|
|
|
|
|
|
|
|
block.getByName("_idx").column->assumeMutableRef().insert(idx);
|
|
|
|
}
|
|
|
|
|
2023-08-18 17:49:40 +00:00
|
|
|
ASTPtr createPathAndFileFilterAst(const ASTPtr & query, const NamesAndTypesList & virtual_columns, const String & path_example, const ContextPtr & context)
|
2023-08-17 16:54:43 +00:00
|
|
|
{
|
|
|
|
if (!query || virtual_columns.empty())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
Block block;
|
|
|
|
for (const auto & column : virtual_columns)
|
|
|
|
block.insert({column.type->createColumn(), column.type, column.name});
|
|
|
|
/// Create a block with one row to construct filter
|
|
|
|
/// Append "idx" column as the filter result
|
|
|
|
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
|
2023-08-18 17:49:40 +00:00
|
|
|
addPathAndFileToVirtualColumns(block, path_example, 0);
|
2023-08-17 16:54:43 +00:00
|
|
|
ASTPtr filter_ast;
|
|
|
|
prepareFilterBlockWithQuery(query, context, block, filter_ast);
|
|
|
|
return filter_ast;
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnPtr getFilterByPathAndFileIndexes(const std::vector<String> & paths, const ASTPtr & query, const NamesAndTypesList & virtual_columns, const ContextPtr & context, ASTPtr filter_ast)
|
|
|
|
{
|
|
|
|
Block block;
|
|
|
|
for (const auto & column : virtual_columns)
|
|
|
|
block.insert({column.type->createColumn(), column.type, column.name});
|
|
|
|
block.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
|
|
|
|
|
|
|
|
for (size_t i = 0; i != paths.size(); ++i)
|
|
|
|
addPathAndFileToVirtualColumns(block, paths[i], i);
|
|
|
|
|
|
|
|
filterBlockWithQuery(query, block, context, filter_ast);
|
2023-08-18 17:49:40 +00:00
|
|
|
|
2023-08-17 16:54:43 +00:00
|
|
|
return block.getByName("_idx").column;
|
|
|
|
}
|
|
|
|
|
2023-08-30 08:40:55 +00:00
|
|
|
void addRequestedPathAndFileVirtualsToChunk(
|
|
|
|
Chunk & chunk, const NamesAndTypesList & requested_virtual_columns, const String & path, const String * filename)
|
2023-08-17 16:54:43 +00:00
|
|
|
{
|
|
|
|
for (const auto & virtual_column : requested_virtual_columns)
|
|
|
|
{
|
|
|
|
if (virtual_column.name == "_path")
|
|
|
|
{
|
2023-09-21 17:14:28 +00:00
|
|
|
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path)->convertToFullColumnIfConst());
|
2023-08-17 16:54:43 +00:00
|
|
|
}
|
|
|
|
else if (virtual_column.name == "_file")
|
|
|
|
{
|
2023-08-30 08:40:55 +00:00
|
|
|
if (filename)
|
|
|
|
{
|
2023-09-21 17:14:28 +00:00
|
|
|
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), *filename)->convertToFullColumnIfConst());
|
2023-08-30 08:40:55 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
size_t last_slash_pos = path.find_last_of('/');
|
|
|
|
auto filename_from_path = path.substr(last_slash_pos + 1);
|
2023-09-21 17:14:28 +00:00
|
|
|
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst());
|
2023-08-30 08:40:55 +00:00
|
|
|
}
|
2023-08-17 16:54:43 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|
2014-07-06 00:19:49 +00:00
|
|
|
|
2014-02-26 10:59:56 +00:00
|
|
|
}
|