Merge pull request #1804 from arkhipov/master

Replace subqueries and table expressions on the right side of the IN …
This commit is contained in:
alexey-milovidov 2018-01-23 22:31:49 +03:00 committed by GitHub
commit b8cb9267c7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 154 additions and 15 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->finalizeOrderedSet();
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,8 +1559,14 @@ 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.
{
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
{
if (settings.use_index_for_in_with_subqueries && storage->mayBenefitFromIndexForIn(args.children.at(0)))
tryMakeSetFromSubquery(arg);
}
else
{
try
{
@ -1555,6 +1581,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
}
}
}
}
void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_block)

View File

@ -312,6 +312,13 @@ 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. The created set is suitable for using the index.
* The set will not be created if its size hits the limit.
*/
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 = std::make_unique<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::finalizeOrderedSet()
{
if (!ordered_set_elements)
{
ordered_set_elements = std::make_unique<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()))
@ -273,7 +292,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
}
if (create_ordered_set)
ordered_set_elements = OrderedSetElementsPtr(new OrderedSetElements());
ordered_set_elements = std::make_unique<OrderedSetElements>();
Block block;
for (size_t i = 0, size = types.size(); i < size; ++i)
@ -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());
}
finalizeOrderedSet();
}

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 finalizeOrderedSet() after all required blocks
* have been inserted.
*/
bool insertFromBlock(const Block & block, bool create_ordered_set = false);
void finalizeOrderedSet();
/** 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 an index if there is a subquery or a table expression on the right side of the IN operator. */ \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index 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 an 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,36 @@ void MergeTreeData::Transaction::replaceParts(MergeTreeData::DataPartState move_
}
}
bool MergeTreeData::isPrimaryKeyColumn(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 (!isPrimaryKeyColumn(item))
/// The tuple itself may be part of the primary key, so check that as a last resort.
return isPrimaryKeyColumn(left_in_operand);
/// 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.empty();
}
else
{
return isPrimaryKeyColumn(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 isPrimaryKeyColumn(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;