mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Revert "Merge pull request #61564 from liuneng1994/optimize_in_single_value"
This reverts commit20a45b4073
, reversing changes made toa642f4d3ec
.
This commit is contained in:
parent
c297e72a4d
commit
b2e764f633
@ -1,75 +0,0 @@
|
||||
#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);
|
||||
auto new_const = std::make_shared<ConstantNode>(constant_node->getValue(), removeNullable(constant_node->getResultType()));
|
||||
new_const->getSourceExpression() = constant_node->getSourceExpression();
|
||||
QueryTreeNodes arguments{column_node->clone(), new_const};
|
||||
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);
|
||||
}
|
||||
}
|
@ -1,27 +0,0 @@
|
||||
#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;
|
||||
};
|
||||
}
|
@ -28,7 +28,6 @@
|
||||
#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>
|
||||
@ -264,7 +263,6 @@ 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>());
|
||||
|
@ -14,8 +14,6 @@ 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
|
||||
|
@ -13,11 +13,6 @@ 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
|
||||
|
@ -1,11 +0,0 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver(bool decimal_check_overflow);
|
||||
}
|
@ -12,11 +12,6 @@ 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
|
||||
|
@ -1,11 +0,0 @@
|
||||
#pragma once
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IFunctionOverloadResolver;
|
||||
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
|
||||
|
||||
FunctionOverloadResolverPtr createInternalFunctionNotEqualOverloadResolver(bool decimal_check_overflow);
|
||||
}
|
@ -306,8 +306,7 @@ 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" != '') AND ("field" LIKE '%test%'))");
|
||||
R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))");
|
||||
}
|
||||
|
||||
TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere)
|
||||
@ -409,6 +408,5 @@ 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" = 1)");
|
||||
R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))");
|
||||
}
|
||||
|
@ -1,28 +0,0 @@
|
||||
<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>
|
@ -1,189 +0,0 @@
|
||||
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)
|
||||
-------------------
|
@ -1,32 +0,0 @@
|
||||
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);
|
||||
select '-------------------';
|
||||
--- fuzzed
|
||||
SELECT number FROM numbers(2) WHERE arrayExists(_ -> (_ IN toNullable(4294967290)), [number]);
|
Loading…
Reference in New Issue
Block a user