未验证 提交 f58d470d 编写于 作者: A Anton Popov 提交者: GitHub

Merge pull request #21830 from foolchi/optimize_select_with_final

optimize select final with prewhere primary key
......@@ -393,7 +393,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
view = nullptr;
}
if (try_move_to_prewhere && storage && query.where() && !query.prewhere() && !query.final())
if (try_move_to_prewhere && storage && query.where() && !query.prewhere())
{
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty())
......
......@@ -37,6 +37,8 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
: table_columns{ext::map<std::unordered_set>(
metadata_snapshot->getColumns().getAllPhysical(), [](const NameAndTypePair & col) { return col.name; })}
, queried_columns{queried_columns_}
, sorting_key_names{NameSet(
metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())}
, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}
, log{log_}
, column_sizes{std::move(column_sizes_)}
......@@ -114,12 +116,12 @@ static bool isConditionGood(const ASTPtr & condition)
}
void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node) const
void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node, bool is_final) const
{
if (const auto * func_and = node->as<ASTFunction>(); func_and && func_and->name == "and")
{
for (const auto & elem : func_and->arguments->children)
analyzeImpl(res, elem);
analyzeImpl(res, elem, is_final);
}
else
{
......@@ -133,7 +135,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node)
cond.viable =
/// Condition depend on some column. Constant expressions are not moved.
!cond.identifiers.empty()
&& !cannotBeMoved(node)
&& !cannotBeMoved(node, is_final)
/// Do not take into consideration the conditions consisting only of the first primary key column
&& !hasPrimaryKeyAtoms(node)
/// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded.
......@@ -149,10 +151,10 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node)
}
/// Transform conjunctions chain in WHERE expression to Conditions list.
MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression) const
MergeTreeWhereOptimizer::Conditions MergeTreeWhereOptimizer::analyze(const ASTPtr & expression, bool is_final) const
{
Conditions res;
analyzeImpl(res, expression);
analyzeImpl(res, expression, is_final);
return res;
}
......@@ -183,7 +185,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const
if (!select.where() || select.prewhere())
return;
Conditions where_conditions = analyze(select.where());
Conditions where_conditions = analyze(select.where(), select.final());
Conditions prewhere_conditions;
UInt64 total_size_of_moved_conditions = 0;
......@@ -300,6 +302,12 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const ASTPtr & ast) const
}
bool MergeTreeWhereOptimizer::isSortingKey(const String & column_name) const
{
return sorting_key_names.count(column_name);
}
bool MergeTreeWhereOptimizer::isConstant(const ASTPtr & expr) const
{
const auto column_name = expr->getColumnName();
......@@ -319,7 +327,7 @@ bool MergeTreeWhereOptimizer::isSubsetOfTableColumns(const NameSet & identifiers
}
bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const
bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) const
{
if (const auto * function_ptr = ptr->as<ASTFunction>())
{
......@@ -336,12 +344,13 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const
{
/// disallow moving result of ARRAY JOIN to PREWHERE
if (array_joined_names.count(*opt_name) ||
array_joined_names.count(Nested::extractTableName(*opt_name)))
array_joined_names.count(Nested::extractTableName(*opt_name)) ||
(is_final && !isSortingKey(*opt_name)))
return true;
}
for (const auto & child : ptr->children)
if (cannotBeMoved(child))
if (cannotBeMoved(child, is_final))
return true;
return false;
......
......@@ -67,10 +67,10 @@ private:
using Conditions = std::list<Condition>;
void analyzeImpl(Conditions & res, const ASTPtr & node) const;
void analyzeImpl(Conditions & res, const ASTPtr & node, bool is_final) const;
/// Transform conjunctions chain in WHERE expression to Conditions list.
Conditions analyze(const ASTPtr & expression) const;
Conditions analyze(const ASTPtr & expression, bool is_final) const;
/// Transform Conditions list to WHERE or PREWHERE expression.
static ASTPtr reconstruct(const Conditions & conditions);
......@@ -85,6 +85,8 @@ private:
bool isPrimaryKeyAtom(const ASTPtr & ast) const;
bool isSortingKey(const String & column_name) const;
bool isConstant(const ASTPtr & expr) const;
bool isSubsetOfTableColumns(const NameSet & identifiers) const;
......@@ -95,7 +97,7 @@ private:
*
* Also, disallow moving expressions with GLOBAL [NOT] IN.
*/
bool cannotBeMoved(const ASTPtr & ptr) const;
bool cannotBeMoved(const ASTPtr & ptr, bool is_final) const;
void determineArrayJoinedNames(ASTSelectQuery & select);
......@@ -104,6 +106,7 @@ private:
String first_primary_key_column;
const StringSet table_columns;
const Names queried_columns;
const NameSet sorting_key_names;
const Block block_with_constants;
Poco::Logger * log;
std::unordered_map<std::string, UInt64> column_sizes;
......
SELECT
x,
y,
z
FROM prewhere_move_select_final
PREWHERE y > 100
SELECT
x,
y,
z
FROM prewhere_move_select_final
FINAL
PREWHERE y > 100
SELECT
x,
y,
z
FROM prewhere_move_select_final
FINAL
WHERE z > 400
SELECT
x,
y,
z
FROM prewhere_move_select_final
FINAL
PREWHERE y > 100
WHERE (y > 100) AND (z > 400)
DROP TABLE IF EXISTS prewhere_move_select_final;
CREATE TABLE prewhere_move_select_final (x Int, y Int, z Int) ENGINE = ReplacingMergeTree() ORDER BY (x, y);
INSERT INTO prewhere_move_select_final SELECT number, number * 2, number * 3 FROM numbers(1000);
-- order key can be pushed down with final
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final WHERE y > 100;
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100;
-- can not be pushed down
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE z > 400;
-- only y can be pushed down
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100 and z > 400;
DROP TABLE prewhere_move_select_final;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册