Merge pull request #61564 from liuneng1994/optimize_in_single_value

New analyzer pass to optimize in single value
This commit is contained in:
Yarik Briukhovetskyi 2024-03-25 12:53:01 +01:00 committed by GitHub
commit 20a45b4073
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 385 additions and 2 deletions

View File

@ -0,0 +1,73 @@
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/Passes/ConvertInToEqualPass.h>
#include <Functions/equals.h>
#include <Functions/notEquals.h>
namespace DB
{
class ConvertInToEqualPassVisitor : public InDepthQueryTreeVisitorWithContext<ConvertInToEqualPassVisitor>
{
public:
using Base = InDepthQueryTreeVisitorWithContext<ConvertInToEqualPassVisitor>;
using Base::Base;
void enterImpl(QueryTreeNodePtr & node)
{
static const std::unordered_map<String, String> MAPPING = {
{"in", "equals"},
{"notIn", "notEquals"}
};
auto * func_node = node->as<FunctionNode>();
if (!func_node
|| !MAPPING.contains(func_node->getFunctionName())
|| func_node->getArguments().getNodes().size() != 2)
return ;
auto args = func_node->getArguments().getNodes();
auto * column_node = args[0]->as<ColumnNode>();
auto * constant_node = args[1]->as<ConstantNode>();
if (!column_node || !constant_node)
return ;
// IN multiple values is not supported
if (constant_node->getValue().getType() == Field::Types::Which::Tuple
|| constant_node->getValue().getType() == Field::Types::Which::Array)
return ;
// x IN null not equivalent to x = null
if (constant_node->getValue().isNull())
return ;
auto result_func_name = MAPPING.at(func_node->getFunctionName());
auto equal = std::make_shared<FunctionNode>(result_func_name);
QueryTreeNodes arguments{column_node->clone(), constant_node->clone()};
equal->getArguments().getNodes() = std::move(arguments);
FunctionOverloadResolverPtr resolver;
bool decimal_check_overflow = getContext()->getSettingsRef().decimal_check_overflow;
if (result_func_name == "equals")
{
resolver = createInternalFunctionEqualOverloadResolver(decimal_check_overflow);
}
else
{
resolver = createInternalFunctionNotEqualOverloadResolver(decimal_check_overflow);
}
try
{
equal->resolveAsFunction(resolver);
}
catch (...)
{
// When function resolver fails, we should not replace the function node
return;
}
node = equal;
}
};
void ConvertInToEqualPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
ConvertInToEqualPassVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Optimize `in` to `equals` if possible.
* 1. convert in single value to equal
* Example: SELECT * from test where x IN (1);
* Result: SELECT * from test where x = 1;
*
* 2. convert not in single value to notEqual
* Example: SELECT * from test where x NOT IN (1);
* Result: SELECT * from test where x != 1;
*
* If value is null or tuple, do not convert.
*/
class ConvertInToEqualPass final : public IQueryTreePass
{
public:
String getName() override { return "ConvertInToEqualPass"; }
String getDescription() override { return "Convert in to equal"; }
void run(QueryTreeNodePtr & query_tree_node, ContextPtr context) override;
};
}

View File

@ -28,6 +28,7 @@
#include <Analyzer/Passes/MultiIfToIfPass.h>
#include <Analyzer/Passes/IfConstantConditionPass.h>
#include <Analyzer/Passes/IfChainToMultiIfPass.h>
#include <Analyzer/Passes/ConvertInToEqualPass.h>
#include <Analyzer/Passes/OrderByTupleEliminationPass.h>
#include <Analyzer/Passes/NormalizeCountVariantsPass.h>
#include <Analyzer/Passes/AggregateFunctionsArithmericOperationsPass.h>
@ -263,6 +264,7 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze)
manager.addPass(std::make_unique<SumIfToCountIfPass>());
manager.addPass(std::make_unique<RewriteArrayExistsToHasPass>());
manager.addPass(std::make_unique<NormalizeCountVariantsPass>());
manager.addPass(std::make_unique<ConvertInToEqualPass>());
/// should before AggregateFunctionsArithmericOperationsPass
manager.addPass(std::make_unique<AggregateFunctionOfGroupByKeysPass>());

View File

@ -14,6 +14,8 @@ extract_into_parent_list(clickhouse_functions_sources dbms_sources
multiMatchAny.cpp
checkHyperscanRegexp.cpp
array/has.cpp
equals.cpp
notEquals.cpp
CastOverloadResolver.cpp
)
extract_into_parent_list(clickhouse_functions_headers dbms_headers

View File

@ -13,6 +13,11 @@ REGISTER_FUNCTION(Equals)
factory.registerFunction<FunctionEquals>();
}
FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver(bool decimal_check_overflow)
{
return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionEquals>(decimal_check_overflow));
}
template <>
ColumnPtr FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const

11
src/Functions/equals.h Normal file
View File

