Revert "Replace ORDER BY ALL by ORDER BY *"

This commit is contained in:
Robert Schulze 2024-02-21 19:05:20 +01:00 committed by GitHub
parent 613bb8f387
commit 52afa46e23
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 211 additions and 70 deletions

View File

@ -4279,6 +4279,41 @@ Result:
└─────┴─────┴───────┘
```
## enable_order_by_all {#enable-order-by-all}
Enables or disables sorting by `ALL` columns, i.e. [ORDER BY](../../sql-reference/statements/select/order-by.md)
Possible values:
- 0 — Disable ORDER BY ALL.
- 1 — Enable ORDER BY ALL.
Default value: `1`.
**Example**
Query:
```sql
CREATE TABLE TAB(C1 Int, C2 Int, ALL Int) ENGINE=Memory();
INSERT INTO TAB VALUES (10, 20, 30), (20, 20, 10), (30, 10, 20);
SELECT * FROM TAB ORDER BY ALL; -- returns an error that ALL is ambiguous
SELECT * FROM TAB ORDER BY ALL SETTINGS enable_order_by_all;
```
Result:
```text
┌─C1─┬─C2─┬─ALL─┐
│ 20 │ 20 │ 10 │
│ 30 │ 10 │ 20 │
│ 10 │ 20 │ 30 │
└────┴────┴─────┘
```
## splitby_max_substrings_includes_remaining_string {#splitby_max_substrings_includes_remaining_string}
Controls whether function [splitBy*()](../../sql-reference/functions/splitting-merging-functions.md) with argument `max_substrings` > 0 will include the remaining string in the last element of the result array.

View File

@ -9,9 +9,10 @@ The `ORDER BY` clause contains
- a list of expressions, e.g. `ORDER BY visits, search_phrase`,
- a list of numbers referring to columns in the `SELECT` clause, e.g. `ORDER BY 2, 1`, or
- `*` (without other expressions or numbers) which means all columns of the `SELECT` clause: `ORDER BY *`.
- `ALL` which means all columns of the `SELECT` clause, e.g. `ORDER BY ALL`.
To disable sorting by column numbers, set setting [enable_positional_arguments](../../../operations/settings/settings.md#enable-positional-arguments) = 0.
To disable sorting by `ALL`, set setting [enable_order_by_all](../../../operations/settings/settings.md#enable-order-by-all) = 0.
The `ORDER BY` clause can be attributed by a `DESC` (descending) or `ASC` (ascending) modifier which determines the sorting direction.
Unless an explicit sort order is specified, `ASC` is used by default.

View File

@ -61,14 +61,14 @@ sidebar_label: ORDER BY
我们只建议使用 `COLLATE` 对于少量行的最终排序,因为排序与 `COLLATE` 比正常的按字节排序效率低。
## ORDER BY *
## ORDER BY ALL
`ORDER BY *` 对所有选定的列进行升序排序。
`ORDER BY ALL` 对所有选定的列进行升序排序。
示例:
``` sql
SELECT a, b, c FROM t ORDER BY *
SELECT a, b, c FROM t ORDER BY ALL
```
等同于:

View File

@ -120,6 +120,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS;
extern const int SYNTAX_ERROR;
extern const int UNEXPECTED_EXPRESSION;
extern const int INVALID_IDENTIFIER;
}
@ -1214,7 +1215,7 @@ private:
static void expandGroupByAll(QueryNode & query_tree_node_typed);
void expandOrderByAll(QueryNode & query_tree_node_typed);
void expandOrderByAll(QueryNode & query_tree_node_typed, const Settings & settings);
static std::string
rewriteAggregateFunctionNameIfNeeded(const std::string & aggregate_function_name, NullsAction action, const ContextPtr & context);
@ -2366,9 +2367,9 @@ void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed)
query_tree_node_typed.setIsGroupByAll(false);
}
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed, const Settings & settings)
{
if (!query_tree_node_typed.isOrderByAll())
if (!settings.enable_order_by_all || !query_tree_node_typed.isOrderByAll())
return;
auto * all_node = query_tree_node_typed.getOrderBy().getNodes()[0]->as<SortNode>();
@ -2389,6 +2390,9 @@ void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expression nodes list expected 1 projection names. Actual {}",
projection_names.size());
if (Poco::toUpper(projection_names[0]) == "ALL")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
}
auto sort_node = std::make_shared<SortNode>(node, all_node->getSortDirection(), all_node->getNullsSortDirection());
@ -7555,7 +7559,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
if (settings.enable_positional_arguments)
replaceNodesWithPositionalArguments(query_node_typed.getOrderByNode(), query_node_typed.getProjection().getNodes(), scope);
expandOrderByAll(query_node_typed);
expandOrderByAll(query_node_typed, settings);
resolveSortNodeList(query_node_typed.getOrderByNode(), scope);
}

View File

@ -219,13 +219,13 @@ public:
is_group_by_all = is_group_by_all_value;
}
/// Returns true, if query node has ORDER BY * modifier, false otherwise
/// Returns true, if query node has ORDER BY ALL modifier, false otherwise
bool isOrderByAll() const
{
return is_order_by_all;
}
/// Set query node ORDER BY * modifier value
/// Set query node ORDER BY ALL modifier value
void setIsOrderByAll(bool is_order_by_all_value)
{
is_order_by_all = is_order_by_all_value;

View File

@ -872,6 +872,7 @@ class IColumn;
M(UInt64, cache_warmer_threads, 4, "Only available in ClickHouse Cloud", 0) \
M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud", 0) \
M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud", 0) \
M(Bool, enable_order_by_all, true, "Enable sorting expression ORDER BY ALL.", 0) \
M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \
// End of COMMON_SETTINGS
@ -939,7 +940,6 @@ class IColumn;
MAKE_OBSOLETE(M, Bool, allow_experimental_undrop_table_query, true) \
MAKE_OBSOLETE(M, Bool, allow_experimental_s3queue, true) \
MAKE_OBSOLETE(M, Bool, query_plan_optimize_primary_key, true) \
MAKE_OBSOLETE(M, Bool, enable_order_by_all, true) \
/** The section above is for obsolete settings. Do not add anything there. */

View File

@ -73,6 +73,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNKNOWN_IDENTIFIER;
extern const int UNEXPECTED_EXPRESSION;
}
namespace
@ -785,6 +786,16 @@ void expandOrderByAll(ASTSelectQuery * select_query)
for (const auto & expr : select_query->select()->children)
{
if (auto * identifier = expr->as<ASTIdentifier>(); identifier != nullptr)
if (Poco::toUpper(identifier->name()) == "ALL" || Poco::toUpper(identifier->alias) == "ALL")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
if (auto * function = expr->as<ASTFunction>(); function != nullptr)
if (Poco::toUpper(function->alias) == "ALL")
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION,
"Cannot use ORDER BY ALL to sort a column with name 'all', please disable setting `enable_order_by_all` and try again");
auto elem = std::make_shared<ASTOrderByElement>();
elem->direction = all_elem->direction;
elem->nulls_direction = all_elem->nulls_direction;
@ -1311,8 +1322,8 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect(
if (select_query->group_by_all)
expandGroupByAll(select_query);
// expand ORDER BY *
if (select_query->order_by_all)
// expand ORDER BY ALL
if (settings.enable_order_by_all && select_query->order_by_all)
expandOrderByAll(select_query);
/// Remove unneeded columns according to 'required_result_columns'.

View File

@ -165,7 +165,7 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
if (order_by_all)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY *" << (s.hilite ? hilite_none : "");
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "ORDER BY ALL" << (s.hilite ? hilite_none : "");
auto * elem = orderBy()->children[0]->as<ASTOrderByElement>();
s.ostr << (s.hilite ? hilite_keyword : "")

View File

@ -1,23 +1,21 @@
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTInterpolateElement.h>
#include <memory>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/IParserBase.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/IParserBase.h>
#include <Parsers/ParserSampleRatio.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserSampleRatio.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/ParserWithElement.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTInterpolateElement.h>
#include <Parsers/ASTIdentifier.h>
#include <Poco/String.h>
#include <memory>
namespace DB
{
@ -292,9 +290,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else if (order_expression_list->children.size() == 1)
{
/// ORDER BY *
auto * asterisk = order_expression_list->children[0]->as<ASTOrderByElement>()->children[0]->as<ASTAsterisk>();
if (asterisk != nullptr)
/// ORDER BY ALL
auto * identifier = order_expression_list->children[0]->as<ASTOrderByElement>()->children[0]->as<ASTIdentifier>();
if (identifier != nullptr && Poco::toUpper(identifier->name()) == "ALL")
select_query->order_by_all = true;
}
}

View File

@ -288,7 +288,7 @@ def test_replicated_merge_tree(cluster, test_case):
WHERE
local_path LIKE '%{uuid}%'
AND local_path NOT LIKE '%format_version.txt%'
ORDER BY *
ORDER BY ALL
"""
).strip()
@ -329,7 +329,7 @@ def test_replicated_merge_tree(cluster, test_case):
WHERE
table = 'test_replicated_merge_tree'
AND active
ORDER BY *
ORDER BY ALL
"""
)
.strip()
@ -349,7 +349,7 @@ def test_replicated_merge_tree(cluster, test_case):
SELECT name
FROM system.zookeeper
WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}'
ORDER BY *
ORDER BY ALL
"""
)
.strip()
@ -363,7 +363,7 @@ def test_replicated_merge_tree(cluster, test_case):
SELECT name
FROM system.zookeeper
WHERE path='/clickhouse/zero_copy/zero_copy_s3/{table_shared_uuid}/{part}/{blob}'
ORDER BY *
ORDER BY ALL
"""
)
.strip()

View File

@ -5,7 +5,7 @@ FROM
)
GROUP BY number
HAVING 1 AND sin(sum(number))
ORDER BY *
ORDER BY ALL
SETTINGS enable_optimize_predicate_expression = 0;
SELECT '=====';
@ -17,7 +17,7 @@ FROM
)
GROUP BY number
HAVING 1 AND sin(1)
ORDER BY *
ORDER BY ALL
SETTINGS enable_optimize_predicate_expression = 0;
SELECT '=====';
@ -29,7 +29,7 @@ FROM
)
GROUP BY number
HAVING x AND sin(sum(number))
ORDER BY *
ORDER BY ALL
SETTINGS enable_optimize_predicate_expression = 1;
SELECT '=====';
@ -41,7 +41,7 @@ FROM
)
GROUP BY number
HAVING 1 AND sin(sum(number))
ORDER BY *
ORDER BY ALL
SETTINGS enable_optimize_predicate_expression = 0;
SELECT '=====';
@ -61,7 +61,7 @@ FROM
)
GROUP BY number
HAVING 1 AND sin(sum(number))
ORDER BY *
ORDER BY ALL
SETTINGS enable_optimize_predicate_expression = 1;
select '#45440';

View File

@ -29,13 +29,13 @@ CREATE TABLE t
INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse');
SELECT '-- non-const arguments';
SELECT 'byteHammingDistance', s1, s2, byteHammingDistance(s1, s2) FROM t ORDER BY *;
SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY *;
SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY *;
SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY *;
SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY *;
SELECT 'jaroSimilarity', s1, s2, jaroSimilarity(s1, s2) FROM t ORDER BY *;
SELECT 'jaroWinklerSimilarity', s1, s2, jaroWinklerSimilarity(s1, s2) FROM t ORDER BY *;
SELECT 'byteHammingDistance', s1, s2, byteHammingDistance(s1, s2) FROM t ORDER BY ALL;
SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY ALL;
SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY ALL;
SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY ALL;
SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY ALL;
SELECT 'jaroSimilarity', s1, s2, jaroSimilarity(s1, s2) FROM t ORDER BY ALL;
SELECT 'jaroWinklerSimilarity', s1, s2, jaroWinklerSimilarity(s1, s2) FROM t ORDER BY ALL;
SELECT '-- Special UTF-8 tests';
-- We do not perform full UTF8 validation, so sometimes it just returns some result

View File

@ -49,9 +49,45 @@ A 2
2 A
3 B
\N C
-- Special case: all columns in SELECT clause, ORDER BY *
-- what happens if some column "all" already exists?
B 3 10
D 1 20
A 2 30
C \N 40
B 3 10
D 1 20
A 2 30
C \N 40
D 1
A 2
B 3
C \N
D 1
-- "*" must appear stand-alone in ORDER BY
A 2
B 3
C \N
A 2
B 3
D 1
\N
A 2
B 3
D 1
\N
B 3 10
D 1 20
A 2 30
C \N 40
B 3 10
D 1 20
A 2 30
C \N 40
-- test SELECT * ORDER BY ALL with no "all" column in the SELECT clause
A 2 30
B 3 10
C \N 40
D 1 20
A 2 30
B 3 10
C \N 40
D 1 20

View File

@ -1,4 +1,4 @@
-- Tests that sort expression ORDER BY *
-- Tests that sort expression ORDER BY ALL
DROP TABLE IF EXISTS order_by_all;
@ -6,48 +6,104 @@ CREATE TABLE order_by_all
(
a String,
b Nullable(Int32),
all UInt64,
)
ENGINE = Memory;
INSERT INTO order_by_all VALUES ('B', 3), ('C', NULL), ('D', 1), ('A', 2);
INSERT INTO order_by_all VALUES ('B', 3, 10), ('C', NULL, 40), ('D', 1, 20), ('A', 2, 30);
SELECT '-- no modifiers';
SET allow_experimental_analyzer = 0;
SELECT a, b FROM order_by_all ORDER BY *;
SELECT b, a FROM order_by_all ORDER BY *;
SELECT a, b FROM order_by_all ORDER BY ALL;
SELECT b, a FROM order_by_all ORDER BY ALL;
SET allow_experimental_analyzer = 1;
SELECT a, b FROM order_by_all ORDER BY *;
SELECT b, a FROM order_by_all ORDER BY *;
SELECT a, b FROM order_by_all ORDER BY ALL;
SELECT b, a FROM order_by_all ORDER BY ALL;
SELECT '-- with ASC/DESC modifiers';
SET allow_experimental_analyzer = 0;
SELECT a, b FROM order_by_all ORDER BY * ASC;
SELECT a, b FROM order_by_all ORDER BY * DESC;
SELECT a, b FROM order_by_all ORDER BY ALL ASC;
SELECT a, b FROM order_by_all ORDER BY ALL DESC;
SET allow_experimental_analyzer = 1;
SELECT a, b FROM order_by_all ORDER BY * ASC;
SELECT a, b FROM order_by_all ORDER BY * DESC;
SELECT a, b FROM order_by_all ORDER BY ALL ASC;
SELECT a, b FROM order_by_all ORDER BY ALL DESC;
SELECT '-- with NULLS FIRST/LAST modifiers';
SET allow_experimental_analyzer = 0;
SELECT b, a FROM order_by_all ORDER BY * NULLS FIRST;
SELECT b, a FROM order_by_all ORDER BY * NULLS LAST;
SELECT b, a FROM order_by_all ORDER BY ALL NULLS FIRST;
SELECT b, a FROM order_by_all ORDER BY ALL NULLS LAST;
SET allow_experimental_analyzer = 1;
SELECT b, a FROM order_by_all ORDER BY * NULLS FIRST;
SELECT b, a FROM order_by_all ORDER BY * NULLS LAST;
SELECT b, a FROM order_by_all ORDER BY ALL NULLS FIRST;
SELECT b, a FROM order_by_all ORDER BY ALL NULLS LAST;
SELECT '-- Special case: all columns in SELECT clause, ORDER BY *';
SELECT * FROM order_by_all ORDER BY * NULLS LAST;
SELECT '-- what happens if some column "all" already exists?';
SELECT '-- "*" must appear stand-alone in ORDER BY';
-- columns
SET allow_experimental_analyzer = 0;
SELECT a, b FROM order_by_all ORDER BY *, a; -- { serverError UNKNOWN_IDENTIFIER }
SELECT a, b, all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b, all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
SET allow_experimental_analyzer = 1;
SELECT a, b FROM order_by_all ORDER BY *, a; -- { serverError UNSUPPORTED_METHOD }
SELECT a, b, all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b, all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b, all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
-- column aliases
SET allow_experimental_analyzer = 0;
SELECT a, b AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b AS all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
SET allow_experimental_analyzer = 1;
SELECT a, b AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b AS all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT a, b AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
-- expressions
SET allow_experimental_analyzer = 0;
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
SET allow_experimental_analyzer = 1;
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all; -- { serverError UNEXPECTED_EXPRESSION }
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY ALL; -- { serverError UNEXPECTED_EXPRESSION }
SELECT format('{} {}', a, b) AS all FROM order_by_all ORDER BY all SETTINGS enable_order_by_all = false;
SET allow_experimental_analyzer = 0;
SELECT a, b, all FROM order_by_all ORDER BY all, a;
SET allow_experimental_analyzer = 1;
SELECT a, b, all FROM order_by_all ORDER BY all, a;
DROP TABLE order_by_all;
SELECT '-- test SELECT * ORDER BY ALL with no "all" column in the SELECT clause';
CREATE TABLE order_by_all
(
a String,
b Nullable(Int32),
c UInt64,
)
ENGINE = Memory;
INSERT INTO order_by_all VALUES ('B', 3, 10), ('C', NULL, 40), ('D', 1, 20), ('A', 2, 30);
SET allow_experimental_analyzer = 0;
SELECT * FROM order_by_all ORDER BY ALL;
SET allow_experimental_analyzer = 1;
SELECT * FROM order_by_all ORDER BY ALL;
DROP TABLE order_by_all;

View File

@ -11,23 +11,23 @@ INSERT INTO r VALUES (NULL, NULL);
SET allow_experimental_analyzer = 0;
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
;
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
SETTINGS join_algorithm = 'partial_merge';
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
SETTINGS join_algorithm = 'full_sorting_merge';
SET allow_experimental_analyzer = 1;
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
;
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
SETTINGS join_algorithm = 'partial_merge';
SELECT x FROM t FULL JOIN r USING (x) ORDER BY *
SELECT x FROM t FULL JOIN r USING (x) ORDER BY ALL
SETTINGS join_algorithm = 'full_sorting_merge';