Replace subqueries and table expressions on the right side of the IN operator with their results in order to make MergeTree storage engines evaluate the corresponding conditions by using the index.

This commit is contained in:
Vlad Arkhipov 2018-01-21 10:30:07 +03:00
parent c430af5fb8
commit be96cf5133
12 changed files with 151 additions and 14 deletions

View File

@ -1528,6 +1528,26 @@ void ExpressionAnalyzer::makeSetsForIndex()
makeSetsForIndexImpl(ast, storage->getSampleBlock());
}
void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table_name)
{
BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute();
SetPtr set = std::make_shared<Set>(settings.limits);
while (Block block = res.in->read())
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
if (!set->insertFromBlock(block, true))
return;
}
set->makeOrderedSet();
prepared_sets[subquery_or_table_name.get()] = std::move(set);
}
void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block)
{
for (auto & child : node->children)
@ -1539,18 +1559,25 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
const IAST & args = *func->arguments;
const ASTPtr & arg = args.children.at(1);
if (!prepared_sets.count(arg.get()) /// Not already prepared.
&& !typeid_cast<ASTSubquery *>(arg.get()) && !typeid_cast<ASTIdentifier *>(arg.get()))
if (!prepared_sets.count(arg.get())) /// Not already prepared.
{
try
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
{
makeExplicitSet(func, sample_block, true);
if (settings.try_primary_key_for_in_with_subqueries && storage->mayBenefitFromIndexForIn(args.children.at(0)))
tryMakeSetFromSubquery(arg);
}
catch (const Exception & e)
else
{
/// in `sample_block` there are no columns that add `getActions`
if (e.code() != ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK)
throw;
try
{
makeExplicitSet(func, sample_block, true);
}
catch (const Exception & e)
{
/// in `sample_block` there are no columns that add `getActions`
if (e.code() != ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK)
throw;
}
}
}
}

View File

@ -312,6 +312,12 @@ private:
* If create_ordered_set = true - create a data structure suitable for using the index.
*/
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set);
/***
* Create Set from a subuqery or a table expression in the query.
*/
void tryMakeSetFromSubquery(const ASTPtr & subquery_or_table_name);
void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block);
/** Translate qualified names such as db.table.column, table.column, table_alias.column

View File

@ -177,8 +177,13 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
}
if (create_ordered_set)
{
if (!ordered_set_elements)
ordered_set_elements = OrderedSetElementsPtr(new OrderedSetElements());
for (size_t i = 0; i < rows; ++i)
ordered_set_elements->push_back((*key_columns[0])[i]); /// ordered_set for index works only for single key, not for tuples
}
if (!checkSetSizeLimits())
{
@ -204,6 +209,20 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
}
void Set::makeOrderedSet()
{
if (!ordered_set_elements)
{
ordered_set_elements = OrderedSetElementsPtr(new OrderedSetElements());
}
else
{
std::sort(ordered_set_elements->begin(), ordered_set_elements->end());
ordered_set_elements->erase(std::unique(ordered_set_elements->begin(), ordered_set_elements->end()), ordered_set_elements->end());
}
}
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)
{
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(node.get()))
@ -282,10 +301,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
insertFromBlock(block, create_ordered_set);
if (create_ordered_set)
{
std::sort(ordered_set_elements->begin(), ordered_set_elements->end());
ordered_set_elements->erase(std::unique(ordered_set_elements->begin(), ordered_set_elements->end()), ordered_set_elements->end());
}
makeOrderedSet();
}

View File

@ -39,8 +39,13 @@ public:
*/
void createFromAST(const DataTypes & types, ASTPtr node, const Context & context, bool create_ordered_set);
// Returns false, if some limit was exceeded and no need to insert more data.
/** Returns false, if some limit was exceeded and no need to insert more data.
*
* Note that if create_ordered_set = true, you must call the method makeOrderedSet() after all required blocks
* have been inserted.
*/
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
void makeOrderedSet();
/** For columns of 'block', check belonging of corresponding rows to the set.
* Return UInt8 column with the result.

View File

@ -305,7 +305,9 @@ struct Settings
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown")
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \
/** Try using the primary key if there is a subquery or a table expression on the right side of the IN operator. */ \
M(SettingBool, try_primary_key_for_in_with_subqueries, true, "Try using the primary key if there is a subquery or a table expression on the right side of the IN operator.")
/// Possible limits for query execution.

View File

