ClickHouse/src/Planner/CollectSets.cpp

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

142 lines
5.6 KiB
C++
Raw Normal View History

2022-09-05 10:24:54 +00:00
#include <Planner/CollectSets.h>
#include <Interpreters/Context.h>
2023-04-04 21:47:05 +00:00
#include <Interpreters/PreparedSets.h>
2022-08-27 14:32:50 +00:00
#include <Storages/StorageSet.h>
#include <Analyzer/Utils.h>
#include <Analyzer/SetUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
2022-08-27 14:32:50 +00:00
#include <Analyzer/TableNode.h>
2023-05-04 17:54:08 +00:00
#include <DataTypes/DataTypeTuple.h>
#include <Planner/Planner.h>
namespace DB
{
2022-09-06 15:25:52 +00:00
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
namespace
{
class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor<CollectSetsVisitor>
{
public:
2023-05-04 17:54:08 +00:00
explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_)
: planner_context(planner_context_)
2023-05-04 17:54:08 +00:00
, select_query_options(select_query_options_)
{}
void visitImpl(const QueryTreeNodePtr & node)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || !isNameOfInFunction(function_node->getFunctionName()))
return;
auto in_first_argument = function_node->getArguments().getNodes().at(0);
auto in_second_argument = function_node->getArguments().getNodes().at(1);
auto in_second_argument_node_type = in_second_argument->getNodeType();
const auto & settings = planner_context.getQueryContext()->getSettingsRef();
2023-05-04 17:54:08 +00:00
// String set_key = planner_context.createSetKey(in_second_argument);
2023-05-04 17:54:08 +00:00
// if (planner_context.hasSet(set_key))
// return;
auto & sets = planner_context.getPreparedSets();
2022-08-27 14:32:50 +00:00
/// Tables and table functions are replaced with subquery at Analysis stage, except special Set table.
auto * second_argument_table = in_second_argument->as<TableNode>();
StorageSet * storage_set = second_argument_table != nullptr ? dynamic_cast<StorageSet *>(second_argument_table->getStorage().get()) : nullptr;
if (storage_set)
{
2023-04-13 16:24:57 +00:00
/// Handle storage_set as ready set.
2023-05-04 17:54:08 +00:00
auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash());
sets.addFromStorage(set_key, storage_set->getSet());
//planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet())));
2022-08-27 14:32:50 +00:00
}
2022-11-29 11:35:05 +00:00
else if (const auto * constant_node = in_second_argument->as<ConstantNode>())
2022-08-31 15:21:17 +00:00
{
auto set = makeSetForConstantValue(
in_first_argument->getResultType(),
2022-11-29 11:35:05 +00:00
constant_node->getValue(),
constant_node->getResultType(),
2023-05-04 17:54:08 +00:00
settings.transform_null_in);
DataTypes set_element_types = {in_first_argument->getResultType()};
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(set_element_types.front().get());
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
set_element_types = left_tuple_type->getElements();
for (auto & element_type : set_element_types)
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
element_type = low_cardinality_type->getDictionaryType();
auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types);
2023-05-25 13:33:52 +00:00
sets.addFromTuple(set_key, std::move(set), settings);
2022-08-31 15:21:17 +00:00
2023-05-04 17:54:08 +00:00
//planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set))));
2022-08-31 15:21:17 +00:00
}
2022-08-27 14:32:50 +00:00
else if (in_second_argument_node_type == QueryTreeNodeType::QUERY ||
in_second_argument_node_type == QueryTreeNodeType::UNION)
{
2023-05-04 17:54:08 +00:00
auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash());
auto subquery_options = select_query_options.subquery();
Planner subquery_planner(
in_second_argument,
subquery_options,
planner_context.getGlobalPlannerContext());
subquery_planner.buildQueryPlanIfNeeded();
// const auto & settings = planner_context.getQueryContext()->getSettingsRef();
// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
// bool tranform_null_in = settings.transform_null_in;
// auto set = std::make_shared<Set>(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in);
SubqueryForSet subquery_for_set;
subquery_for_set.key = planner_context.createSetKey(in_second_argument);
subquery_for_set.source = std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan());
sets.addFromSubquery(set_key, std::move(subquery_for_set));
//planner_context.registerSet(set_key, PlannerSet(in_second_argument));
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Function '{}' is supported only if second argument is constant or table expression",
2022-09-06 15:25:52 +00:00
function_node->getFunctionName());
}
}
2023-02-18 16:06:00 +00:00
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child_node)
{
auto child_node_type = child_node->getNodeType();
return !(child_node_type == QueryTreeNodeType::QUERY || child_node_type == QueryTreeNodeType::UNION);
}
private:
2022-10-17 15:09:45 +00:00
PlannerContext & planner_context;
2023-05-04 17:54:08 +00:00
const SelectQueryOptions & select_query_options;
};
}
2023-05-04 17:54:08 +00:00
void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options)
{
2023-05-04 17:54:08 +00:00
CollectSetsVisitor visitor(planner_context, select_query_options);
visitor.visit(node);
}
}