ClickHouse/src/Storages/VirtualColumnUtils.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

451 lines
16 KiB
C++
Raw Normal View History

#include <algorithm>
2022-07-18 15:53:30 +00:00
#include <memory>
2017-05-23 18:01:50 +00:00
#include <Core/NamesAndTypes.h>
#include <Core/TypeId.h>
#include <Interpreters/Context.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/IdentifierSemantic.h>
2021-03-03 08:36:20 +00:00
#include <Interpreters/misc.h>
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +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>
2021-04-20 14:53:17 +00:00
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
#include <Columns/ColumnsCommon.h>
#include <Columns/FilterDescription.h>
Squashed commit of the following: commit e712f469a55ff34ad34b482b15cc4153b7ad7233 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:59:13 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a002823084e3a79bffcc17d479620a68eb0644b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:58:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9e06f407c8ee781ed8ddf98bdfcc31846bf2a0fe Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:55:14 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 9581620f1e839f456fa7894aa1f996d5162ac6cd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:54:22 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2a8564c68cb6cc3649fafaf401256d43c9a2e777 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:47:34 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit cf60632d78ec656be3304ef4565e859bb6ce80ba Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:40:09 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit ee3d1dc6e0c4ca60e3ac1e0c30d4b3ed1e66eca0 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:22:49 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 65592ef7116a90104fcd524b53ef8b7cf22640f2 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:18:17 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 37972c257320d3b7e7b294e0fdeffff218647bfd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:17:06 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit dd909d149974ce5bed2456de1261aa5a368fd3ff Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:16:28 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 3cf43266ca7e30adf01212b1a739ba5fe43639fd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:15:42 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 6731a3df96d1609286e2536b6432916af7743f0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:13:35 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 1b5727e0d56415b7add4cb76110105358663602c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:11:18 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit bbcf726a55685b8e72f5b40ba0bf1904bd1c0407 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:09:04 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit c03b477d5e2e65014e8906ecfa2efb67ee295af1 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:06:30 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 2986e2fb0466bc18d73693dcdded28fccc0dc66b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:05:44 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit 5d6cdef13d2e02bd5c4954983334e9162ab2635b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:04:53 2017 +0300 Less dependencies [#CLICKHOUSE-2] commit f2b819b25ce8b2ccdcb201eefb03e1e6f5aab590 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Jan 14 11:01:47 2017 +0300 Less dependencies [#CLICKHOUSE-2]
2017-01-14 09:00:19 +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>
#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>
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
{
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)
{
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);
}
2017-03-25 20:12:56 +00:00
/// Extract all subfunctions of the main conjunction, but depending only on the specified columns
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, ASTs & result)
{
2019-03-11 13:22:51 +00:00
const auto * function = expression->as<ASTFunction>();
if (function)
{
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 = false;
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)));
result.push_back(makeASTForLogicalOr(std::move(or_args)));
}
return ret;
}
}
if (isValidFunction(expression, is_constant))
{
result.push_back(expression->clone());
2021-03-03 08:36:20 +00:00
return true;
}
2021-03-03 08:36:20 +00:00
else
return false;
}
2017-03-25 20:12:56 +00:00
/// Construct a conjunction from given functions
ASTPtr buildWhereExpression(ASTs && functions)
{
2020-03-08 21:40:00 +00:00
if (functions.empty())
return nullptr;
if (functions.size() == 1)
return functions[0];
return makeASTForLogicalAnd(std::move(functions));
}
2019-05-20 12:16:51 +00:00
}
namespace VirtualColumnUtils
{
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);
}
}
bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block block, ASTPtr & expression_ast)
{
2021-04-20 14:53:17 +00:00
if (block.rows() == 0)
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 &>();
if (!select.where() && !select.prewhere())
2021-03-08 03:09:06 +00:00
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)
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(
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 });
ActionsVisitor(visitor_data).visit(expr);
2021-03-09 18:48:46 +00:00
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;
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);
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
};
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.
ASTs functions;
if (select.where())
2021-03-09 18:48:46 +00:00
unmodified &= extractFunctions(select.where(), is_constant, functions);
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
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-03-25 20:12:56 +00:00
/// Filter the block.
String filter_column_name = expression_ast->getColumnName();
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullIfNeeded();
if (filter_column->getDataType() != TypeIndex::UInt8)
return;
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);
}
}
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;
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)
{
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);
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
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)
{
for (const auto & virtual_column : requested_virtual_columns)
{
if (virtual_column.name == "_path")
{
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path)->convertToFullColumnIfConst());
}
else if (virtual_column.name == "_file")
{
2023-08-30 08:40:55 +00:00
if (filename)
{
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);
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), filename_from_path)->convertToFullColumnIfConst());
2023-08-30 08:40:55 +00:00
}
}
}
}
}
}