Merge pull request #2388 from yandex/fix-prepared-sets-creation

Fix prepared sets creation
This commit is contained in:
alexey-milovidov 2018-05-19 19:03:04 +03:00 committed by GitHub
commit 0fe2984d00
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 65 additions and 52 deletions

View File

@ -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).

View File

@ -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>;

View File

@ -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; }

View File

@ -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.

View File

@ -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();
}
};
}

View File

@ -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));

View File

@ -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;
}

View File

@ -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,

View File

@ -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;

View File

@ -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

View File

@ -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;