mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #2388 from yandex/fix-prepared-sets-creation
Fix prepared sets creation
This commit is contained in:
commit
0fe2984d00
@ -1486,7 +1486,7 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
|
||||
return;
|
||||
}
|
||||
|
||||
prepared_sets[subquery_or_table_name.get()] = std::move(set);
|
||||
prepared_sets[subquery_or_table_name->range] = std::move(set);
|
||||
}
|
||||
|
||||
|
||||
@ -1515,7 +1515,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
|
||||
if (!prepared_sets.count(arg.get())) /// Not already prepared.
|
||||
if (!prepared_sets.count(arg->range)) /// Not already prepared.
|
||||
{
|
||||
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
|
||||
{
|
||||
@ -1550,7 +1550,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
|
||||
/// Already converted.
|
||||
if (prepared_sets.count(arg.get()))
|
||||
if (prepared_sets.count(arg->range))
|
||||
return;
|
||||
|
||||
/// If the subquery or table name for SELECT.
|
||||
@ -1573,7 +1573,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
|
||||
if (storage_set)
|
||||
{
|
||||
prepared_sets[arg.get()] = storage_set->getSet();
|
||||
prepared_sets[arg->range] = storage_set->getSet();
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1584,7 +1584,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
/// If you already created a Set with the same subquery / table.
|
||||
if (subquery_for_set.set)
|
||||
{
|
||||
prepared_sets[arg.get()] = subquery_for_set.set;
|
||||
prepared_sets[arg->range] = subquery_for_set.set;
|
||||
return;
|
||||
}
|
||||
|
||||
@ -1630,7 +1630,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
}
|
||||
|
||||
subquery_for_set.set = set;
|
||||
prepared_sets[arg.get()] = set;
|
||||
prepared_sets[arg->range] = set;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1712,7 +1712,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
|
||||
|
||||
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
|
||||
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
|
||||
prepared_sets[right_arg.get()] = std::move(set);
|
||||
prepared_sets[right_arg->range] = std::move(set);
|
||||
}
|
||||
|
||||
|
||||
@ -2102,12 +2102,12 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (prepared_sets.count(child.get()) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
|
||||
else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
const SetPtr & set = prepared_sets[child.get()];
|
||||
const SetPtr & set = prepared_sets[child->range];
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
|
@ -3,9 +3,9 @@
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Interpreters/Settings.h>
|
||||
#include <Core/Block.h>
|
||||
#include "ExpressionActions.h"
|
||||
#include "ProjectionManipulation.h"
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ProjectionManipulation.h>
|
||||
#include <Parsers/StringRange.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -23,7 +23,7 @@ using ASTPtr = std::shared_ptr<IAST>;
|
||||
|
||||
class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
|
||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangeHash>;
|
||||
|
||||
class IBlockInputStream;
|
||||
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
|
||||
|
@ -26,7 +26,7 @@ public:
|
||||
Kind kind;
|
||||
|
||||
ASTIdentifier(const String & name_, const Kind kind_ = Column)
|
||||
: name(name_), kind(kind_) {}
|
||||
: name(name_), kind(kind_) { range = StringRange(name.data(), name.data() + name.size()); }
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID() const override { return "Identifier_" + name; }
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
|
||||
ASTPtr ptr() { return shared_from_this(); }
|
||||
|
||||
/** Get a deep copy of the tree. */
|
||||
/** Get a deep copy of the tree. Cloned object must have the same range. */
|
||||
virtual ASTPtr clone() const = 0;
|
||||
|
||||
/** Get hash code, identifying this element and its subtree.
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <map>
|
||||
#include <memory>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,9 +15,10 @@ struct StringRange
|
||||
const char * first = nullptr;
|
||||
const char * second = nullptr;
|
||||
|
||||
StringRange() {}
|
||||
StringRange() = default;
|
||||
StringRange(const StringRange & other) = default;
|
||||
StringRange(const char * begin, const char * end) : first(begin), second(end) {}
|
||||
StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
explicit StringRange(TokenIterator token) : first(token->begin), second(token->end) {}
|
||||
|
||||
StringRange(TokenIterator token_begin, TokenIterator token_end)
|
||||
{
|
||||
@ -34,6 +36,8 @@ struct StringRange
|
||||
first = token_begin->begin;
|
||||
second = token_last->end;
|
||||
}
|
||||
|
||||
bool operator==(const StringRange & rhs) const { return std::tie(first, second) == std::tie(rhs.first, rhs.second); }
|
||||
};
|
||||
|
||||
using StringPtr = std::shared_ptr<String>;
|
||||
@ -44,4 +48,16 @@ inline String toString(const StringRange & range)
|
||||
return range.first ? String(range.first, range.second) : String();
|
||||
}
|
||||
|
||||
struct StringRangeHash
|
||||
{
|
||||
UInt64 operator()(const StringRange & range) const
|
||||
{
|
||||
SipHash hash;
|
||||
hash.update(range.first);
|
||||
hash.update(range.second);
|
||||
return hash.get64();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -641,8 +641,8 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
||||
bool is_set_const = false;
|
||||
bool is_constant_transformed = false;
|
||||
|
||||
if (prepared_sets.count(args[1].get())
|
||||
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1].get()], key_column_num))
|
||||
if (prepared_sets.count(args[1]->range)
|
||||
&& isTupleIndexable(args[0], context, out, prepared_sets[args[1]->range], key_column_num))
|
||||
{
|
||||
key_arg_pos = 0;
|
||||
is_set_const = true;
|
||||
@ -1016,7 +1016,7 @@ bool KeyCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, c
|
||||
{
|
||||
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
|
||||
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
|
||||
PreparedSets::const_iterator it = prepared_sets.find(args[1].get());
|
||||
PreparedSets::const_iterator it = prepared_sets.find(args[1]->range);
|
||||
if (in_func && it != prepared_sets.end())
|
||||
{
|
||||
rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types));
|
||||
|
@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
|
||||
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) ||
|
||||
(primary_key_columns.count(second_arg_name) && isConstant(args[0])) ||
|
||||
(primary_key_columns.count(first_arg_name)
|
||||
&& (prepared_sets.count(args[1].get()) || typeid_cast<const ASTSubquery *>(args[1].get()))))
|
||||
&& (prepared_sets.count(args[1]->range) || typeid_cast<const ASTSubquery *>(args[1].get()))))
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <memory>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Parsers/StringRange.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,7 +14,7 @@ class Set;
|
||||
using SetPtr = std::shared_ptr<Set>;
|
||||
|
||||
/// Information about calculated sets in right hand side of IN.
|
||||
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
|
||||
using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangeHash>;
|
||||
|
||||
|
||||
/** Query along with some additional data,
|
||||
|
@ -84,33 +84,6 @@ bool StorageMerge::isRemote() const
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
using NodeHashToSet = std::map<IAST::Hash, SetPtr>;
|
||||
|
||||
void relinkSetsImpl(const ASTPtr & query, const NodeHashToSet & node_hash_to_set, PreparedSets & new_sets)
|
||||
{
|
||||
auto hash = query->getTreeHash();
|
||||
auto it = node_hash_to_set.find(hash);
|
||||
if (node_hash_to_set.end() != it)
|
||||
new_sets[query.get()] = it->second;
|
||||
|
||||
for (const auto & child : query->children)
|
||||
relinkSetsImpl(child, node_hash_to_set, new_sets);
|
||||
}
|
||||
|
||||
/// Re-link prepared sets onto cloned and modified AST.
|
||||
void relinkSets(const ASTPtr & query, const PreparedSets & old_sets, PreparedSets & new_sets)
|
||||
{
|
||||
NodeHashToSet node_hash_to_set;
|
||||
for (const auto & node_set : old_sets)
|
||||
node_hash_to_set.emplace(node_set.first->getTreeHash(), node_set.second);
|
||||
|
||||
relinkSetsImpl(query, node_hash_to_set, new_sets);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const
|
||||
{
|
||||
/// It's beneficial if it is true for at least one table.
|
||||
@ -210,8 +183,7 @@ BlockInputStreams StorageMerge::read(
|
||||
|
||||
SelectQueryInfo modified_query_info;
|
||||
modified_query_info.query = modified_query_ast;
|
||||
|
||||
relinkSets(modified_query_info.query, query_info.sets, modified_query_info.sets);
|
||||
modified_query_info.sets = query_info.sets;
|
||||
|
||||
BlockInputStreams source_streams;
|
||||
|
||||
|
@ -0,0 +1,10 @@
|
||||
1 1 1 test_in_tuple_1
|
||||
1 2 2 test_in_tuple_1
|
||||
2 1 1 test_in_tuple_2
|
||||
2 2 2 test_in_tuple_2
|
||||
-
|
||||
1 1 1 test_in_tuple_1
|
||||
2 1 1 test_in_tuple_2
|
||||
-
|
||||
1 1 1 test_in_tuple_1
|
||||
2 1 1 test_in_tuple_2
|
@ -0,0 +1,15 @@
|
||||
drop table if exists test_in_tuple_1;
|
||||
drop table if exists test_in_tuple_2;
|
||||
drop table if exists test_in_tuple;
|
||||
|
||||
create table test_in_tuple_1 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
|
||||
create table test_in_tuple_2 (key Int32, key_2 Int32, x Array(Int32), y Array(Int32)) engine = MergeTree order by (key, key_2);
|
||||
create table test_in_tuple as test_in_tuple_1 engine = Merge('default', '^test_in_tuple_[0-9]+$');
|
||||
|
||||
insert into test_in_tuple_1 values (1, 1, [1, 2], [1, 2]);
|
||||
insert into test_in_tuple_2 values (2, 1, [1, 2], [1, 2]);
|
||||
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y order by _table;
|
||||
select '-';
|
||||
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
|
||||
select '-';
|
||||
select key, arr_x, arr_y, _table from test_in_tuple left array join arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), x, x ,y) as arr_x, arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), y, x ,y) as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
|
Loading…
Reference in New Issue
Block a user