diff --git a/src/Interpreters/DuplicateOrderByVisitor.cpp b/src/Interpreters/DuplicateOrderByVisitor.cpp new file mode 100644 index 00000000000..df063fc849e --- /dev/null +++ b/src/Interpreters/DuplicateOrderByVisitor.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE_OF_AST_NODE; +} + + +namespace +{ + +/// Checks if SELECT has stateful functions +class ASTFunctionStatefulData +{ +public: + using TypeToVisit = ASTFunction; + + ContextPtr context; + bool & is_stateful; + void visit(ASTFunction & ast_function, ASTPtr &) + { + auto aggregate_function_properties = AggregateFunctionFactory::instance().tryGetProperties(ast_function.name); + + if (aggregate_function_properties && aggregate_function_properties->is_order_dependent) + { + is_stateful = true; + return; + } + + const auto & function = FunctionFactory::instance().tryGet(ast_function.name, context); + + if (function && function->isStateful()) + { + is_stateful = true; + return; + } + } +}; + +using ASTFunctionStatefulMatcher = OneTypeMatcher; +using ASTFunctionStatefulVisitor = InDepthNodeVisitor; + +} + + +void DuplicateOrderByFromSubqueriesData::visit(ASTSelectQuery & select_query, ASTPtr &) +{ + if (done) + return; + done = true; + + if (select_query.orderBy()) + { + /// If we have limits then the ORDER BY is non-removable. + if (select_query.limitBy() + || select_query.limitByOffset() + || select_query.limitByLength() + || select_query.limitLength() + || select_query.limitOffset()) + { + return; + } + + /// If ORDER BY contains filling (in addition to sorting) it is non-removable. + for (const auto & child : select_query.orderBy()->children) + { + auto * ast = child->as(); + if (!ast || ast->children.empty()) + throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + + if (ast->with_fill) + return; + } + + select_query.setExpression(ASTSelectQuery::Expression::ORDER_BY, nullptr); + } +} + +void DuplicateOrderByData::visit(ASTSelectQuery & select_query, ASTPtr &) +{ + if (select_query.orderBy() || select_query.groupBy()) + { + for (auto & elem : select_query.children) + { + if (elem->as()) + { + bool is_stateful = false; + ASTFunctionStatefulVisitor::Data data{context, is_stateful}; + ASTFunctionStatefulVisitor(data).visit(elem); + if (is_stateful) //-V547 + return; + } + } + + if (auto select_table_ptr = select_query.tables()) + { + if (auto * select_table = select_table_ptr->as()) + { + if (!select_table->children.empty()) + { + DuplicateOrderByFromSubqueriesVisitor::Data data{false}; + DuplicateOrderByFromSubqueriesVisitor(data).visit(select_table->children[0]); + } + } + } + } +} + +} + diff --git a/src/Interpreters/DuplicateOrderByVisitor.h b/src/Interpreters/DuplicateOrderByVisitor.h index 4231b2600af..de8cb4c8f32 100644 --- a/src/Interpreters/DuplicateOrderByVisitor.h +++ b/src/Interpreters/DuplicateOrderByVisitor.h @@ -1,51 +1,13 @@ #pragma once -#include -#include -#include #include -#include -#include -#include -#include -#include #include -#include + namespace DB { -/// Checks if SELECT has stateful functions -class ASTFunctionStatefulData -{ -public: - using TypeToVisit = ASTFunction; - - ContextPtr context; - bool & is_stateful; - void visit(ASTFunction & ast_function, ASTPtr &) - { - auto aggregate_function_properties = AggregateFunctionFactory::instance().tryGetProperties(ast_function.name); - - if (aggregate_function_properties && aggregate_function_properties->is_order_dependent) - { - is_stateful = true; - return; - } - - const auto & function = FunctionFactory::instance().tryGet(ast_function.name, context); - - if (function && function->isStateful()) - { - is_stateful = true; - return; - } - } -}; - -using ASTFunctionStatefulMatcher = OneTypeMatcher; -using ASTFunctionStatefulVisitor = InDepthNodeVisitor; - +class ASTSelectQuery; /// Erases unnecessary ORDER BY from subquery class DuplicateOrderByFromSubqueriesData @@ -55,19 +17,7 @@ public: bool done = false; - void visit(ASTSelectQuery & select_query, ASTPtr &) - { - if (done) - return; - - if (select_query.orderBy() && !select_query.limitBy() && !select_query.limitByOffset() && - !select_query.limitByLength() && !select_query.limitLength() && !select_query.limitOffset()) - { - select_query.setExpression(ASTSelectQuery::Expression::ORDER_BY, nullptr); - } - - done = true; - } + void visit(ASTSelectQuery & select_query, ASTPtr &); }; using DuplicateOrderByFromSubqueriesMatcher = OneTypeMatcher; @@ -82,35 +32,7 @@ public: ContextPtr context; - void visit(ASTSelectQuery & select_query, ASTPtr &) - { - if (select_query.orderBy() || select_query.groupBy()) - { - for (auto & elem : select_query.children) - { - if (elem->as()) - { - bool is_stateful = false; - ASTFunctionStatefulVisitor::Data data{context, is_stateful}; - ASTFunctionStatefulVisitor(data).visit(elem); - if (is_stateful) //-V547 - return; - } - } - - if (auto select_table_ptr = select_query.tables()) - { - if (auto * select_table = select_table_ptr->as()) - { - if (!select_table->children.empty()) - { - DuplicateOrderByFromSubqueriesVisitor::Data data{false}; - DuplicateOrderByFromSubqueriesVisitor(data).visit(select_table->children[0]); - } - } - } - } - } + void visit(ASTSelectQuery & select_query, ASTPtr &); }; using DuplicateOrderByMatcher = OneTypeMatcher; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 318f392c2a5..1dd63695ad4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1240,7 +1240,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { auto * ast = child->as(); if (!ast || ast->children.empty()) - throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); if (getContext()->getSettingsRef().enable_positional_arguments) { diff --git a/src/Interpreters/FillingRow.h b/src/Interpreters/FillingRow.h index 434a9270718..604f4b1ee74 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -24,14 +24,14 @@ public: void initFromDefaults(size_t from_pos = 0); - Field & operator[](size_t ind) { return row[ind]; } - const Field & operator[](size_t ind) const { return row[ind]; } + Field & operator[](size_t index) { return row[index]; } + const Field & operator[](size_t index) const { return row[index]; } size_t size() const { return row.size(); } bool operator<(const FillingRow & other) const; bool operator==(const FillingRow & other) const; - int getDirection(size_t ind) const { return description[ind].direction; } - FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } + int getDirection(size_t index) const { return description[index].direction; } + FillColumnDescription & getFillDescription(size_t index) { return description[index].fill_description; } private: Row row; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 518c041d785..833d76a26b8 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -39,6 +39,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TYPE_OF_AST_NODE; } namespace @@ -282,7 +283,8 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query) String name = elem->children.front()->getColumnName(); const auto & order_by_elem = elem->as(); - if (elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second) + if (order_by_elem.with_fill /// Always keep elements WITH FILL as they affects other. + || elems_set.emplace(name, order_by_elem.collation ? order_by_elem.collation->getColumnName() : "").second) unique_elems.emplace_back(elem); } @@ -425,6 +427,17 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context if (!order_by) return; + for (const auto & child : order_by->children) + { + auto * order_by_element = child->as(); + + if (!order_by_element || order_by_element->children.empty()) + throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + + if (order_by_element->with_fill) + return; + } + std::unordered_set group_by_hashes; if (auto group_by = select_query->groupBy()) { @@ -440,6 +453,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context for (size_t i = 0; i < order_by->children.size(); ++i) { auto * order_by_element = order_by->children[i]->as(); + auto & ast_func = order_by_element->children[0]; if (!ast_func->as()) continue; @@ -475,6 +489,17 @@ void optimizeRedundantFunctionsInOrderBy(const ASTSelectQuery * select_query, Co if (!order_by) return; + for (const auto & child : order_by->children) + { + auto * order_by_element = child->as(); + + if (!order_by_element || order_by_element->children.empty()) + throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + + if (order_by_element->with_fill) + return; + } + std::unordered_set prev_keys; ASTs modified; modified.reserve(order_by->children.size()); diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 831130d06d1..3ff89c302ff 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -81,7 +81,7 @@ FillingTransform::FillingTransform( }; std::vector is_fill_column(header_.columns()); - for (size_t i = 0; i < sort_description.size(); ++i) + for (size_t i = 0, size = sort_description.size(); i < size; ++i) { size_t block_position = header_.getPositionByName(sort_description[i].column_name); is_fill_column[block_position] = true; @@ -103,6 +103,11 @@ FillingTransform::FillingTransform( } } + std::set unique_positions; + for (auto pos : fill_column_positions) + if (!unique_positions.insert(pos).second) + throw Exception("Multiple WITH FILL for identical expressions is not supported in ORDER BY", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + for (size_t i = 0; i < header_.columns(); ++i) if (!is_fill_column[i]) other_column_positions.push_back(i); @@ -114,7 +119,7 @@ IProcessor::Status FillingTransform::prepare() { should_insert_first = next_row < filling_row; - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0, size = filling_row.size(); i < size; ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; if (filling_row < next_row) @@ -227,9 +232,9 @@ void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_col /// fill_columns always non-empty. size_t num_rows = fill_columns[0]->size(); - for (size_t i = 0; i < fill_columns.size(); ++i) + for (size_t i = 0, size = fill_columns.size(); i < size; ++i) result_columns[fill_column_positions[i]] = std::move(fill_columns[i]); - for (size_t i = 0; i < other_columns.size(); ++i) + for (size_t i = 0, size = other_columns.size(); i < size; ++i) result_columns[other_column_positions[i]] = std::move(other_columns[i]); chunk.setColumns(std::move(result_columns), num_rows); diff --git a/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.reference b/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.reference new file mode 100644 index 00000000000..07258cd829a --- /dev/null +++ b/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.reference @@ -0,0 +1,9 @@ + + + + + + + + +Hello diff --git a/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.sql b/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.sql new file mode 100644 index 00000000000..f0d90f151b2 --- /dev/null +++ b/tests/queries/0_stateless/2015_order_by_with_fill_misoptimization.sql @@ -0,0 +1 @@ +SELECT s FROM (SELECT 5 AS x, 'Hello' AS s ORDER BY x WITH FILL FROM 1 TO 10) ORDER BY s; diff --git a/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.reference b/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.reference new file mode 100644 index 00000000000..264f29a6ecd --- /dev/null +++ b/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.reference @@ -0,0 +1,3 @@ +2021-07-07 15:21:00 +2021-07-07 15:21:05 +2021-07-07 15:21:10 diff --git a/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.sql b/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.sql new file mode 100644 index 00000000000..bf232ed5c86 --- /dev/null +++ b/tests/queries/0_stateless/2016_order_by_with_fill_monotonic_functions_removal.sql @@ -0,0 +1,6 @@ +SELECT toStartOfMinute(some_time) AS ts +FROM +( + SELECT toDateTime('2021-07-07 15:21:05') AS some_time +) +ORDER BY ts ASC WITH FILL FROM toDateTime('2021-07-07 15:21:00') TO toDateTime('2021-07-07 15:21:15') STEP 5; diff --git a/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.reference b/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.reference new file mode 100644 index 00000000000..07193989308 --- /dev/null +++ b/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.reference @@ -0,0 +1,9 @@ +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.sql b/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.sql new file mode 100644 index 00000000000..6f3e6787c34 --- /dev/null +++ b/tests/queries/0_stateless/2017_order_by_with_fill_redundant_functions.sql @@ -0,0 +1 @@ +SELECT x FROM (SELECT 5 AS x) ORDER BY -x, x WITH FILL FROM 1 TO 10; diff --git a/tests/queries/0_stateless/2018_multiple_with_fill_for_the_same_column.reference b/tests/queries/0_stateless/2018_multiple_with_fill_for_the_same_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/2018_multiple_with_fill_for_the_same_column.sql b/tests/queries/0_stateless/2018_multiple_with_fill_for_the_same_column.sql new file mode 100644 index 00000000000..32b38388cf6 --- /dev/null +++ b/tests/queries/0_stateless/2018_multiple_with_fill_for_the_same_column.sql @@ -0,0 +1 @@ +SELECT x, y FROM (SELECT 5 AS x, 'Hello' AS y) ORDER BY x WITH FILL FROM 3 TO 7, y, x WITH FILL FROM 1 TO 10; -- { serverError 475 } diff --git a/tests/queries/0_stateless/2019_multiple_weird_with_fill.reference b/tests/queries/0_stateless/2019_multiple_weird_with_fill.reference new file mode 100644 index 00000000000..822d290564a --- /dev/null +++ b/tests/queries/0_stateless/2019_multiple_weird_with_fill.reference @@ -0,0 +1,45 @@ +3 -10 +3 -9 +3 -8 +3 -7 +3 -6 +3 -5 +3 -4 +3 -3 +3 -2 +4 -10 +4 -9 +4 -8 +4 -7 +4 -6 +4 -5 +4 -4 +4 -3 +4 -2 +5 -10 +5 -9 +5 -8 +5 -7 +5 -6 +5 -5 Hello +5 -4 +5 -3 +5 -2 +6 -10 +6 -9 +6 -8 +6 -7 +6 -6 +6 -5 +6 -4 +6 -3 +6 -2 +7 -10 +7 -9 +7 -8 +7 -7 +7 -6 +7 -5 +7 -4 +7 -3 +7 -2 diff --git a/tests/queries/0_stateless/2019_multiple_weird_with_fill.sql b/tests/queries/0_stateless/2019_multiple_weird_with_fill.sql new file mode 100644 index 00000000000..a2ed33c51dd --- /dev/null +++ b/tests/queries/0_stateless/2019_multiple_weird_with_fill.sql @@ -0,0 +1,14 @@ +SELECT + x, + -x, + y +FROM +( + SELECT + 5 AS x, + 'Hello' AS y +) +ORDER BY + x ASC WITH FILL FROM 3 TO 7, + y ASC, + -x ASC WITH FILL FROM -10 TO -1;