2022-07-18 15:53:30 +00:00
|
|
|
#include <memory>
|
2017-05-23 18:01:50 +00:00
|
|
|
#include <Core/NamesAndTypes.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")
|
|
|
|
{
|
|
|
|
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;
|
|
|
|
}
|
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
|
|
|
|
2021-03-09 18:48:46 +00:00
|
|
|
// 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());
|
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
|
|
|
|
2021-03-09 18:48:46 +00:00
|
|
|
ActionsVisitor(visitor_data).visit(node);
|
|
|
|
actions = visitor_data.getActions();
|
|
|
|
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);
|
|
|
|
};
|
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();
|
2018-12-21 16:00:07 +00:00
|
|
|
ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst();
|
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")
|
|
|
|
{
|
|
|
|
chunk.addColumn(virtual_column.type->createColumnConst(chunk.getNumRows(), path));
|
|
|
|
}
|
|
|
|
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));
|
|
|
|
}
|
|
|
|
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));
|
|
|
|
}
|
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
|
|
|
}
|