@ -277,6 +277,9 @@ public:
/// Does table support index for IN sections
virtual bool supportsIndexForIn() const { return false; }
/// Provides a hint that the storage engine may evaluate the IN-condition by using the index.
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */) const { return false; }
/// Checks validity of the data
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }

View File

@ -2159,5 +2159,35 @@ void MergeTreeData::Transaction::replaceParts(MergeTreeData::DataPartState move_
}
}
bool MergeTreeData::isIndexColumn(const ASTPtr & node) const
{
String column_name = node->getColumnName();
for (const auto & column : sort_descr)
if (column_name == column.column_name)
return true;
return false;
}
bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const
{
/// Make sure that the left side of the IN operator is part of the primary key. If there is a tuple on the left side
/// of the IN operator, each item of the tuple must be part of the primary key.
const ASTFunction * left_in_operand_tuple = typeid_cast<const ASTFunction *>(left_in_operand.get());
if (left_in_operand_tuple && left_in_operand_tuple->name == "tuple")
{
for (const auto & item : left_in_operand_tuple->arguments->children)
if (!isIndexColumn(item))
return false;
/// tuple() is invalid but can still be found here since this method may be called before the arguments are validated.
return left_in_operand_tuple->arguments->children.size() != 0;
}
else
{
return isIndexColumn(left_in_operand);
}
}
}

View File

@ -303,6 +303,8 @@ public:
|| merging_params.mode == MergingParams::Replacing;
}
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const;
Int64 getMaxDataPartIndex();
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
@ -662,6 +664,9 @@ private:
/// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock.
DataPartPtr getAnyPartInPartition(const String & partition_id, std::unique_lock<std::mutex> & data_parts_lock);
/// Checks whether the column is in the primary key.
bool isIndexColumn(const ASTPtr & node) const;
};
}

View File

@ -73,6 +73,7 @@ public:
void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override;
bool supportsIndexForIn() const override { return true; }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const override { return data.mayBenefitFromIndexForIn(left_in_operand); }
bool checkTableCanBeDropped() const override;

View File

@ -123,6 +123,7 @@ public:
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
bool supportsIndexForIn() const override { return true; }
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const override { return data.mayBenefitFromIndexForIn(left_in_operand); }
bool checkTableCanBeDropped() const override;

View File

@ -0,0 +1,13 @@
2 test2 8
3 test3 8
4 test4 1985
2 test2 8
3 test3 8
4 test4 1985
2 test2
3 test3
4 test4
2 test2
3 test3
5 test5
2 test2

View File

@ -0,0 +1,28 @@
DROP TABLE IF EXISTS test.merge_tree_in_subqueries;
CREATE TABLE test.merge_tree_in_subqueries (id UInt64, name String, num UInt64) ENGINE = MergeTree ORDER BY (id, name);
INSERT INTO test.merge_tree_in_subqueries VALUES(1, 'test1', 42);
INSERT INTO test.merge_tree_in_subqueries VALUES(2, 'test2', 8);
INSERT INTO test.merge_tree_in_subqueries VALUES(3, 'test3', 8);
INSERT INTO test.merge_tree_in_subqueries VALUES(4, 'test4', 1985);
INSERT INTO test.merge_tree_in_subqueries VALUES(5, 'test5', 0);
-- Index scans.
SET force_primary_key = 1;
SELECT * FROM test.merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 0);
SELECT * FROM test.merge_tree_in_subqueries WHERE id IN (SELECT * FROM system.numbers LIMIT 2, 3) ORDER BY id;
SELECT * FROM test.merge_tree_in_subqueries WHERE name IN (SELECT 'test' || toString(number) FROM system.numbers LIMIT 2, 3) ORDER BY id;
/* This should be removed when the issue of using the index for tuples in the IN operator is addressed. */
SET force_primary_key = 0;
SELECT id AS id2, name AS value FROM test.merge_tree_in_subqueries WHERE (value, id2) IN (SELECT 'test' || toString(number), number FROM system.numbers LIMIT 2, 3) ORDER BY id;
-- Non-index scans.
SET force_primary_key = 0;
SELECT id AS id2, name AS value FROM test.merge_tree_in_subqueries WHERE num IN (SELECT number FROM system.numbers LIMIT 10) ORDER BY id;
SELECT id AS id2, name AS value FROM test.merge_tree_in_subqueries WHERE (id, num) IN (SELECT number, number + 6 FROM system.numbers LIMIT 10) ORDER BY id;
DROP TABLE IF EXISTS test.merge_tree_in_subqueries;