@ -0,0 +1,11 @@
#pragma once
#include <memory>
namespace DB
{
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver(bool decimal_check_overflow);
}

View File

@ -12,6 +12,11 @@ REGISTER_FUNCTION(NotEquals)
factory.registerFunction<FunctionNotEquals>();
}
FunctionOverloadResolverPtr createInternalFunctionNotEqualOverloadResolver(bool decimal_check_overflow)
{
return std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionNotEquals>(decimal_check_overflow));
}
template <>
ColumnPtr FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(
const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const

11
src/Functions/notEquals.h Normal file
View File

@ -0,0 +1,11 @@
#pragma once
#include <memory>
namespace DB
{
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
FunctionOverloadResolverPtr createInternalFunctionNotEqualOverloadResolver(bool decimal_check_overflow);
}

View File

@ -306,7 +306,8 @@ TEST(TransformQueryForExternalDatabase, Aliases)
check(state, 1, {"field"},
"SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'",
R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))");
R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))",
R"(SELECT "field" FROM "test"."table" WHERE ("field" != '') AND ("field" LIKE '%test%'))");
}
TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere)
@ -408,5 +409,6 @@ TEST(TransformQueryForExternalDatabase, Analyzer)
check(state, 1, {"column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo"},
"SELECT * FROM table WHERE (column) IN (1)",
R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))");
R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))",
R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" = 1)");
}

View File

@ -0,0 +1,28 @@
<test>
<settings>
<max_insert_threads>8</max_insert_threads>
<max_threads>1</max_threads>
</settings>
<create_query>
CREATE TABLE t_nullable
(
key_string1 Nullable(String),
key_string2 Nullable(String),
key_string3 Nullable(String),
key_int64_1 Nullable(Int64),
key_int64_2 Nullable(Int64),
key_int64_3 Nullable(Int64),
key_int64_4 Nullable(Int64),
key_int64_5 Nullable(Int64),
m1 Int64,
m2 Int64
)
ENGINE = Memory
</create_query>
<fill_query>insert into t_nullable select ['aaaaaa','bbaaaa','ccaaaa','ddaaaa'][number % 101 + 1], ['aa','bb','cc','dd'][number % 100 + 1], ['aa','bb','cc','dd'][number % 102 + 1], number%10+1, number%10+2, number%10+3, number%10+4,number%10+5, number%6000+1, number%5000+2 from numbers_mt(30000000)</fill_query>
<query>select * from t_nullable where key_string1 in ('aaaaaa') format Null SETTINGS allow_experimental_analyzer=1</query>
<query>select * from t_nullable where key_string2 in ('3') format Null SETTINGS allow_experimental_analyzer=1</query>
<drop_query>drop table if exists t_nullable</drop_query>
</test>

View File

@ -0,0 +1,188 @@
a 1
-------------------
0
0
0
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'a\', constant_value_type: String
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'A\', constant_value_type: String
EXPRESSION
FUNCTION id: 9, function_name: upper, function_type: ordinary, result_type: String
ARGUMENTS
LIST id: 10, nodes: 1
CONSTANT id: 11, constant_value: \'a\', constant_value_type: String
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String)
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String)
-------------------
b 2
c 3
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'a\', constant_value_type: String
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: \'A\', constant_value_type: String
EXPRESSION
FUNCTION id: 9, function_name: upper, function_type: ordinary, result_type: String
ARGUMENTS
LIST id: 10, nodes: 1
CONSTANT id: 11, constant_value: \'a\', constant_value_type: String
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String)
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String)
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing)
-------------------
QUERY id: 0
PROJECTION COLUMNS
x String
y Int32
PROJECTION
LIST id: 1, nodes: 2
COLUMN id: 2, column_name: x, result_type: String, source_id: 3
COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3
JOIN TREE
TABLE id: 3, alias: __table1, table_name: default.test
WHERE
FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8
ARGUMENTS
LIST id: 6, nodes: 2
COLUMN id: 7, column_name: x, result_type: String, source_id: 3
CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing)

View File

@ -0,0 +1,29 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x;
SET allow_experimental_analyzer = 1;
INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3);
select * from test where x in ('a');
select '-------------------';
select x in Null from test;
select '-------------------';
explain query tree select * from test where x in ('a');
select '-------------------';
explain query tree select * from test where x in (upper('a'));
select '-------------------';
explain query tree select * from test where x in ('a','b');
select '-------------------';
explain query tree select * from test where x in ['a','b'];
select '-------------------';
select * from test where x not in ('a');
select '-------------------';
explain query tree select * from test where x not in ('a');
select '-------------------';
explain query tree select * from test where x not in (upper('a'));
select '-------------------';
explain query tree select * from test where x not in ('a','b');
select '-------------------';
explain query tree select * from test where x not in ['a','b'];
select '-------------------';
explain query tree select * from test where x not in (NULL);
select '-------------------';
explain query tree select * from test where x in (NULL);