Added IN function Set table support

This commit is contained in:
Maksim Kita 2022-08-27 16:32:50 +02:00
parent de1f470cfe
commit efc2433347
6 changed files with 46 additions and 6 deletions

View File

@ -53,6 +53,7 @@
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Storages/StorageSet.h>
#include <Interpreters/convertFieldToType.h> #include <Interpreters/convertFieldToType.h>
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
@ -2358,7 +2359,11 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc
auto * query_node = in_second_argument->as<QueryNode>(); auto * query_node = in_second_argument->as<QueryNode>();
auto * union_node = in_second_argument->as<UnionNode>(); auto * union_node = in_second_argument->as<UnionNode>();
if (table_node || table_function_node) if (table_node && dynamic_cast<StorageSet *>(table_node->getStorage().get()) != nullptr)
{
/// If table is already prepared set, we do not replace it with subquery
}
else if (table_node || table_function_node)
{ {
const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
@ -2412,7 +2417,6 @@ void QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveSc
function_lambda_arguments_indexes.push_back(function_argument_index); function_lambda_arguments_indexes.push_back(function_argument_index);
} }
else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE || else if (is_special_function_in && (function_argument->getNodeType() == QueryTreeNodeType::TABLE ||
function_argument->getNodeType() == QueryTreeNodeType::TABLE_FUNCTION ||
function_argument->getNodeType() == QueryTreeNodeType::QUERY || function_argument->getNodeType() == QueryTreeNodeType::QUERY ||
function_argument->getNodeType() == QueryTreeNodeType::UNION)) function_argument->getNodeType() == QueryTreeNodeType::UNION))
{ {

View File

@ -35,7 +35,7 @@ namespace DB
* *
* For function `in` and its variations arguments are resolved, but sets are not build. * For function `in` and its variations arguments are resolved, but sets are not build.
* If left and right arguments are constants constant folding is performed. * If left and right arguments are constants constant folding is performed.
* If right argument resolved as table function, or table, it is replaced with query that read only ordinary columns from underlying * If right argument resolved as table function, or table, and table is not of type Set, it is replaced with query that read only ordinary columns from underlying
* storage. * storage.
* Example: SELECT id FROM test_table WHERE id IN test_table_other; * Example: SELECT id FROM test_table WHERE id IN test_table_other;
* Result: SELECT id FROM test_table WHERE id IN (SELECT test_table_column FROM test_table_other); * Result: SELECT id FROM test_table WHERE id IN (SELECT test_table_column FROM test_table_other);

View File

@ -508,7 +508,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon
for (size_t i = 0; i < function_parameters_nodes_size; ++i) for (size_t i = 0; i < function_parameters_nodes_size; ++i)
{ {
const auto & function_parameter_node = function_parameters_nodes[i]; const auto & function_parameter_node = function_parameters_nodes[i];
calculateActionsDAGNodeName(function_parameter_node.get(), planner_context); calculateActionsDAGNodeName(function_parameter_node.get(), planner_context, node_to_name);
if (i + 1 != function_parameters_nodes_size) if (i + 1 != function_parameters_nodes_size)
buffer << ", "; buffer << ", ";
@ -525,7 +525,7 @@ String calculateActionsDAGNodeName(const IQueryTreeNode * node, const PlannerCon
for (size_t i = 0; i < function_arguments_nodes_size; ++i) for (size_t i = 0; i < function_arguments_nodes_size; ++i)
{ {
const auto & function_argument_node = function_arguments_nodes[i]; const auto & function_argument_node = function_arguments_nodes[i];
buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context); buffer << calculateActionsDAGNodeName(function_argument_node.get(), planner_context, node_to_name);
if (i + 1 != function_arguments_nodes_size) if (i + 1 != function_arguments_nodes_size)
buffer << ", "; buffer << ", ";

View File

@ -2,11 +2,14 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Storages/StorageSet.h>
#include <Analyzer/Utils.h> #include <Analyzer/Utils.h>
#include <Analyzer/SetUtils.h> #include <Analyzer/SetUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h> #include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ConstantNode.h> #include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h> #include <Analyzer/FunctionNode.h>
#include <Analyzer/TableNode.h>
namespace DB namespace DB
{ {
@ -44,7 +47,15 @@ public:
if (prepared_set) if (prepared_set)
return; return;
if (in_second_argument_node_type == QueryTreeNodeType::QUERY || /// 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)
{
global_planner_context->registerSet(set_key, storage_set->getSet());
}
else if (in_second_argument_node_type == QueryTreeNodeType::QUERY ||
in_second_argument_node_type == QueryTreeNodeType::UNION) in_second_argument_node_type == QueryTreeNodeType::UNION)
{ {
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};

View File

@ -0,0 +1,2 @@
0 Value_0
1 Value_1

View File

@ -0,0 +1,23 @@
SET use_analyzer = 1;
DROP TABLE IF EXISTS test_table;
CREATE TABLE test_table
(
id UInt64,
value String
) ENGINE=TinyLog;
INSERT INTO test_table VALUES (0, 'Value_0'), (1, 'Value_1'), (2, 'Value_2');
DROP TABLE IF EXISTS special_set_table;
CREATE TABLE special_set_table
(
id UInt64
) ENGINE=Set;
INSERT INTO special_set_table VALUES (0), (1);
SELECT id, value FROM test_table WHERE id IN special_set_table;
DROP TABLE special_set_table;
DROP TABLE test_table;