mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 02:21:59 +00:00
Merge 8953babcd0
into e0f8b8d351
This commit is contained in:
commit
77f49aa2fe
@ -1091,7 +1091,7 @@ Default value: 0 bytes.
|
|||||||
|
|
||||||
Note that if both `min_free_disk_bytes_to_perform_insert` and `min_free_disk_ratio_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory.
|
Note that if both `min_free_disk_bytes_to_perform_insert` and `min_free_disk_ratio_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory.
|
||||||
|
|
||||||
## min_free_disk_ratio_to_perform_insert
|
## min_free_disk_ratio_to_perform_insert
|
||||||
|
|
||||||
The minimum free to total disk space ratio to perform an `INSERT`. Must be a floating point value between 0 and 1. Note that this setting:
|
The minimum free to total disk space ratio to perform an `INSERT`. Must be a floating point value between 0 and 1. Note that this setting:
|
||||||
- takes into account the `keep_free_space_bytes` setting.
|
- takes into account the `keep_free_space_bytes` setting.
|
||||||
@ -1106,6 +1106,31 @@ Default value: 0.0
|
|||||||
|
|
||||||
Note that if both `min_free_disk_ratio_to_perform_insert` and `min_free_disk_bytes_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory.
|
Note that if both `min_free_disk_ratio_to_perform_insert` and `min_free_disk_bytes_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory.
|
||||||
|
|
||||||
|
## allow_experimental_reverse_key
|
||||||
|
|
||||||
|
Enables support for descending sort order in MergeTree sorting keys. This setting is particularly useful for time series analysis and Top-N queries, allowing data to be stored in reverse chronological order to optimize query performance.
|
||||||
|
|
||||||
|
With `allow_experimental_reverse_key` enabled, you can define descending sort orders within the `ORDER BY` clause of a MergeTree table. This enables the use of more efficient `ReadInOrder` optimizations instead of `ReadInReverseOrder` for descending queries.
|
||||||
|
|
||||||
|
**Example**
|
||||||
|
|
||||||
|
```sql
|
||||||
|
CREATE TABLE example
|
||||||
|
(
|
||||||
|
time DateTime,
|
||||||
|
key Int32,
|
||||||
|
value String
|
||||||
|
) ENGINE = MergeTree
|
||||||
|
ORDER BY (time DESC, key) -- Descending order on 'time' field
|
||||||
|
SETTINGS allow_experimental_reverse_key = 1;
|
||||||
|
|
||||||
|
SELECT * FROM example WHERE key = 'xxx' ORDER BY time DESC LIMIT 10;
|
||||||
|
```
|
||||||
|
|
||||||
|
By using `ORDER BY time DESC` in the query, `ReadInOrder` is applied.
|
||||||
|
|
||||||
|
**Default Value:** false
|
||||||
|
|
||||||
## cache_populated_by_fetch
|
## cache_populated_by_fetch
|
||||||
|
|
||||||
A Cloud only setting.
|
A Cloud only setting.
|
||||||
@ -1114,4 +1139,4 @@ When `cache_populated_by_fetch` is disabled (the default setting), new data part
|
|||||||
|
|
||||||
If enabled, `cache_populated_by_fetch` will instead cause all nodes to load new data parts from storage into their cache without requiring a query to trigger such an action.
|
If enabled, `cache_populated_by_fetch` will instead cause all nodes to load new data parts from storage into their cache without requiring a query to trigger such an action.
|
||||||
|
|
||||||
Default value: 0.
|
Default value: 0.
|
||||||
|
@ -1449,6 +1449,7 @@ size_t MutationsInterpreter::evaluateCommandsSize()
|
|||||||
std::optional<SortDescription> MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const
|
std::optional<SortDescription> MutationsInterpreter::getStorageSortDescriptionIfPossible(const Block & header) const
|
||||||
{
|
{
|
||||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||||
|
std::vector<bool> reverse_flags = metadata_snapshot->getSortingKeyReverseFlags();
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
size_t sort_columns_size = sort_columns.size();
|
size_t sort_columns_size = sort_columns.size();
|
||||||
sort_description.reserve(sort_columns_size);
|
sort_description.reserve(sort_columns_size);
|
||||||
@ -1456,9 +1457,16 @@ std::optional<SortDescription> MutationsInterpreter::getStorageSortDescriptionIf
|
|||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
{
|
{
|
||||||
if (header.has(sort_columns[i]))
|
if (header.has(sort_columns[i]))
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
{
|
||||||
|
if (!reverse_flags.empty() && reverse_flags[i])
|
||||||
|
sort_description.emplace_back(sort_columns[i], -1, 1);
|
||||||
|
else
|
||||||
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
|
{
|
||||||
return {};
|
return {};
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return sort_description;
|
return sort_description;
|
||||||
|
@ -62,4 +62,18 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ASTStorageOrderByElement::updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const
|
||||||
|
{
|
||||||
|
hash_state.update(direction);
|
||||||
|
IAST::updateTreeHashImpl(hash_state, ignore_aliases);
|
||||||
|
}
|
||||||
|
|
||||||
|
void ASTStorageOrderByElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||||
|
{
|
||||||
|
children.front()->formatImpl(settings, state, frame);
|
||||||
|
|
||||||
|
if (direction == -1)
|
||||||
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << " DESC" << (settings.hilite ? hilite_none : "");
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -85,4 +85,23 @@ private:
|
|||||||
std::unordered_map<Child, size_t> positions;
|
std::unordered_map<Child, size_t> positions;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class ASTStorageOrderByElement : public IAST
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
int direction = 1; /// 1 for ASC, -1 for DESC
|
||||||
|
|
||||||
|
ASTPtr clone() const override
|
||||||
|
{
|
||||||
|
auto clone = std::make_shared<ASTStorageOrderByElement>(*this);
|
||||||
|
clone->cloneChildren();
|
||||||
|
return clone;
|
||||||
|
}
|
||||||
|
|
||||||
|
String getID(char) const override { return "StorageOrderByElement"; }
|
||||||
|
void updateTreeHashImpl(SipHash & hash_state, bool ignore_aliases) const override;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||||
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2163,6 +2163,39 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool ParserStorageOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
ParserExpression elem_p;
|
||||||
|
ParserKeyword ascending(Keyword::ASCENDING);
|
||||||
|
ParserKeyword descending(Keyword::DESCENDING);
|
||||||
|
ParserKeyword asc(Keyword::ASC);
|
||||||
|
ParserKeyword desc(Keyword::DESC);
|
||||||
|
|
||||||
|
ASTPtr expr_elem;
|
||||||
|
if (!elem_p.parse(pos, expr_elem, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (!allow_order)
|
||||||
|
{
|
||||||
|
node = std::move(expr_elem);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
int direction = 1;
|
||||||
|
|
||||||
|
if (descending.ignore(pos, expected) || desc.ignore(pos, expected))
|
||||||
|
direction = -1;
|
||||||
|
else
|
||||||
|
ascending.ignore(pos, expected) || asc.ignore(pos, expected);
|
||||||
|
|
||||||
|
auto storage_elem = std::make_shared<ASTStorageOrderByElement>();
|
||||||
|
storage_elem->children.push_back(std::move(expr_elem));
|
||||||
|
storage_elem->direction = direction;
|
||||||
|
|
||||||
|
node = std::move(storage_elem);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
|
@ -432,6 +432,19 @@ protected:
|
|||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/** Element of storage ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
|
||||||
|
*/
|
||||||
|
class ParserStorageOrderByElement : public IParserBase
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ParserStorageOrderByElement(bool allow_order_) : allow_order(allow_order_) {}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
bool allow_order;
|
||||||
|
|
||||||
|
const char * getName() const override { return "element of storage ORDER BY expression"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
/** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
|
/** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
|
||||||
* and optionally, NULLS LAST|FIRST
|
* and optionally, NULLS LAST|FIRST
|
||||||
|
@ -325,6 +325,12 @@ bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected
|
|||||||
return nested_parser.parse(pos, node, expected) && !node->children.empty();
|
return nested_parser.parse(pos, node, expected) && !node->children.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool ParserStorageOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
return ParserList(std::make_unique<ParserStorageOrderByElement>(allow_order), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||||
|
.parse(pos, node, expected);
|
||||||
|
}
|
||||||
|
|
||||||
bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
return ParserList(std::make_unique<ParserOrderByElement>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
return ParserList(std::make_unique<ParserOrderByElement>(), std::make_unique<ParserToken>(TokenType::Comma), false)
|
||||||
|
@ -249,6 +249,17 @@ protected:
|
|||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class ParserStorageOrderByExpressionList : public IParserBase
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ParserStorageOrderByExpressionList(bool allow_order_) : allow_order(allow_order_) {}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
bool allow_order;
|
||||||
|
|
||||||
|
const char * getName() const override { return "storage order by expression"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
class ParserOrderByExpressionList : public IParserBase
|
class ParserOrderByExpressionList : public IParserBase
|
||||||
{
|
{
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
#include <Parsers/ParserViewTargets.h>
|
#include <Parsers/ParserViewTargets.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Parsers/ASTColumnDeclaration.h>
|
#include <Parsers/ASTColumnDeclaration.h>
|
||||||
|
#include <Parsers/ASTOrderByElement.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -494,6 +495,47 @@ bool ParserTablePropertiesDeclarationList::parseImpl(Pos & pos, ASTPtr & node, E
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool ParserStorageOrderByClause::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
ParserStorageOrderByExpressionList order_list_p(allow_order);
|
||||||
|
ParserStorageOrderByElement order_elem_p(allow_order);
|
||||||
|
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||||
|
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||||
|
|
||||||
|
ASTPtr order_by;
|
||||||
|
|
||||||
|
/// Check possible ASC|DESC suffix for single key
|
||||||
|
if (order_elem_p.parse(pos, order_by, expected))
|
||||||
|
{
|
||||||
|
/// This is needed because 'order by (x, y)' is parsed as tuple.
|
||||||
|
/// We can remove ASTStorageOrderByElement if no ASC|DESC suffix was specified.
|
||||||
|
if (const auto * elem = order_by->as<ASTStorageOrderByElement>(); elem && elem->direction > 0)
|
||||||
|
order_by = elem->children.front();
|
||||||
|
|
||||||
|
node = order_by;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Check possible ASC|DESC suffix for a list of keys
|
||||||
|
if (pos->type == TokenType::BareWord && std::string_view(pos->begin, pos->size()) == "tuple")
|
||||||
|
++pos;
|
||||||
|
|
||||||
|
if (!s_lparen.ignore(pos, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (!order_list_p.parse(pos, order_by, expected))
|
||||||
|
order_by = std::make_shared<ASTExpressionList>();
|
||||||
|
|
||||||
|
if (!s_rparen.ignore(pos, expected))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
auto tuple_function = std::make_shared<ASTFunction>();
|
||||||
|
tuple_function->name = "tuple";
|
||||||
|
tuple_function->arguments = std::move(order_by);
|
||||||
|
|
||||||
|
node = std::move(tuple_function);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
@ -508,6 +550,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
|
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
|
||||||
ParserExpression expression_p;
|
ParserExpression expression_p;
|
||||||
|
ParserStorageOrderByClause order_by_p(/*allow_order_*/ true);
|
||||||
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
|
ParserSetQuery settings_p(/* parse_only_internals_ = */ true);
|
||||||
ParserTTLExpressionList parser_ttl_list;
|
ParserTTLExpressionList parser_ttl_list;
|
||||||
ParserStringLiteral string_literal_parser;
|
ParserStringLiteral string_literal_parser;
|
||||||
@ -556,7 +599,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|||||||
|
|
||||||
if (!order_by && s_order_by.ignore(pos, expected))
|
if (!order_by && s_order_by.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
if (expression_p.parse(pos, order_by, expected))
|
if (order_by_p.parse(pos, order_by, expected))
|
||||||
{
|
{
|
||||||
storage_like = true;
|
storage_like = true;
|
||||||
continue;
|
continue;
|
||||||
|
@ -88,6 +88,17 @@ protected:
|
|||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
class ParserStorageOrderByClause : public IParserBase
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ParserStorageOrderByClause(bool allow_order_) : allow_order(allow_order_) {}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
bool allow_order;
|
||||||
|
|
||||||
|
const char * getName() const override { return "storage order by clause"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
template <typename NameParser>
|
template <typename NameParser>
|
||||||
class IParserColumnDeclaration : public IParserBase
|
class IParserColumnDeclaration : public IParserBase
|
||||||
|
@ -22,7 +22,7 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
|||||||
|
|
||||||
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
|
ParserNotEmptyExpressionList exp_list_for_with_clause(false);
|
||||||
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
|
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
|
||||||
ParserExpression order_expression_p;
|
ParserStorageOrderByExpressionList order_list_p(/*allow_order_*/ false);
|
||||||
|
|
||||||
ASTPtr with_expression_list;
|
ASTPtr with_expression_list;
|
||||||
ASTPtr select_expression_list;
|
ASTPtr select_expression_list;
|
||||||
@ -59,7 +59,7 @@ bool ParserProjectionSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
|||||||
if (s_order_by.ignore(pos, expected))
|
if (s_order_by.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
ASTPtr expr_list;
|
ASTPtr expr_list;
|
||||||
if (!ParserList(std::make_unique<ParserExpression>(), std::make_unique<ParserToken>(TokenType::Comma)).parse(pos, expr_list, expected))
|
if (!order_list_p.parse(pos, expr_list, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (expr_list->children.size() == 1)
|
if (expr_list->children.size() == 1)
|
||||||
|
@ -926,6 +926,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
|
|||||||
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node)
|
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node)
|
||||||
{
|
{
|
||||||
const auto & function_node = node->as<FunctionNode &>();
|
const auto & function_node = node->as<FunctionNode &>();
|
||||||
|
|
||||||
if (function_node.getFunctionName() == "indexHint")
|
if (function_node.getFunctionName() == "indexHint")
|
||||||
return visitIndexHintFunction(node);
|
return visitIndexHintFunction(node);
|
||||||
|
|
||||||
|
@ -391,8 +391,8 @@ SortingInputOrder buildInputOrderFromSortDescription(
|
|||||||
}
|
}
|
||||||
|
|
||||||
/// This is a result direction we will read from MergeTree
|
/// This is a result direction we will read from MergeTree
|
||||||
/// 1 - in order,
|
/// 1 - in same order of keys,
|
||||||
/// -1 - in reverse order,
|
/// -1 - in reverse order of keys,
|
||||||
/// 0 - usual read, don't apply optimization
|
/// 0 - usual read, don't apply optimization
|
||||||
///
|
///
|
||||||
/// So far, 0 means any direction is possible. It is ok for constant prefix.
|
/// So far, 0 means any direction is possible. It is ok for constant prefix.
|
||||||
@ -415,6 +415,7 @@ SortingInputOrder buildInputOrderFromSortDescription(
|
|||||||
while (next_description_column < description.size() && next_sort_key < sorting_key.column_names.size())
|
while (next_description_column < description.size() && next_sort_key < sorting_key.column_names.size())
|
||||||
{
|
{
|
||||||
const auto & sorting_key_column = sorting_key.column_names[next_sort_key];
|
const auto & sorting_key_column = sorting_key.column_names[next_sort_key];
|
||||||
|
int reverse_indicator = (!sorting_key.reverse_flags.empty() && sorting_key.reverse_flags[next_sort_key]) ? -1 : 1;
|
||||||
const auto & sort_column_description = description[next_description_column];
|
const auto & sort_column_description = description[next_description_column];
|
||||||
|
|
||||||
/// If required order depend on collation, it cannot be matched with primary key order.
|
/// If required order depend on collation, it cannot be matched with primary key order.
|
||||||
@ -425,7 +426,7 @@ SortingInputOrder buildInputOrderFromSortDescription(
|
|||||||
/// Since sorting key columns are always sorted with NULLS LAST, reading in order
|
/// Since sorting key columns are always sorted with NULLS LAST, reading in order
|
||||||
/// supported only for ASC NULLS LAST ("in order"), and DESC NULLS FIRST ("reverse")
|
/// supported only for ASC NULLS LAST ("in order"), and DESC NULLS FIRST ("reverse")
|
||||||
const auto column_is_nullable = sorting_key.data_types[next_sort_key]->isNullable();
|
const auto column_is_nullable = sorting_key.data_types[next_sort_key]->isNullable();
|
||||||
if (column_is_nullable && sort_column_description.nulls_direction != 1)
|
if (column_is_nullable && sort_column_description.nulls_direction != sort_column_description.direction)
|
||||||
break;
|
break;
|
||||||
|
|
||||||
/// Direction for current sort key.
|
/// Direction for current sort key.
|
||||||
@ -448,8 +449,7 @@ SortingInputOrder buildInputOrderFromSortDescription(
|
|||||||
if (sort_column_description.column_name != sorting_key_column)
|
if (sort_column_description.column_name != sorting_key_column)
|
||||||
break;
|
break;
|
||||||
|
|
||||||
current_direction = sort_column_description.direction;
|
current_direction = sort_column_description.direction * reverse_indicator;
|
||||||
|
|
||||||
|
|
||||||
//std::cerr << "====== (no dag) Found direct match" << std::endl;
|
//std::cerr << "====== (no dag) Found direct match" << std::endl;
|
||||||
|
|
||||||
@ -477,7 +477,7 @@ SortingInputOrder buildInputOrderFromSortDescription(
|
|||||||
/// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1'
|
/// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1'
|
||||||
/// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order.
|
/// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order.
|
||||||
|
|
||||||
current_direction = sort_column_description.direction;
|
current_direction = sort_column_description.direction * reverse_indicator;
|
||||||
if (match.monotonicity)
|
if (match.monotonicity)
|
||||||
{
|
{
|
||||||
current_direction *= match.monotonicity->direction;
|
current_direction *= match.monotonicity->direction;
|
||||||
|
@ -225,10 +225,15 @@ static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts)
|
|||||||
|
|
||||||
/// build sort description for output stream
|
/// build sort description for output stream
|
||||||
static SortDescription getSortDescriptionForOutputHeader(
|
static SortDescription getSortDescriptionForOutputHeader(
|
||||||
const Header & output_header, const Names & sorting_key_columns, const int sort_direction, InputOrderInfoPtr input_order_info, PrewhereInfoPtr prewhere_info, bool enable_vertical_final)
|
const Header & output_header,
|
||||||
|
const Names & sorting_key_columns,
|
||||||
|
const int sort_direction,
|
||||||
|
InputOrderInfoPtr input_order_info,
|
||||||
|
PrewhereInfoPtr prewhere_info,
|
||||||
|
bool enable_vertical_final)
|
||||||
{
|
{
|
||||||
/// Updating sort description can be done after PREWHERE actions are applied to the header.
|
/// Updating sort description can be done after PREWHERE actions are applied to the header.
|
||||||
/// Aftert PREWHERE actions are applied, column names in header can differ from storage column names due to aliases
|
/// After PREWHERE actions are applied, column names in header can differ from storage column names due to aliases
|
||||||
/// To mitigate it, we're trying to build original header and use it to deduce sorting description
|
/// To mitigate it, we're trying to build original header and use it to deduce sorting description
|
||||||
/// TODO: this approach is fragile, it'd be more robust to update sorting description for the whole plan during plan optimization
|
/// TODO: this approach is fragile, it'd be more robust to update sorting description for the whole plan during plan optimization
|
||||||
Block original_header = output_header.cloneEmpty();
|
Block original_header = output_header.cloneEmpty();
|
||||||
@ -1418,6 +1423,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
|||||||
continue;
|
continue;
|
||||||
|
|
||||||
Names sort_columns = storage_snapshot->metadata->getSortingKeyColumns();
|
Names sort_columns = storage_snapshot->metadata->getSortingKeyColumns();
|
||||||
|
std::vector<bool> reverse_flags = storage_snapshot->metadata->getSortingKeyReverseFlags();
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
sort_description.compile_sort_description = settings[Setting::compile_sort_description];
|
sort_description.compile_sort_description = settings[Setting::compile_sort_description];
|
||||||
sort_description.min_count_to_compile_sort_description = settings[Setting::min_count_to_compile_sort_description];
|
sort_description.min_count_to_compile_sort_description = settings[Setting::min_count_to_compile_sort_description];
|
||||||
@ -1428,7 +1434,12 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
|
|||||||
Names partition_key_columns = storage_snapshot->metadata->getPartitionKey().column_names;
|
Names partition_key_columns = storage_snapshot->metadata->getPartitionKey().column_names;
|
||||||
|
|
||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
{
|
||||||
|
if (!reverse_flags.empty() && reverse_flags[i])
|
||||||
|
sort_description.emplace_back(sort_columns[i], -1, 1);
|
||||||
|
else
|
||||||
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
for (auto & pipe : pipes)
|
for (auto & pipe : pipes)
|
||||||
addMergingFinal(
|
addMergingFinal(
|
||||||
|
@ -411,7 +411,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep(
|
|||||||
context_)
|
context_)
|
||||||
, column_names{column_names_}
|
, column_names{column_names_}
|
||||||
, storage{std::move(storage_)}
|
, storage{std::move(storage_)}
|
||||||
, key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context).expression}
|
, key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context, false).expression}
|
||||||
, max_block_size{max_block_size_}
|
, max_block_size{max_block_size_}
|
||||||
, num_streams{num_streams_}
|
, num_streams{num_streams_}
|
||||||
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery &>(), context))
|
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery &>(), context))
|
||||||
|
@ -9,7 +9,8 @@
|
|||||||
#include <Storages/extractKeyExpressionList.h>
|
#include <Storages/extractKeyExpressionList.h>
|
||||||
#include <Common/quoteString.h>
|
#include <Common/quoteString.h>
|
||||||
#include <Interpreters/FunctionNameNormalizer.h>
|
#include <Interpreters/FunctionNameNormalizer.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ASTOrderByElement.h>
|
||||||
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
|
|
||||||
|
|
||||||
@ -27,6 +28,7 @@ KeyDescription::KeyDescription(const KeyDescription & other)
|
|||||||
, expression_list_ast(other.expression_list_ast ? other.expression_list_ast->clone() : nullptr)
|
, expression_list_ast(other.expression_list_ast ? other.expression_list_ast->clone() : nullptr)
|
||||||
, sample_block(other.sample_block)
|
, sample_block(other.sample_block)
|
||||||
, column_names(other.column_names)
|
, column_names(other.column_names)
|
||||||
|
, reverse_flags(other.reverse_flags)
|
||||||
, data_types(other.data_types)
|
, data_types(other.data_types)
|
||||||
, additional_column(other.additional_column)
|
, additional_column(other.additional_column)
|
||||||
{
|
{
|
||||||
@ -49,7 +51,6 @@ KeyDescription & KeyDescription::operator=(const KeyDescription & other)
|
|||||||
else
|
else
|
||||||
expression_list_ast.reset();
|
expression_list_ast.reset();
|
||||||
|
|
||||||
|
|
||||||
if (other.expression)
|
if (other.expression)
|
||||||
expression = other.expression->clone();
|
expression = other.expression->clone();
|
||||||
else
|
else
|
||||||
@ -57,6 +58,7 @@ KeyDescription & KeyDescription::operator=(const KeyDescription & other)
|
|||||||
|
|
||||||
sample_block = other.sample_block;
|
sample_block = other.sample_block;
|
||||||
column_names = other.column_names;
|
column_names = other.column_names;
|
||||||
|
reverse_flags = other.reverse_flags;
|
||||||
data_types = other.data_types;
|
data_types = other.data_types;
|
||||||
|
|
||||||
/// additional_column is constant property It should never be lost.
|
/// additional_column is constant property It should never be lost.
|
||||||
@ -122,18 +124,38 @@ KeyDescription KeyDescription::getSortingKeyFromAST(
|
|||||||
{
|
{
|
||||||
KeyDescription result;
|
KeyDescription result;
|
||||||
result.definition_ast = definition_ast;
|
result.definition_ast = definition_ast;
|
||||||
result.expression_list_ast = extractKeyExpressionList(definition_ast);
|
auto key_expression_list = extractKeyExpressionList(definition_ast);
|
||||||
|
|
||||||
|
result.expression_list_ast = std::make_shared<ASTExpressionList>();
|
||||||
|
for (const auto & child : key_expression_list->children)
|
||||||
|
{
|
||||||
|
auto real_key = child;
|
||||||
|
if (auto * elem = child->as<ASTStorageOrderByElement>())
|
||||||
|
{
|
||||||
|
real_key = elem->children.front();
|
||||||
|
result.reverse_flags.emplace_back(elem->direction < 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
result.expression_list_ast->children.push_back(real_key);
|
||||||
|
result.column_names.emplace_back(real_key->getColumnName());
|
||||||
|
}
|
||||||
|
|
||||||
if (additional_column)
|
if (additional_column)
|
||||||
{
|
{
|
||||||
result.additional_column = additional_column;
|
result.additional_column = additional_column;
|
||||||
ASTPtr column_identifier = std::make_shared<ASTIdentifier>(*additional_column);
|
ASTPtr column_identifier = std::make_shared<ASTIdentifier>(*additional_column);
|
||||||
|
result.column_names.emplace_back(column_identifier->getColumnName());
|
||||||
result.expression_list_ast->children.push_back(column_identifier);
|
result.expression_list_ast->children.push_back(column_identifier);
|
||||||
|
|
||||||
|
if (!result.reverse_flags.empty())
|
||||||
|
result.reverse_flags.emplace_back(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
const auto & children = result.expression_list_ast->children;
|
if (!result.reverse_flags.empty() && result.reverse_flags.size() != result.expression_list_ast->children.size())
|
||||||
for (const auto & child : children)
|
throw Exception(
|
||||||
result.column_names.emplace_back(child->getColumnName());
|
ErrorCodes::LOGICAL_ERROR,
|
||||||
|
"The size of reverse_flags ({}) does not match the size of KeyDescription {}",
|
||||||
|
result.reverse_flags.size(), result.expression_list_ast->children.size());
|
||||||
|
|
||||||
{
|
{
|
||||||
auto expr = result.expression_list_ast->clone();
|
auto expr = result.expression_list_ast->clone();
|
||||||
@ -168,6 +190,24 @@ KeyDescription KeyDescription::getSortingKeyFromAST(
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ASTPtr KeyDescription::getOriginalExpressionList() const
|
||||||
|
{
|
||||||
|
if (!expression_list_ast || reverse_flags.empty())
|
||||||
|
return expression_list_ast;
|
||||||
|
|
||||||
|
auto expr_list = std::make_shared<ASTExpressionList>();
|
||||||
|
size_t size = expression_list_ast->children.size();
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
auto column_ast = std::make_shared<ASTStorageOrderByElement>();
|
||||||
|
column_ast->children.push_back(expression_list_ast->children[i]);
|
||||||
|
column_ast->direction = (!reverse_flags.empty() && reverse_flags[i]) ? -1 : 1;
|
||||||
|
expr_list->children.push_back(std::move(column_ast));
|
||||||
|
}
|
||||||
|
|
||||||
|
return expr_list;
|
||||||
|
}
|
||||||
|
|
||||||
KeyDescription KeyDescription::buildEmptyKey()
|
KeyDescription KeyDescription::buildEmptyKey()
|
||||||
{
|
{
|
||||||
KeyDescription result;
|
KeyDescription result;
|
||||||
@ -176,13 +216,13 @@ KeyDescription KeyDescription::buildEmptyKey()
|
|||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
KeyDescription KeyDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr context)
|
KeyDescription KeyDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr context, bool allow_order)
|
||||||
{
|
{
|
||||||
KeyDescription result;
|
KeyDescription result;
|
||||||
if (str.empty())
|
if (str.empty())
|
||||||
return result;
|
return result;
|
||||||
|
|
||||||
ParserExpression parser;
|
ParserStorageOrderByClause parser(allow_order);
|
||||||
ASTPtr ast = parseQuery(parser, "(" + str + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
|
ASTPtr ast = parseQuery(parser, "(" + str + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
|
||||||
FunctionNameNormalizer::visit(ast.get());
|
FunctionNameNormalizer::visit(ast.get());
|
||||||
|
|
||||||
|
@ -14,7 +14,7 @@ struct KeyDescription
|
|||||||
/// primary key in merge tree can be part of sorting key)
|
/// primary key in merge tree can be part of sorting key)
|
||||||
ASTPtr definition_ast;
|
ASTPtr definition_ast;
|
||||||
|
|
||||||
/// ASTExpressionList with key fields, example: (x, toStartOfMonth(date))).
|
/// ASTExpressionList with key fields, example: (x DESC, toStartOfMonth(date))).
|
||||||
ASTPtr expression_list_ast;
|
ASTPtr expression_list_ast;
|
||||||
|
|
||||||
/// Expression from expression_list_ast created by ExpressionAnalyzer. Useful,
|
/// Expression from expression_list_ast created by ExpressionAnalyzer. Useful,
|
||||||
@ -27,6 +27,9 @@ struct KeyDescription
|
|||||||
/// Column names in key definition, example: x, toStartOfMonth(date), a * b.
|
/// Column names in key definition, example: x, toStartOfMonth(date), a * b.
|
||||||
Names column_names;
|
Names column_names;
|
||||||
|
|
||||||
|
/// Indicator of key column being sorted reversely, example: x DESC, y -> {1, 0}.
|
||||||
|
std::vector<bool> reverse_flags;
|
||||||
|
|
||||||
/// Types from sample block ordered in columns order.
|
/// Types from sample block ordered in columns order.
|
||||||
DataTypes data_types;
|
DataTypes data_types;
|
||||||
|
|
||||||
@ -67,6 +70,8 @@ struct KeyDescription
|
|||||||
const ColumnsDescription & columns,
|
const ColumnsDescription & columns,
|
||||||
ContextPtr context);
|
ContextPtr context);
|
||||||
|
|
||||||
|
ASTPtr getOriginalExpressionList() const;
|
||||||
|
|
||||||
KeyDescription() = default;
|
KeyDescription() = default;
|
||||||
|
|
||||||
/// We need custom copy constructors because we don't want
|
/// We need custom copy constructors because we don't want
|
||||||
@ -78,7 +83,7 @@ struct KeyDescription
|
|||||||
static bool moduloToModuloLegacyRecursive(ASTPtr node_expr);
|
static bool moduloToModuloLegacyRecursive(ASTPtr node_expr);
|
||||||
|
|
||||||
/// Parse description from string
|
/// Parse description from string
|
||||||
static KeyDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context);
|
static KeyDescription parse(const String & str, const ColumnsDescription & columns, ContextPtr context, bool allow_order);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1763,6 +1763,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
|
|||||||
/// Merge
|
/// Merge
|
||||||
{
|
{
|
||||||
Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns();
|
Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns();
|
||||||
|
std::vector<bool> reverse_flags = global_ctx->metadata_snapshot->getSortingKeyReverseFlags();
|
||||||
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef()[Setting::compile_sort_description];
|
sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef()[Setting::compile_sort_description];
|
||||||
sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef()[Setting::min_count_to_compile_sort_description];
|
sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef()[Setting::min_count_to_compile_sort_description];
|
||||||
|
|
||||||
@ -1772,7 +1773,12 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
|
|||||||
Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names;
|
Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names;
|
||||||
|
|
||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
{
|
||||||
|
if (!reverse_flags.empty() && reverse_flags[i])
|
||||||
|
sort_description.emplace_back(sort_columns[i], -1, 1);
|
||||||
|
else
|
||||||
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
const bool is_vertical_merge = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
const bool is_vertical_merge = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical);
|
||||||
/// If merge is vertical we cannot calculate it
|
/// If merge is vertical we cannot calculate it
|
||||||
|
@ -185,6 +185,7 @@ namespace Setting
|
|||||||
|
|
||||||
namespace MergeTreeSetting
|
namespace MergeTreeSetting
|
||||||
{
|
{
|
||||||
|
extern const MergeTreeSettingsBool allow_experimental_reverse_key;
|
||||||
extern const MergeTreeSettingsBool allow_nullable_key;
|
extern const MergeTreeSettingsBool allow_nullable_key;
|
||||||
extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication;
|
extern const MergeTreeSettingsBool allow_remote_fs_zero_copy_replication;
|
||||||
extern const MergeTreeSettingsBool allow_suspicious_indices;
|
extern const MergeTreeSettingsBool allow_suspicious_indices;
|
||||||
@ -460,6 +461,7 @@ MergeTreeData::MergeTreeData(
|
|||||||
bool sanity_checks = mode <= LoadingStrictnessLevel::CREATE;
|
bool sanity_checks = mode <= LoadingStrictnessLevel::CREATE;
|
||||||
|
|
||||||
allow_nullable_key = !sanity_checks || (*settings)[MergeTreeSetting::allow_nullable_key];
|
allow_nullable_key = !sanity_checks || (*settings)[MergeTreeSetting::allow_nullable_key];
|
||||||
|
allow_reverse_key = !sanity_checks || (*settings)[MergeTreeSetting::allow_experimental_reverse_key];
|
||||||
|
|
||||||
/// Check sanity of MergeTreeSettings. Only when table is created.
|
/// Check sanity of MergeTreeSettings. Only when table is created.
|
||||||
if (sanity_checks)
|
if (sanity_checks)
|
||||||
@ -666,12 +668,28 @@ void MergeTreeData::checkProperties(
|
|||||||
const StorageInMemoryMetadata & old_metadata,
|
const StorageInMemoryMetadata & old_metadata,
|
||||||
bool attach,
|
bool attach,
|
||||||
bool allow_empty_sorting_key,
|
bool allow_empty_sorting_key,
|
||||||
|
bool allow_reverse_sorting_key,
|
||||||
bool allow_nullable_key_,
|
bool allow_nullable_key_,
|
||||||
ContextPtr local_context) const
|
ContextPtr local_context) const
|
||||||
{
|
{
|
||||||
if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key)
|
if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key)
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty");
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty");
|
||||||
|
|
||||||
|
if (!allow_reverse_sorting_key)
|
||||||
|
{
|
||||||
|
size_t num_sorting_keys = new_metadata.sorting_key.column_names.size();
|
||||||
|
for (size_t i = 0; i < num_sorting_keys; ++i)
|
||||||
|
{
|
||||||
|
if (!new_metadata.sorting_key.reverse_flags.empty() && new_metadata.sorting_key.reverse_flags[i])
|
||||||
|
{
|
||||||
|
throw Exception(
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN,
|
||||||
|
"Sorting key {} is reversed, but merge tree setting `allow_experimental_reverse_key` is disabled",
|
||||||
|
new_metadata.sorting_key.column_names[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
KeyDescription new_sorting_key = new_metadata.sorting_key;
|
KeyDescription new_sorting_key = new_metadata.sorting_key;
|
||||||
KeyDescription new_primary_key = new_metadata.primary_key;
|
KeyDescription new_primary_key = new_metadata.primary_key;
|
||||||
|
|
||||||
@ -800,7 +818,14 @@ void MergeTreeData::checkProperties(
|
|||||||
|
|
||||||
/// We cannot alter a projection so far. So here we do not try to find a projection in old metadata.
|
/// We cannot alter a projection so far. So here we do not try to find a projection in old metadata.
|
||||||
bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate;
|
bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate;
|
||||||
checkProperties(*projection.metadata, *projection.metadata, attach, is_aggregate, true /* allow_nullable_key */, local_context);
|
checkProperties(
|
||||||
|
*projection.metadata,
|
||||||
|
*projection.metadata,
|
||||||
|
attach,
|
||||||
|
is_aggregate,
|
||||||
|
allow_reverse_key,
|
||||||
|
true /* allow_nullable_key */,
|
||||||
|
local_context);
|
||||||
projections_names.insert(projection.name);
|
projections_names.insert(projection.name);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -820,7 +845,14 @@ void MergeTreeData::setProperties(
|
|||||||
bool attach,
|
bool attach,
|
||||||
ContextPtr local_context)
|
ContextPtr local_context)
|
||||||
{
|
{
|
||||||
checkProperties(new_metadata, old_metadata, attach, false, allow_nullable_key, local_context);
|
checkProperties(
|
||||||
|
new_metadata,
|
||||||
|
old_metadata,
|
||||||
|
attach,
|
||||||
|
false,
|
||||||
|
allow_reverse_key,
|
||||||
|
allow_nullable_key,
|
||||||
|
local_context);
|
||||||
setInMemoryMetadata(new_metadata);
|
setInMemoryMetadata(new_metadata);
|
||||||
setVirtuals(createVirtuals(new_metadata));
|
setVirtuals(createVirtuals(new_metadata));
|
||||||
}
|
}
|
||||||
@ -3701,7 +3733,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
|||||||
}
|
}
|
||||||
|
|
||||||
checkColumnFilenamesForCollision(new_metadata, /*throw_on_error=*/ true);
|
checkColumnFilenamesForCollision(new_metadata, /*throw_on_error=*/ true);
|
||||||
checkProperties(new_metadata, old_metadata, false, false, allow_nullable_key, local_context);
|
checkProperties(new_metadata, old_metadata, false, false, allow_reverse_key, allow_nullable_key, local_context);
|
||||||
checkTTLExpressions(new_metadata, old_metadata);
|
checkTTLExpressions(new_metadata, old_metadata);
|
||||||
|
|
||||||
if (!columns_to_check_conversion.empty())
|
if (!columns_to_check_conversion.empty())
|
||||||
|
@ -1334,6 +1334,7 @@ protected:
|
|||||||
const StorageInMemoryMetadata & old_metadata,
|
const StorageInMemoryMetadata & old_metadata,
|
||||||
bool attach,
|
bool attach,
|
||||||
bool allow_empty_sorting_key,
|
bool allow_empty_sorting_key,
|
||||||
|
bool allow_reverse_sorting_key,
|
||||||
bool allow_nullable_key_,
|
bool allow_nullable_key_,
|
||||||
ContextPtr local_context) const;
|
ContextPtr local_context) const;
|
||||||
|
|
||||||
@ -1714,7 +1715,8 @@ private:
|
|||||||
|
|
||||||
virtual void startBackgroundMovesIfNeeded() = 0;
|
virtual void startBackgroundMovesIfNeeded() = 0;
|
||||||
|
|
||||||
bool allow_nullable_key{};
|
bool allow_nullable_key = false;
|
||||||
|
bool allow_reverse_key = false;
|
||||||
|
|
||||||
void addPartContributionToDataVolume(const DataPartPtr & part);
|
void addPartContributionToDataVolume(const DataPartPtr & part);
|
||||||
void removePartContributionToDataVolume(const DataPartPtr & part);
|
void removePartContributionToDataVolume(const DataPartPtr & part);
|
||||||
|
@ -1070,6 +1070,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
|||||||
|
|
||||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
||||||
auto index_columns = std::make_shared<ColumnsWithTypeAndName>();
|
auto index_columns = std::make_shared<ColumnsWithTypeAndName>();
|
||||||
|
std::vector<bool> reverse_flags;
|
||||||
const auto & key_indices = key_condition.getKeyIndices();
|
const auto & key_indices = key_condition.getKeyIndices();
|
||||||
DataTypes key_types;
|
DataTypes key_types;
|
||||||
if (!key_indices.empty())
|
if (!key_indices.empty())
|
||||||
@ -1079,9 +1080,15 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
|||||||
for (size_t i : key_indices)
|
for (size_t i : key_indices)
|
||||||
{
|
{
|
||||||
if (i < index->size())
|
if (i < index->size())
|
||||||
|
{
|
||||||
index_columns->emplace_back(index->at(i), primary_key.data_types[i], primary_key.column_names[i]);
|
index_columns->emplace_back(index->at(i), primary_key.data_types[i], primary_key.column_names[i]);
|
||||||
|
reverse_flags.push_back(!primary_key.reverse_flags.empty() && primary_key.reverse_flags[i]);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
|
{
|
||||||
index_columns->emplace_back(); /// The column of the primary key was not loaded in memory - we'll skip it.
|
index_columns->emplace_back(); /// The column of the primary key was not loaded in memory - we'll skip it.
|
||||||
|
reverse_flags.push_back(false);
|
||||||
|
}
|
||||||
|
|
||||||
key_types.emplace_back(primary_key.data_types[i]);
|
key_types.emplace_back(primary_key.data_types[i]);
|
||||||
}
|
}
|
||||||
@ -1130,28 +1137,32 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
|
|||||||
{
|
{
|
||||||
for (size_t i = 0; i < used_key_size; ++i)
|
for (size_t i = 0; i < used_key_size; ++i)
|
||||||
{
|
{
|
||||||
|
auto & left = reverse_flags[i] ? index_right[i] : index_left[i];
|
||||||
|
auto & right = reverse_flags[i] ? index_left[i] : index_right[i];
|
||||||
if ((*index_columns)[i].column)
|
if ((*index_columns)[i].column)
|
||||||
create_field_ref(range.begin, i, index_left[i]);
|
create_field_ref(range.begin, i, left);
|
||||||
else
|
else
|
||||||
index_left[i] = NEGATIVE_INFINITY;
|
left = NEGATIVE_INFINITY;
|
||||||
|
|
||||||
index_right[i] = POSITIVE_INFINITY;
|
right = POSITIVE_INFINITY;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
for (size_t i = 0; i < used_key_size; ++i)
|
for (size_t i = 0; i < used_key_size; ++i)
|
||||||
{
|
{
|
||||||
|
auto & left = reverse_flags[i] ? index_right[i] : index_left[i];
|
||||||
|
auto & right = reverse_flags[i] ? index_left[i] : index_right[i];
|
||||||
if ((*index_columns)[i].column)
|
if ((*index_columns)[i].column)
|
||||||
{
|
{
|
||||||
create_field_ref(range.begin, i, index_left[i]);
|
create_field_ref(range.begin, i, left);
|
||||||
create_field_ref(range.end, i, index_right[i]);
|
create_field_ref(range.end, i, right);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
/// If the PK column was not loaded in memory - exclude it from the analysis.
|
/// If the PK column was not loaded in memory - exclude it from the analysis.
|
||||||
index_left[i] = NEGATIVE_INFINITY;
|
left = NEGATIVE_INFINITY;
|
||||||
index_right[i] = POSITIVE_INFINITY;
|
right = POSITIVE_INFINITY;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -517,12 +517,18 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
|
|||||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, indices)->execute(block);
|
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, indices)->execute(block);
|
||||||
|
|
||||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||||
|
std::vector<bool> reverse_flags = metadata_snapshot->getSortingKeyReverseFlags();
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
size_t sort_columns_size = sort_columns.size();
|
size_t sort_columns_size = sort_columns.size();
|
||||||
sort_description.reserve(sort_columns_size);
|
sort_description.reserve(sort_columns_size);
|
||||||
|
|
||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
{
|
||||||
|
if (!reverse_flags.empty() && reverse_flags[i])
|
||||||
|
sort_description.emplace_back(sort_columns[i], -1, 1);
|
||||||
|
else
|
||||||
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
|
ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocks);
|
||||||
|
|
||||||
@ -795,12 +801,18 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl(
|
|||||||
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, {})->execute(block);
|
data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, {})->execute(block);
|
||||||
|
|
||||||
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
Names sort_columns = metadata_snapshot->getSortingKeyColumns();
|
||||||
|
std::vector<bool> reverse_flags = metadata_snapshot->getSortingKeyReverseFlags();
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
size_t sort_columns_size = sort_columns.size();
|
size_t sort_columns_size = sort_columns.size();
|
||||||
sort_description.reserve(sort_columns_size);
|
sort_description.reserve(sort_columns_size);
|
||||||
|
|
||||||
for (size_t i = 0; i < sort_columns_size; ++i)
|
for (size_t i = 0; i < sort_columns_size; ++i)
|
||||||
sort_description.emplace_back(sort_columns[i], 1, 1);
|
{
|
||||||
|
if (!reverse_flags.empty() && reverse_flags[i])
|
||||||
|
sort_description.emplace_back(sort_columns[i], -1, 1);
|
||||||
|
else
|
||||||
|
sort_description.emplace_back(sort_columns[i], 1, 1);
|
||||||
|
}
|
||||||
|
|
||||||
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocks);
|
ProfileEvents::increment(ProfileEvents::MergeTreeDataProjectionWriterBlocks);
|
||||||
|
|
||||||
|
@ -201,6 +201,7 @@ namespace ErrorCodes
|
|||||||
DECLARE(String, storage_policy, "default", "Name of storage disk policy", 0) \
|
DECLARE(String, storage_policy, "default", "Name of storage disk policy", 0) \
|
||||||
DECLARE(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \
|
DECLARE(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \
|
||||||
DECLARE(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
|
DECLARE(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
|
||||||
|
DECLARE(Bool, allow_experimental_reverse_key, false, "Allow descending sorting key in MergeTree tables (experimental feature).", 0) \
|
||||||
DECLARE(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \
|
DECLARE(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \
|
||||||
DECLARE(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \
|
DECLARE(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \
|
||||||
DECLARE(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \
|
DECLARE(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \
|
||||||
|
@ -77,10 +77,9 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
|||||||
/// So rules in zookeeper metadata is following:
|
/// So rules in zookeeper metadata is following:
|
||||||
/// - When we have only ORDER BY, than store it in "primary key:" row of /metadata
|
/// - When we have only ORDER BY, than store it in "primary key:" row of /metadata
|
||||||
/// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata
|
/// - When we have both, than store PRIMARY KEY in "primary key:" row and ORDER BY in "sorting key:" row of /metadata
|
||||||
|
|
||||||
primary_key = formattedASTNormalized(metadata_snapshot->getPrimaryKey().expression_list_ast);
|
|
||||||
if (metadata_snapshot->isPrimaryKeyDefined())
|
if (metadata_snapshot->isPrimaryKeyDefined())
|
||||||
{
|
{
|
||||||
|
primary_key = formattedASTNormalized(metadata_snapshot->getPrimaryKey().expression_list_ast);
|
||||||
/// We don't use preparsed AST `sorting_key.expression_list_ast` because
|
/// We don't use preparsed AST `sorting_key.expression_list_ast` because
|
||||||
/// it contain version column for VersionedCollapsingMergeTree, which
|
/// it contain version column for VersionedCollapsingMergeTree, which
|
||||||
/// is not stored in ZooKeeper for compatibility reasons. So the best
|
/// is not stored in ZooKeeper for compatibility reasons. So the best
|
||||||
@ -89,6 +88,10 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr
|
|||||||
/// used.
|
/// used.
|
||||||
sorting_key = formattedASTNormalized(extractKeyExpressionList(metadata_snapshot->getSortingKey().definition_ast));
|
sorting_key = formattedASTNormalized(extractKeyExpressionList(metadata_snapshot->getSortingKey().definition_ast));
|
||||||
}
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
primary_key = formattedASTNormalized(metadata_snapshot->getPrimaryKey().getOriginalExpressionList());
|
||||||
|
}
|
||||||
|
|
||||||
data_format_version = data.format_version;
|
data_format_version = data.format_version;
|
||||||
|
|
||||||
@ -301,7 +304,7 @@ void ReplicatedMergeTreeTableMetadata::checkImmutableFieldsEquals(const Replicat
|
|||||||
|
|
||||||
/// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
|
/// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
|
||||||
/// in formatAST code.
|
/// in formatAST code.
|
||||||
String parsed_zk_primary_key = formattedAST(KeyDescription::parse(from_zk.primary_key, columns, context).expression_list_ast);
|
String parsed_zk_primary_key = formattedAST(KeyDescription::parse(from_zk.primary_key, columns, context, true).getOriginalExpressionList());
|
||||||
if (primary_key != parsed_zk_primary_key)
|
if (primary_key != parsed_zk_primary_key)
|
||||||
throw Exception(ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in primary key. "
|
throw Exception(ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in primary key. "
|
||||||
"Stored in ZooKeeper: {}, parsed from ZooKeeper: {}, local: {}",
|
"Stored in ZooKeeper: {}, parsed from ZooKeeper: {}, local: {}",
|
||||||
@ -313,7 +316,7 @@ void ReplicatedMergeTreeTableMetadata::checkImmutableFieldsEquals(const Replicat
|
|||||||
"Stored in ZooKeeper: {}, local: {}", DB::toString(from_zk.data_format_version.toUnderType()),
|
"Stored in ZooKeeper: {}, local: {}", DB::toString(from_zk.data_format_version.toUnderType()),
|
||||||
DB::toString(data_format_version.toUnderType()));
|
DB::toString(data_format_version.toUnderType()));
|
||||||
|
|
||||||
String parsed_zk_partition_key = formattedAST(KeyDescription::parse(from_zk.partition_key, columns, context).expression_list_ast);
|
String parsed_zk_partition_key = formattedAST(KeyDescription::parse(from_zk.partition_key, columns, context, false).expression_list_ast);
|
||||||
if (partition_key != parsed_zk_partition_key)
|
if (partition_key != parsed_zk_partition_key)
|
||||||
throw Exception(ErrorCodes::METADATA_MISMATCH,
|
throw Exception(ErrorCodes::METADATA_MISMATCH,
|
||||||
"Existing table metadata in ZooKeeper differs in partition key expression. "
|
"Existing table metadata in ZooKeeper differs in partition key expression. "
|
||||||
@ -326,7 +329,7 @@ void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTabl
|
|||||||
|
|
||||||
checkImmutableFieldsEquals(from_zk, columns, context);
|
checkImmutableFieldsEquals(from_zk, columns, context);
|
||||||
|
|
||||||
String parsed_zk_sampling_expression = formattedAST(KeyDescription::parse(from_zk.sampling_expression, columns, context).definition_ast);
|
String parsed_zk_sampling_expression = formattedAST(KeyDescription::parse(from_zk.sampling_expression, columns, context, false).definition_ast);
|
||||||
if (sampling_expression != parsed_zk_sampling_expression)
|
if (sampling_expression != parsed_zk_sampling_expression)
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in sample expression. "
|
throw Exception(ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in sample expression. "
|
||||||
@ -334,7 +337,7 @@ void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTabl
|
|||||||
from_zk.sampling_expression, parsed_zk_sampling_expression, sampling_expression);
|
from_zk.sampling_expression, parsed_zk_sampling_expression, sampling_expression);
|
||||||
}
|
}
|
||||||
|
|
||||||
String parsed_zk_sorting_key = formattedAST(extractKeyExpressionList(KeyDescription::parse(from_zk.sorting_key, columns, context).definition_ast));
|
String parsed_zk_sorting_key = formattedAST(extractKeyExpressionList(KeyDescription::parse(from_zk.sorting_key, columns, context, true).definition_ast));
|
||||||
if (sorting_key != parsed_zk_sorting_key)
|
if (sorting_key != parsed_zk_sorting_key)
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::METADATA_MISMATCH,
|
throw Exception(ErrorCodes::METADATA_MISMATCH,
|
||||||
@ -343,7 +346,7 @@ void ReplicatedMergeTreeTableMetadata::checkEquals(const ReplicatedMergeTreeTabl
|
|||||||
from_zk.sorting_key, parsed_zk_sorting_key, sorting_key);
|
from_zk.sorting_key, parsed_zk_sorting_key, sorting_key);
|
||||||
}
|
}
|
||||||
|
|
||||||
auto parsed_primary_key = KeyDescription::parse(primary_key, columns, context);
|
auto parsed_primary_key = KeyDescription::parse(primary_key, columns, context, true);
|
||||||
String parsed_zk_ttl_table = formattedAST(TTLTableDescription::parse(from_zk.ttl_table, columns, context, parsed_primary_key).definition_ast);
|
String parsed_zk_ttl_table = formattedAST(TTLTableDescription::parse(from_zk.ttl_table, columns, context, parsed_primary_key).definition_ast);
|
||||||
if (ttl_table != parsed_zk_ttl_table)
|
if (ttl_table != parsed_zk_ttl_table)
|
||||||
{
|
{
|
||||||
|
@ -513,6 +513,13 @@ Names StorageInMemoryMetadata::getSortingKeyColumns() const
|
|||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
std::vector<bool> StorageInMemoryMetadata::getSortingKeyReverseFlags() const
|
||||||
|
{
|
||||||
|
if (hasSortingKey())
|
||||||
|
return sorting_key.reverse_flags;
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
const KeyDescription & StorageInMemoryMetadata::getSamplingKey() const
|
const KeyDescription & StorageInMemoryMetadata::getSamplingKey() const
|
||||||
{
|
{
|
||||||
return sampling_key;
|
return sampling_key;
|
||||||
|
@ -222,6 +222,9 @@ struct StorageInMemoryMetadata
|
|||||||
/// Returns columns names in sorting key specified by user in ORDER BY
|
/// Returns columns names in sorting key specified by user in ORDER BY
|
||||||
/// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc.
|
/// expression. For example: 'a', 'x * y', 'toStartOfMonth(date)', etc.
|
||||||
Names getSortingKeyColumns() const;
|
Names getSortingKeyColumns() const;
|
||||||
|
/// Returns reverse indicators of columns in sorting key specified by user in ORDER BY
|
||||||
|
/// expression. For example: ('a' DESC, 'x * y', 'toStartOfMonth(date)' DESC) -> {1, 0, 1}.
|
||||||
|
std::vector<bool> getSortingKeyReverseFlags() const;
|
||||||
|
|
||||||
/// Returns column names that need to be read for FINAL to work.
|
/// Returns column names that need to be read for FINAL to work.
|
||||||
Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); }
|
||||||
|
@ -275,7 +275,7 @@ TTLDescription TTLDescription::getTTLFromAST(
|
|||||||
for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i)
|
for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i)
|
||||||
{
|
{
|
||||||
if (ttl_element->group_by_key[i]->getColumnName() != pk_columns[i])
|
if (ttl_element->group_by_key[i]->getColumnName() != pk_columns[i])
|
||||||
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "TTL Expression GROUP BY key should be a prefix of primary key");
|
throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "TTL Expression GROUP BY key should be a prefix of primary key {} {}", ttl_element->group_by_key[i]->getColumnName(), pk_columns[i]);
|
||||||
|
|
||||||
used_primary_key_columns_set.insert(pk_columns[i]);
|
used_primary_key_columns_set.insert(pk_columns[i]);
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,75 @@
|
|||||||
|
3
|
||||||
|
8
|
||||||
|
Sorting (Sorting for ORDER BY)
|
||||||
|
Prefix sort description: __table1.i DESC
|
||||||
|
Result sort description: __table1.i DESC
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(Limit)
|
||||||
|
Limit
|
||||||
|
(Sorting)
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(ReadFromMergeTree)
|
||||||
|
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||||
|
99
|
||||||
|
98
|
||||||
|
97
|
||||||
|
96
|
||||||
|
95
|
||||||
|
Sorting (Sorting for ORDER BY)
|
||||||
|
Prefix sort description: __table1.i ASC
|
||||||
|
Result sort description: __table1.i ASC
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(Limit)
|
||||||
|
Limit
|
||||||
|
(Sorting)
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(ReadFromMergeTree)
|
||||||
|
ReverseTransform
|
||||||
|
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
4
|
||||||
|
3 1003
|
||||||
|
6
|
||||||
|
Sorting (Sorting for ORDER BY)
|
||||||
|
Prefix sort description: __table1.i ASC, __table1.j DESC
|
||||||
|
Result sort description: __table1.i ASC, __table1.j DESC
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(Limit)
|
||||||
|
Limit
|
||||||
|
(Sorting)
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(ReadFromMergeTree)
|
||||||
|
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||||
|
0 1090
|
||||||
|
0 1080
|
||||||
|
0 1070
|
||||||
|
0 1060
|
||||||
|
0 1050
|
||||||
|
Sorting (Sorting for ORDER BY)
|
||||||
|
Prefix sort description: __table1.i ASC
|
||||||
|
Result sort description: __table1.i ASC, __table1.j ASC
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(Limit)
|
||||||
|
Limit
|
||||||
|
(Sorting)
|
||||||
|
FinishSortingTransform
|
||||||
|
PartialSortingTransform
|
||||||
|
(Expression)
|
||||||
|
ExpressionTransform
|
||||||
|
(ReadFromMergeTree)
|
||||||
|
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||||
|
0 1000
|
||||||
|
0 1010
|
||||||
|
0 1020
|
||||||
|
0 1030
|
||||||
|
0 1040
|
52
tests/queries/0_stateless/03257_reverse_sorting_key.sql
Normal file
52
tests/queries/0_stateless/03257_reverse_sorting_key.sql
Normal file
@ -0,0 +1,52 @@
|
|||||||
|
-- Tags: no-random-merge-tree-settings
|
||||||
|
|
||||||
|
set optimize_read_in_order = 1;
|
||||||
|
set read_in_order_two_level_merge_threshold=100;
|
||||||
|
|
||||||
|
drop table if exists x1;
|
||||||
|
|
||||||
|
drop table if exists x2;
|
||||||
|
|
||||||
|
create table x1 (i Nullable(int)) engine MergeTree order by i desc settings allow_nullable_key = 1, index_granularity = 2, allow_experimental_reverse_key = 1;
|
||||||
|
|
||||||
|
insert into x1 select * from numbers(100);
|
||||||
|
|
||||||
|
optimize table x1 final;
|
||||||
|
|
||||||
|
select * from x1 where i = 3;
|
||||||
|
|
||||||
|
select count() from x1 where i between 3 and 10;
|
||||||
|
|
||||||
|
select trimLeft(explain) from (explain actions=1 select * from x1 order by i desc limit 5) where explain ilike '%sort%' settings max_threads=1, enable_analyzer=1;
|
||||||
|
explain pipeline select * from x1 order by i desc limit 5 settings max_threads=1;
|
||||||
|
|
||||||
|
select * from x1 order by i desc limit 5;
|
||||||
|
|
||||||
|
select trimLeft(explain) from (explain actions=1 select * from x1 order by i limit 5) where explain ilike '%sort%' settings max_threads=1, enable_analyzer=1;
|
||||||
|
explain pipeline select * from x1 order by i limit 5 settings max_threads=1;
|
||||||
|
|
||||||
|
select * from x1 order by i limit 5;
|
||||||
|
|
||||||
|
create table x2 (i Nullable(int), j Nullable(int)) engine MergeTree order by (i, j desc) settings allow_nullable_key = 1, index_granularity = 2, allow_experimental_reverse_key = 1;
|
||||||
|
|
||||||
|
insert into x2 select number % 10, number + 1000 from numbers(100);
|
||||||
|
|
||||||
|
optimize table x2 final;
|
||||||
|
|
||||||
|
select * from x2 where j = 1003;
|
||||||
|
|
||||||
|
select count() from x2 where i between 3 and 10 and j between 1003 and 1008;
|
||||||
|
|
||||||
|
select trimLeft(explain) from (explain actions=1 select * from x2 order by i, j desc limit 5) where explain ilike '%sort%' settings max_threads=1, enable_analyzer=1;
|
||||||
|
explain pipeline select * from x2 order by i, j desc limit 5 settings max_threads=1;
|
||||||
|
|
||||||
|
select * from x2 order by i, j desc limit 5;
|
||||||
|
|
||||||
|
select trimLeft(explain) from (explain actions=1 select * from x2 order by i, j limit 5) where explain ilike '%sort%' settings max_threads=1, enable_analyzer=1;
|
||||||
|
explain pipeline select * from x2 order by i, j limit 5 settings max_threads=1;
|
||||||
|
|
||||||
|
select * from x2 order by i, j limit 5;
|
||||||
|
|
||||||
|
drop table x1;
|
||||||
|
|
||||||
|
drop table x2;
|
@ -0,0 +1,2 @@
|
|||||||
|
metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 2\nmode: 0\nsign column: \nprimary key: i DESC\ndata format version: 1\npartition key: \ngranularity bytes: 10000\nmerge parameters format version: 2\n
|
||||||
|
metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 2\nmode: 0\nsign column: \nprimary key: i, j DESC\ndata format version: 1\npartition key: \ngranularity bytes: 10000\nmerge parameters format version: 2\n
|
@ -0,0 +1,16 @@
|
|||||||
|
-- Tags: zookeeper, no-random-merge-tree-settings, no-replicated-database
|
||||||
|
|
||||||
|
drop table if exists x1;
|
||||||
|
drop table if exists x2;
|
||||||
|
|
||||||
|
create table x1 (i Nullable(int)) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x1', 'r1') order by i desc settings allow_nullable_key = 1, index_granularity = 2, index_granularity_bytes = 10000, allow_experimental_reverse_key = 1;
|
||||||
|
|
||||||
|
create table x2 (i Nullable(int), j Nullable(int)) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x2', 'r1') order by (i, j desc) settings allow_nullable_key = 1, index_granularity = 2, index_granularity_bytes = 10000, allow_experimental_reverse_key = 1;
|
||||||
|
|
||||||
|
set allow_unrestricted_reads_from_keeper = 'true';
|
||||||
|
|
||||||
|
select value from system.zookeeper where path = '/clickhouse/tables/' || currentDatabase() || '/x1' and name = 'metadata';
|
||||||
|
select value from system.zookeeper where path = '/clickhouse/tables/' || currentDatabase() || '/x2' and name = 'metadata';
|
||||||
|
|
||||||
|
drop table x1;
|
||||||
|
drop table x2;
|
Loading…
Reference in New Issue
Block a user