Merge remote-tracking branch 'origin/pr-plan-rewrite' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-05-29 10:52:59 +00:00
commit e28ca7ebe8
71 changed files with 2272 additions and 1821 deletions

View File

@ -11,6 +11,7 @@ tests/ci/cancel_and_rerun_workflow_lambda/app.py
- Backward Incompatible Change
- Build/Testing/Packaging Improvement
- Documentation (changelog entry is not required)
- Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)
- Bug Fix (user-visible misbehavior in an official stable release)
- CI Fix or Improvement (changelog entry is not required)
- Not for changelog (changelog entry is not required)

View File

@ -1235,6 +1235,168 @@ Result:
- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter.
## toStartOfMillisecond
Rounds down a date with time to the start of the milliseconds.
**Syntax**
``` sql
toStartOfMillisecond(value, [timezone])
```
**Arguments**
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Input value with sub-milliseconds. [DateTime64](../../sql-reference/data-types/datetime64.md).
**Examples**
Query without timezone:
``` sql
WITH toDateTime64('2020-01-01 10:20:30.999999999', 9) AS dt64
SELECT toStartOfMillisecond(dt64);
```
Result:
``` text
┌────toStartOfMillisecond(dt64)─┐
│ 2020-01-01 10:20:30.999000000 │
└───────────────────────────────┘
```
Query with timezone:
``` sql
┌─toStartOfMillisecond(dt64, 'Asia/Istanbul')─┐
│ 2020-01-01 12:20:30.999000000 │
└─────────────────────────────────────────────┘
```
Result:
``` text
┌─toStartOfMillisecond(dt64, 'Asia/Istanbul')─┐
│ 2020-01-01 12:20:30.999 │
└─────────────────────────────────────────────┘
```
## toStartOfMicrosecond
Rounds down a date with time to the start of the microseconds.
**Syntax**
``` sql
toStartOfMicrosecond(value, [timezone])
```
**Arguments**
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Input value with sub-microseconds. [DateTime64](../../sql-reference/data-types/datetime64.md).
**Examples**
Query without timezone:
``` sql
WITH toDateTime64('2020-01-01 10:20:30.999999999', 9) AS dt64
SELECT toStartOfMicrosecond(dt64);
```
Result:
``` text
┌────toStartOfMicrosecond(dt64)─┐
│ 2020-01-01 10:20:30.999999000 │
└───────────────────────────────┘
```
Query with timezone:
``` sql
WITH toDateTime64('2020-01-01 10:20:30.999999999', 9) AS dt64
SELECT toStartOfMicrosecond(dt64, 'Asia/Istanbul');
```
Result:
``` text
┌─toStartOfMicrosecond(dt64, 'Asia/Istanbul')─┐
│ 2020-01-01 12:20:30.999999000 │
└─────────────────────────────────────────────┘
```
**See also**
- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter.
## toStartOfNanosecond
Rounds down a date with time to the start of the nanoseconds.
**Syntax**
``` sql
toStartOfNanosecond(value, [timezone])
```
**Arguments**
- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md).
- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md).
**Returned value**
- Input value with nanoseconds. [DateTime64](../../sql-reference/data-types/datetime64.md).
**Examples**
Query without timezone:
``` sql
WITH toDateTime64('2020-01-01 10:20:30.999999999', 9) AS dt64
SELECT toStartOfNanosecond(dt64);
```
Result:
``` text
┌─────toStartOfNanosecond(dt64)─┐
│ 2020-01-01 10:20:30.999999999 │
└───────────────────────────────┘
```
Query with timezone:
``` sql
WITH toDateTime64('2020-01-01 10:20:30.999999999', 9) AS dt64
SELECT toStartOfNanosecond(dt64, 'Asia/Istanbul');
```
Result:
``` text
┌─toStartOfNanosecond(dt64, 'Asia/Istanbul')─┐
│ 2020-01-01 12:20:30.999999999 │
└────────────────────────────────────────────┘
```
**See also**
- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter.
## toStartOfFiveMinutes
Rounds down a date with time to the start of the five-minute interval.
@ -3953,6 +4115,43 @@ Result:
│ 2023-03-16 18:00:00.000 │
└─────────────────────────────────────────────────────────────────────────┘
```
## UTCTimestamp
Returns the current date and time at the moment of query analysis. The function is a constant expression.
:::note
This function gives the same result that `now('UTC')` would. It was added only for MySQL support and [`now`](#now-now) is the preferred usage.
:::
**Syntax**
```sql
UTCTimestamp()
```
Alias: `UTC_timestamp`.
**Returned value**
- Returns the current date and time at the moment of query analysis. [DateTime](../data-types/datetime.md).
**Example**
Query:
```sql
SELECT UTCTimestamp();
```
Result:
```response
┌──────UTCTimestamp()─┐
│ 2024-05-28 08:32:09 │
└─────────────────────┘
```
## timeDiff
Returns the difference between two dates or dates with time values. The difference is calculated in units of seconds. It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is preferred.

View File

@ -746,71 +746,6 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2);
└────────────────────────┴────────────────────────┘
```
## generateSnowflakeIDThreadMonotonic
Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID).
The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond.
For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes.
In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0.
This function behaves like `generateSnowflakeID` but gives no guarantee on counter monotony across different simultaneous requests.
Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.
```
0 1 2 3
0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤
|0| timestamp |
├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤
| | machine_id | machine_seq_num |
└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘
```
**Syntax**
``` sql
generateSnowflakeIDThreadMonotonic([expr])
```
**Arguments**
- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional.
**Returned value**
A value of type UInt64.
**Example**
First, create a table with a column of type UInt64, then insert a generated Snowflake ID into the table.
``` sql
CREATE TABLE tab (id UInt64) ENGINE = Memory;
INSERT INTO tab SELECT generateSnowflakeIDThreadMonotonic();
SELECT * FROM tab;
```
Result:
```response
┌──────────────────id─┐
│ 7199082832006627328 │
└─────────────────────┘
```
**Example with multiple Snowflake IDs generated per row**
```sql
SELECT generateSnowflakeIDThreadMonotonic(1), generateSnowflakeIDThreadMonotonic(2);
┌─generateSnowflakeIDThreadMonotonic(1)─┬─generateSnowflakeIDThreadMonotonic(2)─┐
│ 7199082940311945216 │ 7199082940316139520 │
└───────────────────────────────────────┴───────────────────────────────────────┘
```
## snowflakeToDateTime
Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](../data-types/datetime.md) format.

View File

@ -79,8 +79,6 @@ ORDER BY ts, event_type;
│ 2020-01-03 00:00:00 │ imp │ │ 2 │ 0 │
└─────────────────────┴────────────┴─────────┴────────────┴──────┘
SET allow_experimental_alter_materialized_view_structure=1;
ALTER TABLE mv MODIFY QUERY
SELECT toStartOfDay(ts) ts, event_type, browser,
count() events_cnt,
@ -178,7 +176,6 @@ SELECT * FROM mv;
└───┘
```
```sql
set allow_experimental_alter_materialized_view_structure=1;
ALTER TABLE mv MODIFY QUERY SELECT a * 2 as a FROM src_table;
INSERT INTO src_table (a) VALUES (3), (4);
SELECT * FROM mv;

View File

@ -206,6 +206,32 @@ Enables background data distribution when inserting data into distributed tables
SYSTEM START DISTRIBUTED SENDS [db.]<distributed_table_name> [ON CLUSTER cluster_name]
```
### STOP LISTEN
Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol.
However, if the corresponding protocol settings were not specified in the clickhouse-server configuration, this command will have no effect.
```sql
SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP WITH PROXY | TCP SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
```
- If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped.
- If `QUERIES ALL [EXCEPT .. [,..]]` modifier is specified, all protocols are stopped, unless specified with `EXCEPT` clause.
- If `QUERIES DEFAULT [EXCEPT .. [,..]]` modifier is specified, all default protocols are stopped, unless specified with `EXCEPT` clause.
- If `QUERIES CUSTOM [EXCEPT .. [,..]]` modifier is specified, all custom protocols are stopped, unless specified with `EXCEPT` clause.
### START LISTEN
Allows new connections to be established on the specified protocols.
However, if the server on the specified port and protocol was not stopped using the SYSTEM STOP LISTEN command, this command will have no effect.
```sql
SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP WITH PROXY | TCP SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
```
## Managing MergeTree Tables
ClickHouse can manage background processes in [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables.
@ -463,30 +489,16 @@ Will do sync syscall.
SYSTEM SYNC FILE CACHE [ON CLUSTER cluster_name]
```
### UNLOAD PRIMARY KEY
## SYSTEM STOP LISTEN
Closes the socket and gracefully terminates the existing connections to the server on the specified port with the specified protocol.
However, if the corresponding protocol settings were not specified in the clickhouse-server configuration, this command will have no effect.
Unload the primary keys for the given table or for all tables.
```sql
SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP WITH PROXY | TCP SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
SYSTEM UNLOAD PRIMARY KEY [db.]name
```
- If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped.
- If `QUERIES ALL [EXCEPT .. [,..]]` modifier is specified, all protocols are stopped, unless specified with `EXCEPT` clause.
- If `QUERIES DEFAULT [EXCEPT .. [,..]]` modifier is specified, all default protocols are stopped, unless specified with `EXCEPT` clause.
- If `QUERIES CUSTOM [EXCEPT .. [,..]]` modifier is specified, all custom protocols are stopped, unless specified with `EXCEPT` clause.
## SYSTEM START LISTEN
Allows new connections to be established on the specified protocols.
However, if the server on the specified port and protocol was not stopped using the SYSTEM STOP LISTEN command, this command will have no effect.
```sql
SYSTEM START LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QUERIES CUSTOM | TCP | TCP WITH PROXY | TCP SECURE | HTTP | HTTPS | MYSQL | GRPC | POSTGRESQL | PROMETHEUS | CUSTOM 'protocol']
SYSTEM UNLOAD PRIMARY KEY
```
## Managing Refreshable Materialized Views {#refreshable-materialized-views}
@ -495,7 +507,7 @@ Commands to control background tasks performed by [Refreshable Materialized View
Keep an eye on [`system.view_refreshes`](../../operations/system-tables/view_refreshes.md) while using them.
### SYSTEM REFRESH VIEW
### REFRESH VIEW
Trigger an immediate out-of-schedule refresh of a given view.
@ -503,7 +515,7 @@ Trigger an immediate out-of-schedule refresh of a given view.
SYSTEM REFRESH VIEW [db.]name
```
### SYSTEM STOP VIEW, SYSTEM STOP VIEWS
### STOP VIEW, STOP VIEWS
Disable periodic refreshing of the given view or all refreshable views. If a refresh is in progress, cancel it too.
@ -514,7 +526,7 @@ SYSTEM STOP VIEW [db.]name
SYSTEM STOP VIEWS
```
### SYSTEM START VIEW, SYSTEM START VIEWS
### START VIEW, START VIEWS
Enable periodic refreshing for the given view or all refreshable views. No immediate refresh is triggered.
@ -525,22 +537,10 @@ SYSTEM START VIEW [db.]name
SYSTEM START VIEWS
```
### SYSTEM CANCEL VIEW
### CANCEL VIEW
If there's a refresh in progress for the given view, interrupt and cancel it. Otherwise do nothing.
```sql
SYSTEM CANCEL VIEW [db.]name
```
### SYSTEM UNLOAD PRIMARY KEY
Unload the primary keys for the given table or for all tables.
```sql
SYSTEM UNLOAD PRIMARY KEY [db.]name
```
```sql
SYSTEM UNLOAD PRIMARY KEY
```

View File

@ -0,0 +1,124 @@
#pragma once
#include <IO/Operators.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Analyzer/FunctionNode.h>
namespace DB
{
class ExpressionsStack
{
public:
void push(const QueryTreeNodePtr & node)
{
if (node->hasAlias())
{
const auto & node_alias = node->getAlias();
alias_name_to_expressions[node_alias].push_back(node);
}
if (const auto * function = node->as<FunctionNode>())
{
if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->getFunctionName()))
++aggregate_functions_counter;
}
expressions.emplace_back(node);
}
void pop()
{
const auto & top_expression = expressions.back();
const auto & top_expression_alias = top_expression->getAlias();
if (!top_expression_alias.empty())
{
auto it = alias_name_to_expressions.find(top_expression_alias);
auto & alias_expressions = it->second;
alias_expressions.pop_back();
if (alias_expressions.empty())
alias_name_to_expressions.erase(it);
}
if (const auto * function = top_expression->as<FunctionNode>())
{
if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->getFunctionName()))
--aggregate_functions_counter;
}
expressions.pop_back();
}
[[maybe_unused]] const QueryTreeNodePtr & getRoot() const
{
return expressions.front();
}
const QueryTreeNodePtr & getTop() const
{
return expressions.back();
}
[[maybe_unused]] bool hasExpressionWithAlias(const std::string & alias) const
{
return alias_name_to_expressions.contains(alias);
}
bool hasAggregateFunction() const
{
return aggregate_functions_counter > 0;
}
QueryTreeNodePtr getExpressionWithAlias(const std::string & alias) const
{
auto expression_it = alias_name_to_expressions.find(alias);
if (expression_it == alias_name_to_expressions.end())
return {};
return expression_it->second.front();
}
[[maybe_unused]] size_t size() const
{
return expressions.size();
}
bool empty() const
{
return expressions.empty();
}
void dump(WriteBuffer & buffer) const
{
buffer << expressions.size() << '\n';
for (const auto & expression : expressions)
{
buffer << "Expression ";
buffer << expression->formatASTForErrorMessage();
const auto & alias = expression->getAlias();
if (!alias.empty())
buffer << " alias " << alias;
buffer << '\n';
}
}
[[maybe_unused]] String dump() const
{
WriteBufferFromOwnString buffer;
dump(buffer);
return buffer.str();
}
private:
QueryTreeNodes expressions;
size_t aggregate_functions_counter = 0;
std::unordered_map<std::string, QueryTreeNodes> alias_name_to_expressions;
};
}

View File

@ -0,0 +1,195 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Identifier.h>
namespace DB
{
/// Identifier lookup context
enum class IdentifierLookupContext : uint8_t
{
EXPRESSION = 0,
FUNCTION,
TABLE_EXPRESSION,
};
inline const char * toString(IdentifierLookupContext identifier_lookup_context)
{
switch (identifier_lookup_context)
{
case IdentifierLookupContext::EXPRESSION: return "EXPRESSION";
case IdentifierLookupContext::FUNCTION: return "FUNCTION";
case IdentifierLookupContext::TABLE_EXPRESSION: return "TABLE_EXPRESSION";
}
}
inline const char * toStringLowercase(IdentifierLookupContext identifier_lookup_context)
{
switch (identifier_lookup_context)
{
case IdentifierLookupContext::EXPRESSION: return "expression";
case IdentifierLookupContext::FUNCTION: return "function";
case IdentifierLookupContext::TABLE_EXPRESSION: return "table expression";
}
}
/** Structure that represent identifier lookup during query analysis.
* Lookup can be in query expression, function, table context.
*/
struct IdentifierLookup
{
Identifier identifier;
IdentifierLookupContext lookup_context;
bool isExpressionLookup() const
{
return lookup_context == IdentifierLookupContext::EXPRESSION;
}
bool isFunctionLookup() const
{
return lookup_context == IdentifierLookupContext::FUNCTION;
}
bool isTableExpressionLookup() const
{
return lookup_context == IdentifierLookupContext::TABLE_EXPRESSION;
}
String dump() const
{
return identifier.getFullName() + ' ' + toString(lookup_context);
}
};
inline bool operator==(const IdentifierLookup & lhs, const IdentifierLookup & rhs)
{
return lhs.identifier.getFullName() == rhs.identifier.getFullName() && lhs.lookup_context == rhs.lookup_context;
}
[[maybe_unused]] inline bool operator!=(const IdentifierLookup & lhs, const IdentifierLookup & rhs)
{
return !(lhs == rhs);
}
struct IdentifierLookupHash
{
size_t operator()(const IdentifierLookup & identifier_lookup) const
{
return std::hash<std::string>()(identifier_lookup.identifier.getFullName()) ^ static_cast<uint8_t>(identifier_lookup.lookup_context);
}
};
enum class IdentifierResolvePlace : UInt8
{
NONE = 0,
EXPRESSION_ARGUMENTS,
ALIASES,
JOIN_TREE,
/// Valid only for table lookup
CTE,
/// Valid only for table lookup
DATABASE_CATALOG
};
inline const char * toString(IdentifierResolvePlace resolved_identifier_place)
{
switch (resolved_identifier_place)
{
case IdentifierResolvePlace::NONE: return "NONE";
case IdentifierResolvePlace::EXPRESSION_ARGUMENTS: return "EXPRESSION_ARGUMENTS";
case IdentifierResolvePlace::ALIASES: return "ALIASES";
case IdentifierResolvePlace::JOIN_TREE: return "JOIN_TREE";
case IdentifierResolvePlace::CTE: return "CTE";
case IdentifierResolvePlace::DATABASE_CATALOG: return "DATABASE_CATALOG";
}
}
struct IdentifierResolveResult
{
IdentifierResolveResult() = default;
QueryTreeNodePtr resolved_identifier;
IdentifierResolvePlace resolve_place = IdentifierResolvePlace::NONE;
bool resolved_from_parent_scopes = false;
[[maybe_unused]] bool isResolved() const
{
return resolve_place != IdentifierResolvePlace::NONE;
}
[[maybe_unused]] bool isResolvedFromParentScopes() const
{
return resolved_from_parent_scopes;
}
[[maybe_unused]] bool isResolvedFromExpressionArguments() const
{
return resolve_place == IdentifierResolvePlace::EXPRESSION_ARGUMENTS;
}
[[maybe_unused]] bool isResolvedFromAliases() const
{
return resolve_place == IdentifierResolvePlace::ALIASES;
}
[[maybe_unused]] bool isResolvedFromJoinTree() const
{
return resolve_place == IdentifierResolvePlace::JOIN_TREE;
}
[[maybe_unused]] bool isResolvedFromCTEs() const
{
return resolve_place == IdentifierResolvePlace::CTE;
}
void dump(WriteBuffer & buffer) const
{
if (!resolved_identifier)
{
buffer << "unresolved";
return;
}
buffer << resolved_identifier->formatASTForErrorMessage() << " place " << toString(resolve_place) << " resolved from parent scopes " << resolved_from_parent_scopes;
}
[[maybe_unused]] String dump() const
{
WriteBufferFromOwnString buffer;
dump(buffer);
return buffer.str();
}
};
struct IdentifierResolveState
{
IdentifierResolveResult resolve_result;
bool cyclic_identifier_resolve = false;
};
struct IdentifierResolveSettings
{
/// Allow to check join tree during identifier resolution
bool allow_to_check_join_tree = true;
/// Allow to check CTEs during table identifier resolution
bool allow_to_check_cte = true;
/// Allow to check parent scopes during identifier resolution
bool allow_to_check_parent_scopes = true;
/// Allow to check database catalog during table identifier resolution
bool allow_to_check_database_catalog = true;
/// Allow to resolve subquery during identifier resolution
bool allow_to_resolve_subquery_during_identifier_resolution = true;
};
}

View File

@ -0,0 +1,184 @@
#include <Analyzer/Resolve/IdentifierResolveScope.h>
#include <Interpreters/Context.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/UnionNode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
IdentifierResolveScope::IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_)
: scope_node(std::move(scope_node_))
, parent_scope(parent_scope_)
{
if (parent_scope)
{
subquery_depth = parent_scope->subquery_depth;
context = parent_scope->context;
projection_mask_map = parent_scope->projection_mask_map;
}
else
projection_mask_map = std::make_shared<std::map<IQueryTreeNode::Hash, size_t>>();
if (auto * union_node = scope_node->as<UnionNode>())
{
context = union_node->getContext();
}
else if (auto * query_node = scope_node->as<QueryNode>())
{
context = query_node->getContext();
group_by_use_nulls = context->getSettingsRef().group_by_use_nulls &&
(query_node->isGroupByWithGroupingSets() || query_node->isGroupByWithRollup() || query_node->isGroupByWithCube());
}
if (context)
join_use_nulls = context->getSettingsRef().join_use_nulls;
else if (parent_scope)
join_use_nulls = parent_scope->join_use_nulls;
aliases.alias_name_to_expression_node = &aliases.alias_name_to_expression_node_before_group_by;
}
[[maybe_unused]] const IdentifierResolveScope * IdentifierResolveScope::getNearestQueryScope() const
{
const IdentifierResolveScope * scope_to_check = this;
while (scope_to_check != nullptr)
{
if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY)
break;
scope_to_check = scope_to_check->parent_scope;
}
return scope_to_check;
}
IdentifierResolveScope * IdentifierResolveScope::getNearestQueryScope()
{
IdentifierResolveScope * scope_to_check = this;
while (scope_to_check != nullptr)
{
if (scope_to_check->scope_node->getNodeType() == QueryTreeNodeType::QUERY)
break;
scope_to_check = scope_to_check->parent_scope;
}
return scope_to_check;
}
AnalysisTableExpressionData & IdentifierResolveScope::getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node)
{
auto it = table_expression_node_to_data.find(table_expression_node);
if (it == table_expression_node_to_data.end())
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Table expression {} data must be initialized. In scope {}",
table_expression_node->formatASTForErrorMessage(),
scope_node->formatASTForErrorMessage());
}
return it->second;
}
const AnalysisTableExpressionData & IdentifierResolveScope::getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node) const
{
auto it = table_expression_node_to_data.find(table_expression_node);
if (it == table_expression_node_to_data.end())
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Table expression {} data must be initialized. In scope {}",
table_expression_node->formatASTForErrorMessage(),
scope_node->formatASTForErrorMessage());
}
return it->second;
}
void IdentifierResolveScope::pushExpressionNode(const QueryTreeNodePtr & node)
{
bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction();
expressions_in_resolve_process_stack.push(node);
if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction())
aliases.alias_name_to_expression_node = &aliases.alias_name_to_expression_node_before_group_by;
}
void IdentifierResolveScope::popExpressionNode()
{
bool had_aggregate_function = expressions_in_resolve_process_stack.hasAggregateFunction();
expressions_in_resolve_process_stack.pop();
if (group_by_use_nulls && had_aggregate_function != expressions_in_resolve_process_stack.hasAggregateFunction())
aliases.alias_name_to_expression_node = &aliases.alias_name_to_expression_node_after_group_by;
}
/// Dump identifier resolve scope
[[maybe_unused]] void IdentifierResolveScope::dump(WriteBuffer & buffer) const
{
buffer << "Scope node " << scope_node->formatASTForErrorMessage() << '\n';
buffer << "Identifier lookup to resolve state " << identifier_lookup_to_resolve_state.size() << '\n';
for (const auto & [identifier, state] : identifier_lookup_to_resolve_state)
{
buffer << "Identifier " << identifier.dump() << " resolve result ";
state.resolve_result.dump(buffer);
buffer << '\n';
}
buffer << "Expression argument name to node " << expression_argument_name_to_node.size() << '\n';
for (const auto & [alias_name, node] : expression_argument_name_to_node)
buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n';
buffer << "Alias name to expression node table size " << aliases.alias_name_to_expression_node->size() << '\n';
for (const auto & [alias_name, node] : *aliases.alias_name_to_expression_node)
buffer << "Alias name " << alias_name << " expression node " << node->dumpTree() << '\n';
buffer << "Alias name to function node table size " << aliases.alias_name_to_lambda_node.size() << '\n';
for (const auto & [alias_name, node] : aliases.alias_name_to_lambda_node)
buffer << "Alias name " << alias_name << " lambda node " << node->formatASTForErrorMessage() << '\n';
buffer << "Alias name to table expression node table size " << aliases.alias_name_to_table_expression_node.size() << '\n';
for (const auto & [alias_name, node] : aliases.alias_name_to_table_expression_node)
buffer << "Alias name " << alias_name << " node " << node->formatASTForErrorMessage() << '\n';
buffer << "CTE name to query node table size " << cte_name_to_query_node.size() << '\n';
for (const auto & [cte_name, node] : cte_name_to_query_node)
buffer << "CTE name " << cte_name << " node " << node->formatASTForErrorMessage() << '\n';
buffer << "WINDOW name to window node table size " << window_name_to_window_node.size() << '\n';
for (const auto & [window_name, node] : window_name_to_window_node)
buffer << "CTE name " << window_name << " node " << node->formatASTForErrorMessage() << '\n';
buffer << "Nodes with duplicated aliases size " << aliases.nodes_with_duplicated_aliases.size() << '\n';
for (const auto & node : aliases.nodes_with_duplicated_aliases)
buffer << "Alias name " << node->getAlias() << " node " << node->formatASTForErrorMessage() << '\n';
buffer << "Expression resolve process stack " << '\n';
expressions_in_resolve_process_stack.dump(buffer);
buffer << "Table expressions in resolve process size " << table_expressions_in_resolve_process.size() << '\n';
for (const auto & node : table_expressions_in_resolve_process)
buffer << "Table expression " << node->formatASTForErrorMessage() << '\n';
buffer << "Non cached identifier lookups during expression resolve " << non_cached_identifier_lookups_during_expression_resolve.size() << '\n';
for (const auto & identifier_lookup : non_cached_identifier_lookups_during_expression_resolve)
buffer << "Identifier lookup " << identifier_lookup.dump() << '\n';
buffer << "Table expression node to data " << table_expression_node_to_data.size() << '\n';
for (const auto & [table_expression_node, table_expression_data] : table_expression_node_to_data)
buffer << "Table expression node " << table_expression_node->formatASTForErrorMessage() << " data " << table_expression_data.dump() << '\n';
buffer << "Use identifier lookup to result cache " << use_identifier_lookup_to_result_cache << '\n';
buffer << "Subquery depth " << subquery_depth << '\n';
}
[[maybe_unused]] String IdentifierResolveScope::dump() const
{
WriteBufferFromOwnString buffer;
dump(buffer);
return buffer.str();
}
}

View File

@ -0,0 +1,231 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Resolve/IdentifierLookup.h>
#include <Analyzer/Resolve/ScopeAliases.h>
#include <Analyzer/Resolve/TableExpressionData.h>
#include <Analyzer/Resolve/ExpressionsStack.h>
namespace DB
{
/** Projection names is name of query tree node that is used in projection part of query node.
* Example: SELECT id FROM test_table;
* `id` is projection name of column node
*
* Example: SELECT id AS id_alias FROM test_table;
* `id_alias` is projection name of column node
*
* Calculation of projection names is done during expression nodes resolution. This is done this way
* because after identifier node is resolved we lose information about identifier name. We could
* potentially save this information in query tree node itself, but that would require to clone it in some cases.
* Example: SELECT big_scalar_subquery AS a, a AS b, b AS c;
* All 3 nodes in projection are the same big_scalar_subquery, but they have different projection names.
* If we want to save it in query tree node, we have to clone subquery node that could lead to performance degradation.
*
* Possible solution is to separate query node metadata and query node content. So only node metadata could be cloned
* if we want to change projection name. This solution does not seem to be easy for client of query tree because projection
* name will be part of interface. If we potentially could hide projection names calculation in analyzer without introducing additional
* changes in query tree structure that would be preferable.
*
* Currently each resolve method returns projection names array. Resolve method must compute projection names of node.
* If node is resolved as list node this is case for `untuple` function or `matcher` result projection names array must contain projection names
* for result nodes.
* If node is not resolved as list node, projection names array contain single projection name for node.
*
* Rules for projection names:
* 1. If node has alias. It is node projection name.
* Except scenario where `untuple` function has alias. Example: SELECT untuple(expr) AS alias, alias.
*
* 2. For constant it is constant value string representation.
*
* 3. For identifier:
* If identifier is resolved from JOIN TREE, we want to remove additional identifier qualifications.
* Example: SELECT default.test_table.id FROM test_table.
* Result projection name is `id`.
*
* Example: SELECT t1.id FROM test_table_1 AS t1, test_table_2 AS t2
* In example both test_table_1, test_table_2 have `id` column.
* In such case projection name is `t1.id` because if additional qualification is removed then column projection name `id` will be ambiguous.
*
* Example: SELECT default.test_table_1.id FROM test_table_1 AS t1, test_table_2 AS t2
* In such case projection name is `test_table_1.id` because we remove unnecessary database qualification, but table name qualification cannot be removed
* because otherwise column projection name `id` will be ambiguous.
*
* If identifier is not resolved from JOIN TREE. Identifier name is projection name.
* Except scenario where `untuple` function resolved using identifier. Example: SELECT untuple(expr) AS alias, alias.
* Example: SELECT sum(1, 1) AS value, value.
* In such case both nodes have `value` projection names.
*
* Example: SELECT id AS value, value FROM test_table.
* In such case both nodes have have `value` projection names.
*
* Special case is `untuple` function. If `untuple` function specified with alias, then result nodes will have alias.tuple_column_name projection names.
* Example: SELECT cast(tuple(1), 'Tuple(id UInt64)') AS value, untuple(value) AS a;
* Result projection names are `value`, `a.id`.
*
* If `untuple` function does not have alias then result nodes will have `tupleElement(untuple_expression_projection_name, 'tuple_column_name') projection names.
*
* Example: SELECT cast(tuple(1), 'Tuple(id UInt64)') AS value, untuple(value);
* Result projection names are `value`, `tupleElement(value, 'id')`;
*
* 4. For function:
* Projection name consists from function_name(parameters_projection_names)(arguments_projection_names).
* Additionally if function is window function. Window node projection name is used with OVER clause.
* Example: function_name (parameters_names)(argument_projection_names) OVER window_name;
* Example: function_name (parameters_names)(argument_projection_names) OVER (PARTITION BY id ORDER BY id).
* Example: function_name (parameters_names)(argument_projection_names) OVER (window_name ORDER BY id).
*
* 5. For lambda:
* If it is standalone lambda that returns single expression, function projection name is used.
* Example: WITH (x -> x + 1) AS lambda SELECT lambda(1).
* Projection name is `lambda(1)`.
*
* If is it standalone lambda that returns list, projection names of list nodes are used.
* Example: WITH (x -> *) AS lambda SELECT lambda(1) FROM test_table;
* If test_table has two columns `id`, `value`. Then result projection names are `id`, `value`.
*
* If lambda is argument of function.
* Then projection name consists from lambda(tuple(lambda_arguments)(lambda_body_projection_name));
*
* 6. For matcher:
* Matched nodes projection names are used as matcher projection names.
*
* Matched nodes must be qualified if needed.
* Example: SELECT * FROM test_table_1 AS t1, test_table_2 AS t2.
* In example table test_table_1 and test_table_2 both have `id`, `value` columns.
* Matched nodes after unqualified matcher resolve must be qualified to avoid ambiguous projection names.
* Result projection names must be `t1.id`, `t1.value`, `t2.id`, `t2.value`.
*
* There are special cases
* 1. For lambda inside APPLY matcher transformer:
* Example: SELECT * APPLY x -> toString(x) FROM test_table.
* In such case lambda argument projection name `x` will be replaced by matched node projection name.
* If table has two columns `id` and `value`. Then result projection names are `toString(id)`, `toString(value)`;
*
* 2. For unqualified matcher when JOIN tree contains JOIN with USING.
* Example: SELECT * FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING(id);
* Result projection names must be `id`, `t1.value`, `t2.value`.
*
* 7. For subquery:
* For subquery projection name consists of `_subquery_` prefix and implementation specific unique number suffix.
* Example: SELECT (SELECT 1), (SELECT 1 UNION DISTINCT SELECT 1);
* Result projection name can be `_subquery_1`, `subquery_2`;
*
* 8. For table:
* Table node can be used in expression context only as right argument of IN function. In that case identifier is used
* as table node projection name.
* Example: SELECT id IN test_table FROM test_table;
* Result projection name is `in(id, test_table)`.
*/
using ProjectionName = String;
using ProjectionNames = std::vector<ProjectionName>;
constexpr auto PROJECTION_NAME_PLACEHOLDER = "__projection_name_placeholder";
struct IdentifierResolveScope
{
/// Construct identifier resolve scope using scope node, and parent scope
IdentifierResolveScope(QueryTreeNodePtr scope_node_, IdentifierResolveScope * parent_scope_);
QueryTreeNodePtr scope_node;
IdentifierResolveScope * parent_scope = nullptr;
ContextPtr context;
/// Identifier lookup to result
std::unordered_map<IdentifierLookup, IdentifierResolveState, IdentifierLookupHash> identifier_lookup_to_resolve_state;
/// Argument can be expression like constant, column, function or table expression
std::unordered_map<std::string, QueryTreeNodePtr> expression_argument_name_to_node;
ScopeAliases aliases;
/// Table column name to column node. Valid only during table ALIAS columns resolve.
ColumnNameToColumnNodeMap column_name_to_column_node;
/// CTE name to query node
std::unordered_map<std::string, QueryTreeNodePtr> cte_name_to_query_node;
/// Window name to window node
std::unordered_map<std::string, QueryTreeNodePtr> window_name_to_window_node;
/// Current scope expression in resolve process stack
ExpressionsStack expressions_in_resolve_process_stack;
/// Table expressions in resolve process
std::unordered_set<const IQueryTreeNode *> table_expressions_in_resolve_process;
/// Current scope expression
std::unordered_set<IdentifierLookup, IdentifierLookupHash> non_cached_identifier_lookups_during_expression_resolve;
/// Table expression node to data
std::unordered_map<QueryTreeNodePtr, AnalysisTableExpressionData> table_expression_node_to_data;
QueryTreeNodePtrWithHashIgnoreTypesSet nullable_group_by_keys;
/// Here we count the number of nullable GROUP BY keys we met resolving expression.
/// E.g. for a query `SELECT tuple(tuple(number)) FROM numbers(10) GROUP BY (number, tuple(number)) with cube`
/// both `number` and `tuple(number)` would be in nullable_group_by_keys.
/// But when we resolve `tuple(tuple(number))` we should figure out that `tuple(number)` is already a key,
/// and we should not convert `number` to nullable.
size_t found_nullable_group_by_key_in_scope = 0;
/** It's possible that after a JOIN, a column in the projection has a type different from the column in the source table.
* (For example, after join_use_nulls or USING column casted to supertype)
* However, the column in the projection still refers to the table as its source.
* This map is used to revert these columns back to their original columns in the source table.
*/
QueryTreeNodePtrWithHashMap<QueryTreeNodePtr> join_columns_with_changed_types;
/// Use identifier lookup to result cache
bool use_identifier_lookup_to_result_cache = true;
/// Apply nullability to aggregation keys
bool group_by_use_nulls = false;
/// Join retutns NULLs instead of default values
bool join_use_nulls = false;
/// JOINs count
size_t joins_count = 0;
/// Subquery depth
size_t subquery_depth = 0;
/** Scope join tree node for expression.
* Valid only during analysis construction for single expression.
*/
QueryTreeNodePtr expression_join_tree_node;
/// Node hash to mask id map
std::shared_ptr<std::map<IQueryTreeNode::Hash, size_t>> projection_mask_map;
struct ResolvedFunctionsCache
{
FunctionOverloadResolverPtr resolver;
FunctionBasePtr function_base;
};
std::map<IQueryTreeNode::Hash, ResolvedFunctionsCache> functions_cache;
[[maybe_unused]] const IdentifierResolveScope * getNearestQueryScope() const;
IdentifierResolveScope * getNearestQueryScope();
AnalysisTableExpressionData & getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node);
const AnalysisTableExpressionData & getTableExpressionDataOrThrow(const QueryTreeNodePtr & table_expression_node) const;
void pushExpressionNode(const QueryTreeNodePtr & node);
void popExpressionNode();
/// Dump identifier resolve scope
[[maybe_unused]] void dump(WriteBuffer & buffer) const;
[[maybe_unused]] String dump() const;
};
}

View File

@ -0,0 +1,22 @@
#include <Analyzer/Passes/QueryAnalysisPass.h>
#include <Analyzer/Resolve/QueryAnalyzer.h>
#include <Analyzer/createUniqueTableAliases.h>
namespace DB
{
QueryAnalysisPass::QueryAnalysisPass(QueryTreeNodePtr table_expression_, bool only_analyze_)
: table_expression(std::move(table_expression_))
, only_analyze(only_analyze_)
{}
QueryAnalysisPass::QueryAnalysisPass(bool only_analyze_) : only_analyze(only_analyze_) {}
void QueryAnalysisPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context)
{
QueryAnalyzer analyzer(only_analyze);
analyzer.resolve(query_tree_node, table_expression, context);
createUniqueTableAliases(query_tree_node, table_expression, context);
}
}

View File

@ -0,0 +1,378 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Analyzer/HashUtils.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Resolve/IdentifierLookup.h>
#include <Core/Joins.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/NullsAction.h>
namespace DB
{
struct GetColumnsOptions;
struct IdentifierResolveScope;
struct AnalysisTableExpressionData;
class QueryExpressionsAliasVisitor ;
class QueryNode;
class JoinNode;
class ColumnNode;
using ProjectionName = String;
using ProjectionNames = std::vector<ProjectionName>;
struct Settings;
/** Query analyzer implementation overview. Please check documentation in QueryAnalysisPass.h first.
* And additional documentation for each method, where special cases are described in detail.
*
* Each node in query must be resolved. For each query tree node resolved state is specific.
*
* For constant node no resolve process exists, it is resolved during construction.
*
* For table node no resolve process exists, it is resolved during construction.
*
* For function node to be resolved parameters and arguments must be resolved, function node must be initialized with concrete aggregate or
* non aggregate function and with result type.
*
* For lambda node there can be 2 different cases.
* 1. Standalone: WITH (x -> x + 1) AS lambda SELECT lambda(1); Such lambdas are inlined in query tree during query analysis pass.
* 2. Function arguments: WITH (x -> x + 1) AS lambda SELECT arrayMap(lambda, [1, 2, 3]); For such lambda resolution must
* set concrete lambda arguments (initially they are identifier nodes) and resolve lambda expression body.
*
* For query node resolve process must resolve all its inner nodes.
*
* For matcher node resolve process must replace it with matched nodes.
*
* For identifier node resolve process must replace it with concrete non identifier node. This part is most complex because
* for identifier resolution scopes and identifier lookup context play important part.
*
* ClickHouse SQL support lexical scoping for identifier resolution. Scope can be defined by query node or by expression node.
* Expression nodes that can define scope are lambdas and table ALIAS columns.
*
* Identifier lookup context can be expression, function, table.
*
* Examples: WITH (x -> x + 1) as func SELECT func() FROM func; During function `func` resolution identifier lookup is performed
* in function context.
*
* If there are no information of identifier context rules are following:
* 1. Try to resolve identifier in expression context.
* 2. Try to resolve identifier in function context, if it is allowed. Example: SELECT func(arguments); Here func identifier cannot be resolved in function context
* because query projection does not support that.
* 3. Try to resolve identifier in table context, if it is allowed. Example: SELECT table; Here table identifier cannot be resolved in function context
* because query projection does not support that.
*
* TODO: This does not supported properly before, because matchers could not be resolved from aliases.
*
* Identifiers are resolved with following rules:
* Resolution starts with current scope.
* 1. Try to resolve identifier from expression scope arguments. Lambda expression arguments are greatest priority.
* 2. Try to resolve identifier from aliases.
* 3. Try to resolve identifier from join tree if scope is query, or if there are registered table columns in scope.
* Steps 2 and 3 can be changed using prefer_column_name_to_alias setting.
* 4. If it is table lookup, try to resolve identifier from CTE.
* If identifier could not be resolved in current scope, resolution must be continued in parent scopes.
* 5. Try to resolve identifier from parent scopes.
*
* Additional rules about aliases and scopes.
* 1. Parent scope cannot refer alias from child scope.
* 2. Child scope can refer to alias in parent scope.
*
* Example: SELECT arrayMap(x -> x + 1 AS a, [1,2,3]), a; Identifier a is unknown in parent scope.
* Example: SELECT a FROM (SELECT 1 as a); Here we do not refer to alias a from child query scope. But we query it projection result, similar to tables.
* Example: WITH 1 as a SELECT (SELECT a) as b; Here in child scope identifier a is resolved using alias from parent scope.
*
* Additional rules about identifier binding.
* Bind for identifier to entity means that identifier first part match some node during analysis.
* If other parts of identifier cannot be resolved in that node, exception must be thrown.
*
* Example:
* CREATE TABLE test_table (id UInt64, compound_value Tuple(value UInt64)) ENGINE=TinyLog;
* SELECT compound_value.value, 1 AS compound_value FROM test_table;
* Identifier first part compound_value bound to entity with alias compound_value, but nested identifier part cannot be resolved from entity,
* lookup should not be continued, and exception must be thrown because if lookup continues that way identifier can be resolved from join tree.
*
* TODO: This was not supported properly before analyzer because nested identifier could not be resolved from alias.
*
* More complex example:
* CREATE TABLE test_table (id UInt64, value UInt64) ENGINE=TinyLog;
* WITH cast(('Value'), 'Tuple (value UInt64') AS value SELECT (SELECT value FROM test_table);
* Identifier first part value bound to test_table column value, but nested identifier part cannot be resolved from it,
* lookup should not be continued, and exception must be thrown because if lookup continues identifier can be resolved from parent scope.
*
* TODO: Update exception messages
* TODO: Table identifiers with optional UUID.
* TODO: Lookup functions arrayReduce(sum, [1, 2, 3]);
* TODO: Support function identifier resolve from parent query scope, if lambda in parent scope does not capture any columns.
*/
class QueryAnalyzer
{
public:
explicit QueryAnalyzer(bool only_analyze_);
~QueryAnalyzer();
void resolve(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, ContextPtr context);
private:
/// Utility functions
static bool isExpressionNodeType(QueryTreeNodeType node_type);
static bool isFunctionExpressionNodeType(QueryTreeNodeType node_type);
static bool isSubqueryNodeType(QueryTreeNodeType node_type);
static bool isTableExpressionNodeType(QueryTreeNodeType node_type);
static DataTypePtr getExpressionNodeResultTypeOrNull(const QueryTreeNodePtr & query_tree_node);
static ProjectionName calculateFunctionProjectionName(const QueryTreeNodePtr & function_node,
const ProjectionNames & parameters_projection_names,
const ProjectionNames & arguments_projection_names);
static ProjectionName calculateWindowProjectionName(const QueryTreeNodePtr & window_node,
const QueryTreeNodePtr & parent_window_node,
const String & parent_window_name,
const ProjectionNames & partition_by_projection_names,
const ProjectionNames & order_by_projection_names,
const ProjectionName & frame_begin_offset_projection_name,
const ProjectionName & frame_end_offset_projection_name);
static ProjectionName calculateSortColumnProjectionName(const QueryTreeNodePtr & sort_column_node,
const ProjectionName & sort_expression_projection_name,
const ProjectionName & fill_from_expression_projection_name,
const ProjectionName & fill_to_expression_projection_name,
const ProjectionName & fill_step_expression_projection_name);
static void collectCompoundExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier,
const DataTypePtr & compound_expression_type,
const Identifier & valid_identifier_prefix,
std::unordered_set<Identifier> & valid_identifiers_result);
static void collectTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier,
const QueryTreeNodePtr & table_expression,
const AnalysisTableExpressionData & table_expression_data,
std::unordered_set<Identifier> & valid_identifiers_result);
static void collectScopeValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier,
const IdentifierResolveScope & scope,
bool allow_expression_identifiers,
bool allow_function_identifiers,
bool allow_table_expression_identifiers,
std::unordered_set<Identifier> & valid_identifiers_result);
static void collectScopeWithParentScopesValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier,
const IdentifierResolveScope & scope,
bool allow_expression_identifiers,
bool allow_function_identifiers,
bool allow_table_expression_identifiers,
std::unordered_set<Identifier> & valid_identifiers_result);
static std::vector<String> collectIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set<Identifier> & valid_identifiers);
static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path);
QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context);
void evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & query_tree_node, IdentifierResolveScope & scope);
static void mergeWindowWithParentWindow(const QueryTreeNodePtr & window_node, const QueryTreeNodePtr & parent_window_node, IdentifierResolveScope & scope);
void replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_list, const QueryTreeNodes & projection_nodes, IdentifierResolveScope & scope);
static void convertLimitOffsetExpression(QueryTreeNodePtr & expression_node, const String & expression_description, IdentifierResolveScope & scope);
static void validateTableExpressionModifiers(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope);
static void validateJoinTableExpressionWithoutAlias(const QueryTreeNodePtr & join_node, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope);
static void checkDuplicateTableNamesOrAlias(const QueryTreeNodePtr & join_node, QueryTreeNodePtr & left_table_expr, QueryTreeNodePtr & right_table_expr, IdentifierResolveScope & scope);
static std::pair<bool, UInt64> recursivelyCollectMaxOrdinaryExpressions(QueryTreeNodePtr & node, QueryTreeNodes & into);
static void expandGroupByAll(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);
static std::optional<JoinTableSide> getColumnSideFromJoinTree(const QueryTreeNodePtr & resolved_identifier, const JoinNode & join_node);
static QueryTreeNodePtr convertJoinedColumnTypeToNullIfNeeded(
const QueryTreeNodePtr & resolved_identifier,
const JoinKind & join_kind,
std::optional<JoinTableSide> resolved_side,
IdentifierResolveScope & scope);
/// Resolve identifier functions
static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context);
QueryTreeNodePtr tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier,
size_t identifier_bind_size,
const QueryTreeNodePtr & compound_expression,
String compound_expression_source,
IdentifierResolveScope & scope,
bool can_be_not_found = false);
QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope);
static bool tryBindIdentifierToAliases(const IdentifierLookup & identifier_lookup, const IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup,
IdentifierResolveScope & scope,
IdentifierResolveSettings identifier_resolve_settings);
QueryTreeNodePtr tryResolveIdentifierFromTableColumns(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope);
static bool tryBindIdentifierToTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
const IdentifierResolveScope & scope);
static bool tryBindIdentifierToTableExpressions(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
const IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromJoin(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr matchArrayJoinSubcolumns(
const QueryTreeNodePtr & array_join_column_inner_expression,
const ColumnNode & array_join_column_expression_typed,
const QueryTreeNodePtr & resolved_expression,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveExpressionFromArrayJoinExpressions(const QueryTreeNodePtr & resolved_expression,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromArrayJoin(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & table_expression_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromJoinTreeNode(const IdentifierLookup & identifier_lookup,
const QueryTreeNodePtr & join_tree_node,
IdentifierResolveScope & scope);
QueryTreeNodePtr tryResolveIdentifierFromJoinTree(const IdentifierLookup & identifier_lookup,
IdentifierResolveScope & scope);
IdentifierResolveResult tryResolveIdentifierInParentScopes(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope);
IdentifierResolveResult tryResolveIdentifier(const IdentifierLookup & identifier_lookup,
IdentifierResolveScope & scope,
IdentifierResolveSettings identifier_resolve_settings = {});
QueryTreeNodePtr tryResolveIdentifierFromStorage(
const Identifier & identifier,
const QueryTreeNodePtr & table_expression_node,
const AnalysisTableExpressionData & table_expression_data,
IdentifierResolveScope & scope,
size_t identifier_column_qualifier_parts,
bool can_be_not_found = false);
/// Resolve query tree nodes functions
void qualifyColumnNodesWithProjectionNames(const QueryTreeNodes & column_nodes,
const QueryTreeNodePtr & table_expression_node,
const IdentifierResolveScope & scope);
static GetColumnsOptions buildGetColumnsOptions(QueryTreeNodePtr & matcher_node, const ContextPtr & context);
using QueryTreeNodesWithNames = std::vector<std::pair<QueryTreeNodePtr, std::string>>;
QueryTreeNodesWithNames getMatchedColumnNodesWithNames(const QueryTreeNodePtr & matcher_node,
const QueryTreeNodePtr & table_expression_node,
const NamesAndTypes & matched_columns,
const IdentifierResolveScope & scope);
void updateMatchedColumnsFromJoinUsing(QueryTreeNodesWithNames & result_matched_column_nodes_with_names, const QueryTreeNodePtr & source_table_expression, IdentifierResolveScope & scope);
QueryTreeNodesWithNames resolveQualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope);
QueryTreeNodesWithNames resolveUnqualifiedMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope);
ProjectionNames resolveMatcher(QueryTreeNodePtr & matcher_node, IdentifierResolveScope & scope);
ProjectionName resolveWindow(QueryTreeNodePtr & window_node, IdentifierResolveScope & scope);
ProjectionNames resolveLambda(const QueryTreeNodePtr & lambda_node,
const QueryTreeNodePtr & lambda_node_to_resolve,
const QueryTreeNodes & lambda_arguments,
IdentifierResolveScope & scope);
ProjectionNames resolveFunction(QueryTreeNodePtr & function_node, IdentifierResolveScope & scope);
ProjectionNames resolveExpressionNode(QueryTreeNodePtr & node, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression, bool ignore_alias = false);
ProjectionNames resolveExpressionNodeList(QueryTreeNodePtr & node_list, IdentifierResolveScope & scope, bool allow_lambda_expression, bool allow_table_expression);
ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope);
void resolveGroupByNode(QueryNode & query_node_typed, IdentifierResolveScope & scope);
void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope);
void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope);
NamesAndTypes resolveProjectionExpressionNodeList(QueryTreeNodePtr & projection_node_list, IdentifierResolveScope & scope);
void initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope);
void initializeTableExpressionData(const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope);
void resolveTableFunction(QueryTreeNodePtr & table_function_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor, bool nested_table_function);
void resolveArrayJoin(QueryTreeNodePtr & array_join_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor);
void resolveJoin(QueryTreeNodePtr & join_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor);
void resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, IdentifierResolveScope & scope, QueryExpressionsAliasVisitor & expressions_visitor);
void resolveQuery(const QueryTreeNodePtr & query_node, IdentifierResolveScope & scope);
void resolveUnion(const QueryTreeNodePtr & union_node, IdentifierResolveScope & scope);
/// Lambdas that are currently in resolve process
std::unordered_set<IQueryTreeNode *> lambdas_in_resolve_process;
/// CTEs that are currently in resolve process
std::unordered_set<std::string_view> ctes_in_resolve_process;
/// Function name to user defined lambda map
std::unordered_map<std::string, QueryTreeNodePtr> function_name_to_user_defined_lambda;
/// Array join expressions counter
size_t array_join_expressions_counter = 1;
/// Subquery counter
size_t subquery_counter = 1;
/// Global expression node to projection name map
std::unordered_map<QueryTreeNodePtr, ProjectionName> node_to_projection_name;
/// Global resolve expression node to projection names map
std::unordered_map<QueryTreeNodePtr, ProjectionNames> resolved_expressions;
/// Global resolve expression node to tree size
std::unordered_map<QueryTreeNodePtr, size_t> node_to_tree_size;
/// Global scalar subquery to scalar value map
std::unordered_map<QueryTreeNodePtrWithHash, Block> scalar_subquery_to_scalar_value_local;
std::unordered_map<QueryTreeNodePtrWithHash, Block> scalar_subquery_to_scalar_value_global;
const bool only_analyze;
};
}

View File

@ -0,0 +1,119 @@
#pragma once
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/Resolve/ScopeAliases.h>
#include <Analyzer/LambdaNode.h>
namespace DB
{
/** Visitor that extracts expression and function aliases from node and initialize scope tables with it.
* Does not go into child lambdas and queries.
*
* Important:
* Identifier nodes with aliases are added both in alias to expression and alias to function map.
*
* These is necessary because identifier with alias can give alias name to any query tree node.
*
* Example:
* WITH (x -> x + 1) AS id, id AS value SELECT value(1);
* In this example id as value is identifier node that has alias, during scope initialization we cannot derive
* that id is actually lambda or expression.
*
* There are no easy solution here, without trying to make full featured expression resolution at this stage.
* Example:
* WITH (x -> x + 1) AS id, id AS id_1, id_1 AS id_2 SELECT id_2(1);
* Example: SELECT a, b AS a, b AS c, 1 AS c;
*
* It is client responsibility after resolving identifier node with alias, make following actions:
* 1. If identifier node was resolved in function scope, remove alias from scope expression map.
* 2. If identifier node was resolved in expression scope, remove alias from scope function map.
*
* That way we separate alias map initialization and expressions resolution.
*/
class QueryExpressionsAliasVisitor : public InDepthQueryTreeVisitor<QueryExpressionsAliasVisitor>
{
public:
explicit QueryExpressionsAliasVisitor(ScopeAliases & aliases_)
: aliases(aliases_)
{}
void visitImpl(QueryTreeNodePtr & node)
{
updateAliasesIfNeeded(node, false /*is_lambda_node*/);
}
bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr & child)
{
if (auto * lambda_node = child->as<LambdaNode>())
{
updateAliasesIfNeeded(child, true /*is_lambda_node*/);
return false;
}
else if (auto * query_tree_node = child->as<QueryNode>())
{
if (query_tree_node->isCTE())
return false;
updateAliasesIfNeeded(child, false /*is_lambda_node*/);
return false;
}
else if (auto * union_node = child->as<UnionNode>())
{
if (union_node->isCTE())
return false;
updateAliasesIfNeeded(child, false /*is_lambda_node*/);
return false;
}
return true;
}
private:
void addDuplicatingAlias(const QueryTreeNodePtr & node)
{
aliases.nodes_with_duplicated_aliases.emplace(node);
auto cloned_node = node->clone();
aliases.cloned_nodes_with_duplicated_aliases.emplace_back(cloned_node);
aliases.nodes_with_duplicated_aliases.emplace(cloned_node);
}
void updateAliasesIfNeeded(const QueryTreeNodePtr & node, bool is_lambda_node)
{
if (!node->hasAlias())
return;
// We should not resolve expressions to WindowNode
if (node->getNodeType() == QueryTreeNodeType::WINDOW)
return;
const auto & alias = node->getAlias();
if (is_lambda_node)
{
if (aliases.alias_name_to_expression_node->contains(alias))
addDuplicatingAlias(node);
auto [_, inserted] = aliases.alias_name_to_lambda_node.insert(std::make_pair(alias, node));
if (!inserted)
addDuplicatingAlias(node);
return;
}
if (aliases.alias_name_to_lambda_node.contains(alias))
addDuplicatingAlias(node);
auto [_, inserted] = aliases.alias_name_to_expression_node->insert(std::make_pair(alias, node));
if (!inserted)
addDuplicatingAlias(node);
/// If node is identifier put it into transitive aliases map.
if (const auto * identifier = typeid_cast<const IdentifierNode *>(node.get()))
aliases.transitive_aliases.insert(std::make_pair(alias, identifier->getIdentifier()));
}
ScopeAliases & aliases;
};
}

View File

@ -0,0 +1,91 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Resolve/IdentifierLookup.h>
namespace DB
{
struct ScopeAliases
{
/// Alias name to query expression node
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_expression_node_before_group_by;
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_expression_node_after_group_by;
std::unordered_map<std::string, QueryTreeNodePtr> * alias_name_to_expression_node = nullptr;
/// Alias name to lambda node
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_lambda_node;
/// Alias name to table expression node
std::unordered_map<std::string, QueryTreeNodePtr> alias_name_to_table_expression_node;
/// Expressions like `x as y` where we can't say whether it's a function, expression or table.
std::unordered_map<std::string, Identifier> transitive_aliases;
/// Nodes with duplicated aliases
std::unordered_set<QueryTreeNodePtr> nodes_with_duplicated_aliases;
std::vector<QueryTreeNodePtr> cloned_nodes_with_duplicated_aliases;
/// Names which are aliases from ARRAY JOIN.
/// This is needed to properly qualify columns from matchers and avoid name collision.
std::unordered_set<std::string> array_join_aliases;
std::unordered_map<std::string, QueryTreeNodePtr> & getAliasMap(IdentifierLookupContext lookup_context)
{
switch (lookup_context)
{
case IdentifierLookupContext::EXPRESSION: return *alias_name_to_expression_node;
case IdentifierLookupContext::FUNCTION: return alias_name_to_lambda_node;
case IdentifierLookupContext::TABLE_EXPRESSION: return alias_name_to_table_expression_node;
}
}
enum class FindOption
{
FIRST_NAME,
FULL_NAME,
};
const std::string & getKey(const Identifier & identifier, FindOption find_option)
{
switch (find_option)
{
case FindOption::FIRST_NAME: return identifier.front();
case FindOption::FULL_NAME: return identifier.getFullName();
}
}
QueryTreeNodePtr * find(IdentifierLookup lookup, FindOption find_option)
{
auto & alias_map = getAliasMap(lookup.lookup_context);
const std::string * key = &getKey(lookup.identifier, find_option);
auto it = alias_map.find(*key);
if (it != alias_map.end())
return &it->second;
if (lookup.lookup_context == IdentifierLookupContext::TABLE_EXPRESSION)
return {};
while (it == alias_map.end())
{
auto jt = transitive_aliases.find(*key);
if (jt == transitive_aliases.end())
return {};
key = &(getKey(jt->second, find_option));
it = alias_map.find(*key);
}
return &it->second;
}
const QueryTreeNodePtr * find(IdentifierLookup lookup, FindOption find_option) const
{
return const_cast<ScopeAliases *>(this)->find(lookup, find_option);
}
};
}

View File

@ -0,0 +1,83 @@
#pragma once
#include <IO/Operators.h>
#include <Analyzer/ColumnNode.h>
namespace DB
{
struct StringTransparentHash
{
using is_transparent = void;
using hash = std::hash<std::string_view>;
[[maybe_unused]] size_t operator()(const char * data) const
{
return hash()(data);
}
size_t operator()(std::string_view data) const
{
return hash()(data);
}
size_t operator()(const std::string & data) const
{
return hash()(data);
}
};
using ColumnNameToColumnNodeMap = std::unordered_map<std::string, ColumnNodePtr, StringTransparentHash, std::equal_to<>>;
struct AnalysisTableExpressionData
{
std::string table_expression_name;
std::string table_expression_description;
std::string database_name;
std::string table_name;
bool should_qualify_columns = true;
NamesAndTypes column_names_and_types;
ColumnNameToColumnNodeMap column_name_to_column_node;
std::unordered_set<std::string> subcolumn_names; /// Subset columns that are subcolumns of other columns
std::unordered_set<std::string, StringTransparentHash, std::equal_to<>> column_identifier_first_parts;
bool hasFullIdentifierName(IdentifierView identifier_view) const
{
return column_name_to_column_node.contains(identifier_view.getFullName());
}
bool canBindIdentifier(IdentifierView identifier_view) const
{
return column_identifier_first_parts.contains(identifier_view.at(0));
}
[[maybe_unused]] void dump(WriteBuffer & buffer) const
{
buffer << "Table expression name " << table_expression_name;
if (!table_expression_description.empty())
buffer << " table expression description " << table_expression_description;
if (!database_name.empty())
buffer << " database name " << database_name;
if (!table_name.empty())
buffer << " table name " << table_name;
buffer << " should qualify columns " << should_qualify_columns;
buffer << " columns size " << column_name_to_column_node.size() << '\n';
for (const auto & [column_name, column_node] : column_name_to_column_node)
buffer << "Column name " << column_name << " column node " << column_node->dumpTree() << '\n';
}
[[maybe_unused]] String dump() const
{
WriteBufferFromOwnString buffer;
dump(buffer);
return buffer.str();
}
};
}

View File

@ -0,0 +1,71 @@
#pragma once
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/Resolve/IdentifierResolveScope.h>
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/JoinNode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS;
}
class TableExpressionsAliasVisitor : public InDepthQueryTreeVisitor<TableExpressionsAliasVisitor>
{
public:
explicit TableExpressionsAliasVisitor(IdentifierResolveScope & scope_)
: scope(scope_)
{}
void visitImpl(QueryTreeNodePtr & node)
{
updateAliasesIfNeeded(node);
}
static bool needChildVisit(const QueryTreeNodePtr & node, const QueryTreeNodePtr & child)
{
auto node_type = node->getNodeType();
switch (node_type)
{
case QueryTreeNodeType::ARRAY_JOIN:
{
const auto & array_join_node = node->as<const ArrayJoinNode &>();
return child.get() == array_join_node.getTableExpression().get();
}
case QueryTreeNodeType::JOIN:
{
const auto & join_node = node->as<const JoinNode &>();
return child.get() == join_node.getLeftTableExpression().get() || child.get() == join_node.getRightTableExpression().get();
}
default:
{
break;
}
}
return false;
}
private:
void updateAliasesIfNeeded(const QueryTreeNodePtr & node)
{
if (!node->hasAlias())
return;
const auto & node_alias = node->getAlias();
auto [_, inserted] = scope.aliases.alias_name_to_table_expression_node.emplace(node_alias, node);
if (!inserted)
throw Exception(ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS,
"Multiple table expressions with same alias {}. In scope {}",
node_alias,
scope.scope_node->formatASTForErrorMessage());
}
IdentifierResolveScope & scope;
};
}

View File

@ -215,6 +215,7 @@ add_object_library(clickhouse_databases_mysql Databases/MySQL)
add_object_library(clickhouse_disks Disks)
add_object_library(clickhouse_analyzer Analyzer)
add_object_library(clickhouse_analyzer_passes Analyzer/Passes)
add_object_library(clickhouse_analyzer_passes Analyzer/Resolve)
add_object_library(clickhouse_planner Planner)
add_object_library(clickhouse_interpreters Interpreters)
add_object_library(clickhouse_interpreters_cache Interpreters/Cache)

View File

@ -322,7 +322,9 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const
size_t res_offset = 0;
auto offset_it = begin();
for (size_t i = 0; i < _size; ++i, ++offset_it)
/// Replace the `++offset_it` with `offset_it.increaseCurrentRow()` and `offset_it.increaseCurrentOffset()`,
/// to remove the redundant `isDefault()` in `++` of `Interator` and reuse the following `isDefault()`.
for (size_t i = 0; i < _size; ++i, offset_it.increaseCurrentRow())
{
if (!offset_it.isDefault())
{
@ -337,6 +339,7 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const
{
values_filter.push_back(0);
}
offset_it.increaseCurrentOffset();
}
else
{

View File

@ -189,6 +189,8 @@ public:
size_t ALWAYS_INLINE getValueIndex() const { return isDefault() ? 0 : current_offset + 1; }
size_t ALWAYS_INLINE getCurrentRow() const { return current_row; }
size_t ALWAYS_INLINE getCurrentOffset() const { return current_offset; }
size_t ALWAYS_INLINE increaseCurrentRow() { return ++current_row; }
size_t ALWAYS_INLINE increaseCurrentOffset() { return ++current_offset; }
bool operator==(const Iterator & other) const
{

View File

@ -174,6 +174,11 @@
M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \
M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \
M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \
\
M(DiskPlainRewritableAzureDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \
M(DiskPlainRewritableLocalDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \
M(DiskPlainRewritableS3DirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \
\
M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \
M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \
M(MergeTreePartsLoaderThreadsScheduled, "Number of queued or active jobs in the MergeTree parts loader thread pool.") \

View File

@ -417,6 +417,13 @@ The server successfully detected this situation and will download merged part fr
M(DiskS3PutObject, "Number of DiskS3 API PutObject calls.") \
M(DiskS3GetObject, "Number of DiskS3 API GetObject calls.") \
\
M(DiskPlainRewritableAzureDirectoryCreated, "Number of directories created by the 'plain_rewritable' metadata storage for AzureObjectStorage.") \
M(DiskPlainRewritableAzureDirectoryRemoved, "Number of directories removed by the 'plain_rewritable' metadata storage for AzureObjectStorage.") \
M(DiskPlainRewritableLocalDirectoryCreated, "Number of directories created by the 'plain_rewritable' metadata storage for LocalObjectStorage.") \
M(DiskPlainRewritableLocalDirectoryRemoved, "Number of directories removed by the 'plain_rewritable' metadata storage for LocalObjectStorage.") \
M(DiskPlainRewritableS3DirectoryCreated, "Number of directories created by the 'plain_rewritable' metadata storage for S3ObjectStorage.") \
M(DiskPlainRewritableS3DirectoryRemoved, "Number of directories removed by the 'plain_rewritable' metadata storage for S3ObjectStorage.") \
\
M(S3Clients, "Number of created S3 clients.") \
M(TinyS3Clients, "Number of S3 clients copies which reuse an existing auth provider from another client.") \
\

View File

@ -924,7 +924,7 @@ class IColumn;
M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \
M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 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) \
M(Bool, allow_deprecated_functions, false, "Allow usage of deprecated functions", 0) \
M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS.

View File

@ -94,7 +94,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"},
{"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"},
}},
{"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"},
{"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"},
{"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."},
{"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"},
{"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."},

View File

@ -936,7 +936,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep
query_context->setSetting("allow_experimental_window_functions", 1);
query_context->setSetting("allow_experimental_geo_types", 1);
query_context->setSetting("allow_experimental_map_type", 1);
query_context->setSetting("allow_deprecated_functions", 1);
query_context->setSetting("allow_deprecated_error_prone_window_functions", 1);
query_context->setSetting("allow_suspicious_low_cardinality_types", 1);
query_context->setSetting("allow_suspicious_fixed_string_types", 1);

View File

@ -18,6 +18,11 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
const MetadataStorageMetrics & IObjectStorage::getMetadataStorageMetrics() const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getMetadataStorageMetrics' is not implemented");
}
bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const
{
RelativePathsWithMetadata files;

View File

@ -13,17 +13,18 @@
#include <IO/WriteSettings.h>
#include <IO/copyData.h>
#include <Disks/ObjectStorages/StoredObject.h>
#include <Disks/DiskType.h>
#include <Common/ThreadPool_fwd.h>
#include <Common/ObjectStorageKey.h>
#include <Disks/WriteMode.h>
#include <Interpreters/Context_fwd.h>
#include <Core/Types.h>
#include <Disks/DirectoryIterator.h>
#include <Common/ThreadPool.h>
#include <Common/threadPoolCallbackRunner.h>
#include <Disks/DiskType.h>
#include <Disks/ObjectStorages/MetadataStorageMetrics.h>
#include <Disks/ObjectStorages/StoredObject.h>
#include <Disks/WriteMode.h>
#include <Interpreters/Context_fwd.h>
#include <Common/Exception.h>
#include <Common/ObjectStorageKey.h>
#include <Common/ThreadPool.h>
#include <Common/ThreadPool_fwd.h>
#include <Common/threadPoolCallbackRunner.h>
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
@ -115,6 +116,8 @@ public:
virtual std::string getDescription() const = 0;
virtual const MetadataStorageMetrics & getMetadataStorageMetrics() const;
/// Object exists or not
virtual bool exists(const StoredObject & object) const = 0;

View File

@ -52,11 +52,16 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
[[maybe_unused]] auto result = path_map.emplace(path, std::move(key_prefix));
chassert(result.second);
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, 1);
writeString(path.string(), *buf);
buf->finalize();
write_finalized = true;
auto event = object_storage->getMetadataStorageMetrics().directory_created;
ProfileEvents::increment(event);
}
void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
@ -65,6 +70,9 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
if (write_finalized)
{
path_map.erase(path);
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::sub(metric, 1);
object_storage->removeObject(StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME));
}
else if (write_created)
@ -165,7 +173,15 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std:
auto object_key = ObjectStorageKey::createAsRelative(key_prefix, PREFIX_PATH_FILE_NAME);
auto object = StoredObject(object_key.serialize(), path / PREFIX_PATH_FILE_NAME);
object_storage->removeObject(object);
path_map.erase(path_it);
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::sub(metric, 1);
removed = true;
auto event = object_storage->getMetadataStorageMetrics().directory_removed;
ProfileEvents::increment(event);
}
void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
@ -185,6 +201,8 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un
buf->finalize();
path_map.emplace(path, std::move(key_prefix));
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, 1);
}
}

View File

@ -50,6 +50,8 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri
res.first->second,
remote_path.parent_path().string());
}
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, result.size());
return result;
}
@ -134,6 +136,12 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita
object_storage->setKeysGenerator(keys_gen);
}
MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage()
{
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::sub(metric, path_map->size());
}
std::vector<std::string> MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(
const std::string & storage_key, const RelativePathsWithMetadata & remote_paths, const std::string & local_path) const
{

View File

@ -14,6 +14,7 @@ private:
public:
MetadataStorageFromPlainRewritableObjectStorage(ObjectStoragePtr object_storage_, String storage_path_prefix_);
~MetadataStorageFromPlainRewritableObjectStorage() override;
MetadataStorageType getType() const override { return MetadataStorageType::PlainRewritable; }

View File

@ -0,0 +1,24 @@
#pragma once
#include <Disks/DiskType.h>
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
namespace DB
{
struct MetadataStorageMetrics
{
const ProfileEvents::Event directory_created = ProfileEvents::end();
const ProfileEvents::Event directory_removed = ProfileEvents::end();
CurrentMetrics::Metric directory_map_size = CurrentMetrics::end();
template <typename ObjectStorage, MetadataStorageType metadata_type>
static MetadataStorageMetrics create()
{
return MetadataStorageMetrics{};
}
};
}

View File

@ -23,6 +23,7 @@
#include <Disks/ObjectStorages/MetadataStorageFactory.h>
#include <Disks/ObjectStorages/PlainObjectStorage.h>
#include <Disks/ObjectStorages/PlainRewritableObjectStorage.h>
#include <Disks/ObjectStorages/createMetadataStorageMetrics.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
@ -85,7 +86,9 @@ ObjectStoragePtr createObjectStorage(
DataSourceDescription{DataSourceType::ObjectStorage, type, MetadataStorageType::PlainRewritable, /*description*/ ""}
.toString());
return std::make_shared<PlainRewritableObjectStorage<BaseObjectStorage>>(std::forward<Args>(args)...);
auto metadata_storage_metrics = DB::MetadataStorageMetrics::create<BaseObjectStorage, MetadataStorageType::PlainRewritable>();
return std::make_shared<PlainRewritableObjectStorage<BaseObjectStorage>>(
std::move(metadata_storage_metrics), std::forward<Args>(args)...);
}
else
return std::make_shared<BaseObjectStorage>(std::forward<Args>(args)...);
@ -256,8 +259,9 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
auto client = getClient(config, config_prefix, context, *settings, true);
auto key_generator = getKeyGenerator(uri, config, config_prefix);
auto metadata_storage_metrics = DB::MetadataStorageMetrics::create<S3ObjectStorage, MetadataStorageType::PlainRewritable>();
auto object_storage = std::make_shared<PlainRewritableObjectStorage<S3ObjectStorage>>(
std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
std::move(metadata_storage_metrics), std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
/// NOTE: should we still perform this check for clickhouse-disks?
if (!skip_access_check)

View File

@ -16,8 +16,9 @@ class PlainRewritableObjectStorage : public BaseObjectStorage
{
public:
template <class... Args>
explicit PlainRewritableObjectStorage(Args &&... args)
explicit PlainRewritableObjectStorage(MetadataStorageMetrics && metadata_storage_metrics_, Args &&... args)
: BaseObjectStorage(std::forward<Args>(args)...)
, metadata_storage_metrics(std::move(metadata_storage_metrics_))
/// A basic key generator is required for checking S3 capabilities,
/// it will be reset later by metadata storage.
, key_generator(createObjectStorageKeysGeneratorAsIsWithPrefix(BaseObjectStorage::getCommonKeyPrefix()))
@ -26,6 +27,8 @@ public:
std::string getName() const override { return "PlainRewritable" + BaseObjectStorage::getName(); }
const MetadataStorageMetrics & getMetadataStorageMetrics() const override { return metadata_storage_metrics; }
bool isWriteOnce() const override { return false; }
bool isPlain() const override { return true; }
@ -37,6 +40,7 @@ public:
void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; }
private:
MetadataStorageMetrics metadata_storage_metrics;
ObjectStorageKeysGeneratorPtr key_generator;
};

View File

@ -0,0 +1,67 @@
#pragma once
#if USE_AWS_S3
# include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
#endif
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
# include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
#endif
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
# include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#endif
#include <Disks/ObjectStorages/MetadataStorageMetrics.h>
namespace ProfileEvents
{
extern const Event DiskPlainRewritableAzureDirectoryCreated;
extern const Event DiskPlainRewritableAzureDirectoryRemoved;
extern const Event DiskPlainRewritableLocalDirectoryCreated;
extern const Event DiskPlainRewritableLocalDirectoryRemoved;
extern const Event DiskPlainRewritableS3DirectoryCreated;
extern const Event DiskPlainRewritableS3DirectoryRemoved;
}
namespace CurrentMetrics
{
extern const Metric DiskPlainRewritableAzureDirectoryMapSize;
extern const Metric DiskPlainRewritableLocalDirectoryMapSize;
extern const Metric DiskPlainRewritableS3DirectoryMapSize;
}
namespace DB
{
#if USE_AWS_S3
template <>
inline MetadataStorageMetrics MetadataStorageMetrics::create<S3ObjectStorage, MetadataStorageType::PlainRewritable>()
{
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableS3DirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableS3DirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableS3DirectoryMapSize};
}
#endif
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
template <>
inline MetadataStorageMetrics MetadataStorageMetrics::create<AzureObjectStorage, MetadataStorageType::PlainRewritable>()
{
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableAzureDirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableAzureDirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableAzureDirectoryMapSize};
}
#endif
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
template <>
inline MetadataStorageMetrics MetadataStorageMetrics::create<LocalObjectStorage, MetadataStorageType::PlainRewritable>()
{
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableLocalDirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableLocalDirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableLocalDirectoryMapSize};
}
#endif
}

View File

@ -123,61 +123,37 @@ SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t in
return {begin, end};
}
struct GlobalCounterPolicy
struct Data
{
static constexpr auto name = "generateSnowflakeID";
static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)";
/// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously.
struct Data
static inline std::atomic<uint64_t> lowest_available_snowflake_id = 0;
SnowflakeId reserveRange(size_t input_rows_count)
{
static inline std::atomic<uint64_t> lowest_available_snowflake_id = 0;
SnowflakeId reserveRange(size_t input_rows_count)
uint64_t available_snowflake_id = lowest_available_snowflake_id.load();
SnowflakeIdRange range;
do
{
uint64_t available_snowflake_id = lowest_available_snowflake_id.load();
SnowflakeIdRange range;
do
{
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count);
}
while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end)));
/// if CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try
/// else --> our thread reserved ID range [begin, end) and return the beginning of the range
return range.begin;
range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count);
}
};
};
while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end)));
/// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try
/// else --> our thread reserved ID range [begin, end) and return the beginning of the range
struct ThreadLocalCounterPolicy
{
static constexpr auto name = "generateSnowflakeIDThreadMonotonic";
static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)";
/// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads.
struct Data
{
static inline thread_local uint64_t lowest_available_snowflake_id = 0;
SnowflakeId reserveRange(size_t input_rows_count)
{
SnowflakeIdRange range = getRangeOfAvailableIds(toSnowflakeId(lowest_available_snowflake_id), input_rows_count);
lowest_available_snowflake_id = fromSnowflakeId(range.end);
return range.begin;
}
};
return range.begin;
}
};
}
template <typename FillPolicy>
class FunctionGenerateSnowflakeID : public IFunction, public FillPolicy
class FunctionGenerateSnowflakeID : public IFunction
{
public:
static constexpr auto name = "generateSnowflakeID";
static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared<FunctionGenerateSnowflakeID>(); }
String getName() const override { return FillPolicy::name; }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 0; }
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override { return false; }
@ -205,7 +181,7 @@ public:
{
vec_to.resize(input_rows_count);
typename FillPolicy::Data data;
Data data;
SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range
for (UInt64 & to_row : vec_to)
@ -229,27 +205,16 @@ public:
};
template<typename FillPolicy>
void registerSnowflakeIDGenerator(auto & factory)
{
static constexpr auto doc_syntax_format = "{}([expression])";
static constexpr auto example_format = "SELECT {}()";
static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)";
FunctionDocumentation::Description description = FillPolicy::description;
FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name);
FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}};
FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64";
FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}};
FunctionDocumentation::Categories categories = {"Snowflake ID"};
factory.template registerFunction<FunctionGenerateSnowflakeID<FillPolicy>>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive);
}
REGISTER_FUNCTION(GenerateSnowflakeID)
{
registerSnowflakeIDGenerator<GlobalCounterPolicy>(factory);
registerSnowflakeIDGenerator<ThreadLocalCounterPolicy>(factory);
FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)";
FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])";
FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}};
FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64";
FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"multiple", "SELECT generateSnowflakeID(1), generateSnowflakeID(2)", ""}};
FunctionDocumentation::Categories categories = {"Snowflake ID"};
factory.registerFunction<FunctionGenerateSnowflakeID>({description, syntax, arguments, returned_value, examples, categories});
}
}

View File

@ -36,11 +36,11 @@ public:
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
if (!context->getSettingsRef().allow_deprecated_error_prone_window_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
"Please use proper window function or set `allow_deprecated_error_prone_window_functions` setting to enable it",
name);
return std::make_shared<FunctionNeighbor>();

View File

@ -39,11 +39,11 @@ public:
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
if (!context->getSettingsRef().allow_deprecated_error_prone_window_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
"Please use proper window function or set `allow_deprecated_error_prone_window_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningAccumulate>();

View File

@ -139,11 +139,11 @@ public:
static FunctionPtr create(ContextPtr context)
{
if (!context->getSettingsRef().allow_deprecated_functions)
if (!context->getSettingsRef().allow_deprecated_error_prone_window_functions)
throw Exception(
ErrorCodes::DEPRECATED_FUNCTION,
"Function {} is deprecated since its usage is error-prone (see docs)."
"Please use proper window function or set `allow_deprecated_functions` setting to enable it",
"Please use proper window function or set `allow_deprecated_error_prone_window_functions` setting to enable it",
name);
return std::make_shared<FunctionRunningDifferenceImpl<is_first_line_zero>>();

View File

@ -879,7 +879,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
return true;
};
if (parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator())
/// query_plan can be empty if there is nothing to read
if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings) && query_context->canUseParallelReplicasOnInitiator())
{
// (1) find read step
QueryPlan::Node * node = query_plan.getRootNode();
@ -898,10 +899,10 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
}
else
{
if (prev_node->step)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree but it's {}", prev_node->step->getName());
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Step is expected to be ReadFromMergeTree, and wtf - last node with empty step");
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Step is expected to be ReadFromMergeTree but it's {}",
prev_node->step->getName());
}
}

View File

@ -9,7 +9,7 @@ from threading import Thread
from typing import Any, Dict, List, Optional
import requests
from lambda_shared.pr import Labels, check_pr_description
from lambda_shared.pr import Labels
from lambda_shared.token import get_cached_access_token
NEED_RERUN_OR_CANCELL_WORKFLOWS = {
@ -321,21 +321,21 @@ def main(event):
return
if action == "edited":
print("PR is edited, check if the body is correct")
error, _ = check_pr_description(
pull_request["body"], pull_request["base"]["repo"]["full_name"]
)
if error:
print(
f"The PR's body is wrong, is going to comment it. The error is: {error}"
)
post_json = {
"body": "This is an automatic comment. The PR descriptions does not "
f"match the [template]({pull_request['base']['repo']['html_url']}/"
"blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1).\n\n"
f"Please, edit it accordingly.\n\nThe error is: {error}"
}
_exec_post_with_retry(pull_request["comments_url"], token, json=post_json)
print("PR is edited - do nothing")
# error, _ = check_pr_description(
# pull_request["body"], pull_request["base"]["repo"]["full_name"]
# )
# if error:
# print(
# f"The PR's body is wrong, is going to comment it. The error is: {error}"
# )
# post_json = {
# "body": "This is an automatic comment. The PR descriptions does not "
# f"match the [template]({pull_request['base']['repo']['html_url']}/"
# "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1).\n\n"
# f"Please, edit it accordingly.\n\nThe error is: {error}"
# }
# _exec_post_with_retry(pull_request["comments_url"], token, json=post_json)
return
if action == "synchronize":

View File

@ -245,6 +245,10 @@ close it.
)
self.cherrypick_pr.add_to_labels(Labels.PR_CHERRYPICK)
self.cherrypick_pr.add_to_labels(Labels.DO_NOT_TEST)
if Labels.PR_CRITICAL_BUGFIX in [label.name for label in self.pr.labels]:
self.cherrypick_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX)
elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]:
self.cherrypick_pr.add_to_labels(Labels.PR_BUGFIX)
self._assign_new_pr(self.cherrypick_pr)
# update cherrypick PR to get the state for PR.mergable
self.cherrypick_pr.update()
@ -280,6 +284,10 @@ close it.
head=self.backport_branch,
)
self.backport_pr.add_to_labels(Labels.PR_BACKPORT)
if Labels.PR_CRITICAL_BUGFIX in [label.name for label in self.pr.labels]:
self.backport_pr.add_to_labels(Labels.PR_CRITICAL_BUGFIX)
elif Labels.PR_BUGFIX in [label.name for label in self.pr.labels]:
self.backport_pr.add_to_labels(Labels.PR_BUGFIX)
self._assign_new_pr(self.backport_pr)
def ping_cherry_pick_assignees(self, dry_run: bool) -> None:

View File

@ -50,6 +50,8 @@ TRUSTED_CONTRIBUTORS = {
class Labels:
PR_BUGFIX = "pr-bugfix"
PR_CRITICAL_BUGFIX = "pr-critical-bugfix"
CAN_BE_TESTED = "can be tested"
DO_NOT_TEST = "do not test"
MUST_BACKPORT = "pr-must-backport"
@ -68,8 +70,8 @@ class Labels:
RELEASE_LTS = "release-lts"
SUBMODULE_CHANGED = "submodule changed"
# pr-bugfix autoport can lead to issues in releases, let's do ci fixes only
AUTO_BACKPORT = {"pr-ci"}
# automatic backport for critical bug fixes
AUTO_BACKPORT = {"pr-critical-bugfix"}
# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there
@ -84,6 +86,7 @@ LABEL_CATEGORIES = {
"Bug Fix (user-visible misbehaviour in official stable or prestable release)",
"Bug Fix (user-visible misbehavior in official stable or prestable release)",
],
"pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"],
"pr-build": [
"Build/Testing/Packaging Improvement",
"Build Improvement",

View File

@ -415,12 +415,12 @@ class BuildResult:
for file in Path(REPORT_PATH).iterdir():
if f"{build_name}.json" in file.name:
any_report = file
if "_master_" in file.name:
master_report = file
elif f"_{head_ref}_" in file.name:
ref_report = file
elif pr_number and f"_{pr_number}_" in file.name:
pr_report = file
if "_master_" in file.name:
master_report = file
elif f"_{head_ref}_" in file.name:
ref_report = file
elif pr_number and f"_{pr_number}_" in file.name:
pr_report = file
if not any_report:
return None

View File

@ -9,20 +9,10 @@ cluster = ClickHouseCluster(__file__)
NUM_WORKERS = 5
nodes = []
for i in range(NUM_WORKERS):
name = "node{}".format(i + 1)
node = cluster.add_instance(
name,
main_configs=["configs/storage_conf.xml"],
env_variables={"ENDPOINT_SUBPATH": name},
with_minio=True,
stay_alive=True,
)
nodes.append(node)
MAX_ROWS = 1000
dirs_created = []
def gen_insert_values(size):
return ",".join(
@ -38,6 +28,17 @@ insert_values = ",".join(
@pytest.fixture(scope="module", autouse=True)
def start_cluster():
for i in range(NUM_WORKERS):
cluster.add_instance(
f"node{i + 1}",
main_configs=["configs/storage_conf.xml"],
with_minio=True,
env_variables={"ENDPOINT_SUBPATH": f"node{i + 1}"},
stay_alive=True,
# Override ENDPOINT_SUBPATH.
instance_env_variables=i > 0,
)
try:
cluster.start()
yield cluster
@ -64,10 +65,10 @@ def test_insert():
gen_insert_values(random.randint(1, MAX_ROWS)) for _ in range(0, NUM_WORKERS)
]
threads = []
assert len(cluster.instances) == NUM_WORKERS
for i in range(NUM_WORKERS):
t = threading.Thread(
target=create_insert, args=(nodes[i], insert_values_arr[i])
)
node = cluster.instances[f"node{i + 1}"]
t = threading.Thread(target=create_insert, args=(node, insert_values_arr[i]))
threads.append(t)
t.start()
@ -75,48 +76,61 @@ def test_insert():
t.join()
for i in range(NUM_WORKERS):
node = cluster.instances[f"node{i + 1}"]
assert (
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
node.query("SELECT * FROM test ORDER BY id FORMAT Values")
== insert_values_arr[i]
)
for i in range(NUM_WORKERS):
nodes[i].query("ALTER TABLE test MODIFY SETTING old_parts_lifetime = 59")
node = cluster.instances[f"node{i + 1}"]
node.query("ALTER TABLE test MODIFY SETTING old_parts_lifetime = 59")
assert (
nodes[i]
.query(
node.query(
"SELECT engine_full from system.tables WHERE database = currentDatabase() AND name = 'test'"
)
.find("old_parts_lifetime = 59")
).find("old_parts_lifetime = 59")
!= -1
)
nodes[i].query("ALTER TABLE test RESET SETTING old_parts_lifetime")
node.query("ALTER TABLE test RESET SETTING old_parts_lifetime")
assert (
nodes[i]
.query(
node.query(
"SELECT engine_full from system.tables WHERE database = currentDatabase() AND name = 'test'"
)
.find("old_parts_lifetime")
).find("old_parts_lifetime")
== -1
)
nodes[i].query("ALTER TABLE test MODIFY COMMENT 'new description'")
node.query("ALTER TABLE test MODIFY COMMENT 'new description'")
assert (
nodes[i]
.query(
node.query(
"SELECT comment from system.tables WHERE database = currentDatabase() AND name = 'test'"
)
.find("new description")
).find("new description")
!= -1
)
created = int(
node.query(
"SELECT value FROM system.events WHERE event = 'DiskPlainRewritableS3DirectoryCreated'"
)
)
assert created > 0
dirs_created.append(created)
assert (
int(
node.query(
"SELECT value FROM system.metrics WHERE metric = 'DiskPlainRewritableS3DirectoryMapSize'"
)
)
== created
)
@pytest.mark.order(1)
def test_restart():
insert_values_arr = []
for i in range(NUM_WORKERS):
node = cluster.instances[f"node{i + 1}"]
insert_values_arr.append(
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
node.query("SELECT * FROM test ORDER BY id FORMAT Values")
)
def restart(node):
@ -124,7 +138,7 @@ def test_restart():
threads = []
for i in range(NUM_WORKERS):
t = threading.Thread(target=restart, args=(nodes[i],))
t = threading.Thread(target=restart, args=(node,))
threads.append(t)
t.start()
@ -132,8 +146,9 @@ def test_restart():
t.join()
for i in range(NUM_WORKERS):
node = cluster.instances[f"node{i + 1}"]
assert (
nodes[i].query("SELECT * FROM test ORDER BY id FORMAT Values")
node.query("SELECT * FROM test ORDER BY id FORMAT Values")
== insert_values_arr[i]
)
@ -141,7 +156,16 @@ def test_restart():
@pytest.mark.order(2)
def test_drop():
for i in range(NUM_WORKERS):
nodes[i].query("DROP TABLE IF EXISTS test SYNC")
node = cluster.instances[f"node{i + 1}"]
node.query("DROP TABLE IF EXISTS test SYNC")
removed = int(
node.query(
"SELECT value FROM system.events WHERE event = 'DiskPlainRewritableS3DirectoryRemoved'"
)
)
assert dirs_created[i] == removed
it = cluster.minio_client.list_objects(
cluster.minio_bucket, "data/", recursive=True

View File

@ -0,0 +1,42 @@
<test>
<substitutions>
<substitution>
<name>serialization</name>
<values>
<value>sparse</value>
</values>
</substitution>
<substitution>
<name>ratio</name>
<values>
<value>10</value>
<value>100</value>
<value>1000</value>
</values>
</substitution>
</substitutions>
<create_query>
CREATE TABLE test_{serialization}_{ratio} (id UInt64, u8 UInt8, u64 UInt64, str String)
ENGINE = MergeTree ORDER BY id
SETTINGS ratio_of_defaults_for_sparse_serialization = 0.8
</create_query>
<create_query>SYSTEM STOP MERGES test_{serialization}_{ratio}</create_query>
<fill_query>
INSERT INTO test_{serialization}_{ratio} SELECT
number,
number % {ratio} = 0 ? rand(1) : 0,
number % {ratio} = 0 ? rand(2) : 0,
number % {ratio} = 0 ? randomPrintableASCII(64, 3) : ''
FROM numbers(100000000)
</fill_query>
<query>SELECT str, COUNT(DISTINCT id) as i FROM test_{serialization}_{ratio} WHERE notEmpty(str) GROUP BY str ORDER BY i DESC LIMIT 10</query>
<query>SELECT str, COUNT(DISTINCT u8) as u FROM test_{serialization}_{ratio} WHERE notEmpty(str) GROUP BY str ORDER BY u DESC LIMIT 10</query>
<query>SELECT str, COUNT(DISTINCT u64) as u FROM test_{serialization}_{ratio} WHERE notEmpty(str) GROUP BY str ORDER BY u DESC LIMIT 10</query>
<drop_query>DROP TABLE IF EXISTS test_{serialization}_{ratio}</drop_query>
</test>

View File

@ -1,5 +1,5 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT k, finalizeAggregation(sum_state), runningAccumulate(sum_state) FROM (SELECT intDiv(number, 50000) AS k, sumState(number) AS sum_state FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k);

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
DROP TABLE IF EXISTS arena;
CREATE TABLE arena (k UInt8, d String) ENGINE = Memory;
INSERT INTO arena SELECT number % 10 AS k, hex(intDiv(number, 10) % 1000) AS d FROM system.numbers LIMIT 10000000;

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
select runningDifference(x) from (select arrayJoin([0, 1, 5, 10]) as x);
select '-';
select runningDifference(x) from (select arrayJoin([2, Null, 3, Null, 10]) as x);

View File

@ -1,6 +1,6 @@
SET send_logs_level = 'fatal';
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
-- no arguments
select neighbor(); -- { serverError 42 }
-- single argument

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT number, neighbor(toString(number), 0) FROM numbers(10);
SELECT number, neighbor(toString(number), 5) FROM numbers(10);

View File

@ -1,6 +1,6 @@
-- Disable external aggregation because the state is reset for each new block of data in 'runningAccumulate' function.
SET max_bytes_before_external_group_by = 0;
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT grouping,
item,

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT runningAccumulate(string_state)
FROM (

View File

@ -1,7 +1,7 @@
SET enable_optimize_predicate_expression = 1;
SET joined_subquery_requires_alias = 0;
SET convert_query_to_cnf = 0;
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
-- https://github.com/ClickHouse/ClickHouse/issues/3885
-- https://github.com/ClickHouse/ClickHouse/issues/5485

View File

@ -1,3 +1,3 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT neighbor(toString(number), -9223372036854775808) FROM numbers(100); -- { serverError 69 }
WITH neighbor(toString(number), toInt64(rand64())) AS x SELECT * FROM system.numbers WHERE NOT ignore(x); -- { serverError 69 }

View File

@ -1,5 +1,5 @@
SET max_insert_threads = 1, max_threads = 100, min_insert_block_size_rows = 1048576, max_block_size = 65536;
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
DROP TABLE IF EXISTS t;
CREATE TABLE t (x UInt64) ENGINE = StripeLog;
-- For trivial INSERT SELECT, max_threads is lowered to max_insert_threads and max_block_size is changed to min_insert_block_size_rows.

View File

@ -1,2 +1,2 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT k, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM (SELECT arrayJoin([(NULL, 65535), ('a', 7), ('a', 3), ('b', 11), ('b', 2), ('', -9223372036854775808)]) AS t)) WHERE i = 9223372036854775807;

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SET output_format_pretty_row_numbers = 0;
SELECT

View File

@ -478,7 +478,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
SETTINGS allow_deprecated_error_prone_window_functions = 1
-- explain
Expression (Projection)
Sorting (Sorting for ORDER BY)

View File

@ -315,7 +315,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1"
SETTINGS allow_deprecated_error_prone_window_functions = 1"
run_query "$query"
echo "-- non-stateful function does _not_ prevent removing inner ORDER BY"

View File

@ -477,7 +477,7 @@ FROM
ORDER BY number DESC
)
ORDER BY number ASC
SETTINGS allow_deprecated_functions = 1
SETTINGS allow_deprecated_error_prone_window_functions = 1
-- explain
Expression (Project names)
Sorting (Sorting for ORDER BY)

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
DROP TABLE IF EXISTS session_events;
DROP TABLE IF EXISTS event_types;

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
drop table if exists largestTriangleThreeBucketsTestFloat64Float64;
CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64
@ -55,10 +55,10 @@ CREATE TABLE largestTriangleTreeBucketsBucketSizeTest
INSERT INTO largestTriangleTreeBucketsBucketSizeTest (x, y) SELECT (number + 1) AS x, (x % 1000) AS y FROM numbers(9999);
SELECT
arrayJoin(lttb(1000)(x, y)) AS point,
tupleElement(point, 1) AS point_x,
point_x - neighbor(point_x, -1) AS point_x_diff_with_previous_row
SELECT
arrayJoin(lttb(1000)(x, y)) AS point,
tupleElement(point, 1) AS point_x,
point_x - neighbor(point_x, -1) AS point_x_diff_with_previous_row
FROM largestTriangleTreeBucketsBucketSizeTest LIMIT 990, 10;
DROP TABLE largestTriangleTreeBucketsBucketSizeTest;

View File

@ -1,4 +1,4 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
CREATE TABLE t
(

View File

@ -6,13 +6,13 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt sync"
${CLICKHOUSE_CLIENT} --query "drop table if exists 03008_test_local_mt sync"
${CLICKHOUSE_CLIENT} -nm --query "
create table test_mt (a Int32, b Int64, c Int64)
create table 03008_test_local_mt (a Int32, b Int64, c Int64)
engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = disk(
name = disk_s3_plain,
name = 03008_local_plain_rewritable,
type = object_storage,
object_storage_type = local,
metadata_type = plain_rewritable,
@ -20,34 +20,36 @@ settings disk = disk(
"
${CLICKHOUSE_CLIENT} -nm --query "
insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into test_mt (*) select number, number, number from numbers_mt(10000);
insert into 03008_test_local_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into 03008_test_local_mt (*) select number, number, number from numbers_mt(10000);
"
${CLICKHOUSE_CLIENT} -nm --query "
select count(*) from test_mt;
select (*) from test_mt order by tuple(a, b) limit 10;
select count(*) from 03008_test_local_mt;
select (*) from 03008_test_local_mt order by tuple(a, b) limit 10;
"
${CLICKHOUSE_CLIENT} --query "optimize table test_mt final"
${CLICKHOUSE_CLIENT} --query "optimize table 03008_test_local_mt final;"
${CLICKHOUSE_CLIENT} -nm --query "
alter table test_mt modify setting disk = 'disk_s3_plain', old_parts_lifetime = 3600;
select engine_full from system.tables WHERE database = currentDatabase() AND name = 'test_mt';
alter table 03008_test_local_mt modify setting disk = '03008_local_plain_rewritable', old_parts_lifetime = 3600;
select engine_full from system.tables WHERE database = currentDatabase() AND name = '03008_test_local_mt';
" | grep -c "old_parts_lifetime = 3600"
${CLICKHOUSE_CLIENT} -nm --query "
select count(*) from test_mt;
select (*) from test_mt order by tuple(a, b) limit 10;
select count(*) from 03008_test_local_mt;
select (*) from 03008_test_local_mt order by tuple(a, b) limit 10;
"
${CLICKHOUSE_CLIENT} -nm --query "
alter table test_mt update c = 0 where a % 2 = 1;
alter table test_mt add column d Int64 after c;
alter table test_mt drop column c;
alter table 03008_test_local_mt update c = 0 where a % 2 = 1;
alter table 03008_test_local_mt add column d Int64 after c;
alter table 03008_test_local_mt drop column c;
" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} -nm --query "
truncate table test_mt;
select count(*) from test_mt;
truncate table 03008_test_local_mt;
select count(*) from 03008_test_local_mt;
"
${CLICKHOUSE_CLIENT} --query "drop table 03008_test_local_mt sync"

View File

@ -7,47 +7,49 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt"
${CLICKHOUSE_CLIENT} --query "drop table if exists test_s3_mt"
${CLICKHOUSE_CLIENT} -nm --query "
create table test_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
create table test_s3_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = disk(
name = s3_plain_rewritable,
name = 03008_s3_plain_rewritable,
type = s3_plain_rewritable,
endpoint = 'http://localhost:11111/test/test_mt/',
endpoint = 'http://localhost:11111/test/03008_test_s3_mt/',
access_key_id = clickhouse,
secret_access_key = clickhouse);
"
${CLICKHOUSE_CLIENT} -nm --query "
insert into test_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into test_mt (*) select number, number, number from numbers_mt(10000);
select count(*) from test_mt;
select (*) from test_mt order by tuple(a, b) limit 10;
insert into test_s3_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5);
insert into test_s3_mt (*) select number, number, number from numbers_mt(10000);
select count(*) from test_s3_mt;
select (*) from test_s3_mt order by tuple(a, b) limit 10;
"
${CLICKHOUSE_CLIENT} --query "optimize table test_mt final"
${CLICKHOUSE_CLIENT} --query "optimize table test_s3_mt final"
${CLICKHOUSE_CLIENT} -m --query "
alter table test_mt add projection test_mt_projection (select * order by b)" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
alter table test_s3_mt add projection test_s3_mt_projection (select * order by b)" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} -nm --query "
alter table test_mt update c = 0 where a % 2 = 1;
alter table test_mt add column d Int64 after c;
alter table test_mt drop column c;
alter table test_s3_mt update c = 0 where a % 2 = 1;
alter table test_s3_mt add column d Int64 after c;
alter table test_s3_mt drop column c;
" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} -nm --query "
detach table test_mt;
attach table test_mt;
detach table test_s3_mt;
attach table test_s3_mt;
"
${CLICKHOUSE_CLIENT} --query "drop table if exists test_mt_dst"
${CLICKHOUSE_CLIENT} --query "drop table if exists test_s3_mt_dst"
${CLICKHOUSE_CLIENT} -m --query "
create table test_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = 's3_plain_rewritable'
create table test_s3_mt_dst (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b)
settings disk = '03008_s3_plain_rewritable'
"
${CLICKHOUSE_CLIENT} -m --query "
alter table test_mt move partition 0 to table test_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
alter table test_s3_mt move partition 0 to table test_s3_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED"
${CLICKHOUSE_CLIENT} --query "drop table test_s3_mt sync"

View File

@ -1,9 +1,5 @@
-- generateSnowflakeID
1
0
0
1
100
-- generateSnowflakeIDThreadMonotonic
1
100

View File

@ -1,4 +1,4 @@
SELECT '-- generateSnowflakeID';
-- Test SQL function 'generateSnowflakeID'
SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero
@ -14,16 +14,3 @@ FROM
SELECT DISTINCT generateSnowflakeID()
FROM numbers(100)
);
SELECT '-- generateSnowflakeIDThreadMonotonic';
SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero
SELECT generateSnowflakeIDThreadMonotonic(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT count(*)
FROM
(
SELECT DISTINCT generateSnowflakeIDThreadMonotonic()
FROM numbers(100)
);

View File

@ -4,7 +4,7 @@ SELECT runningDifference(number) FROM system.numbers LIMIT 10; -- { serverError
SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); -- { serverError 721 }
SET allow_deprecated_functions=1;
SET allow_deprecated_error_prone_window_functions=1;
SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10 FORMAT Null;

View File

@ -1,3 +1,3 @@
SET allow_deprecated_functions = 1;
SET allow_deprecated_error_prone_window_functions = 1;
SELECT EventDate, finalizeAggregation(state), runningAccumulate(state) FROM (SELECT EventDate, uniqState(UserID) AS state FROM test.hits GROUP BY EventDate ORDER BY EventDate);

View File

@ -25,6 +25,7 @@ categories_preferred_order = (
"New Feature",
"Performance Improvement",
"Improvement",
"Critical Bug Fix",
"Bug Fix",
"Build/Testing/Packaging Improvement",
"Other",
@ -112,7 +113,7 @@ def get_descriptions(prs: PullRequests) -> Dict[str, List[Description]]:
in_changelog = merge_commit in SHA_IN_CHANGELOG
if in_changelog:
desc = generate_description(pr, repos[repo_name])
if desc is not None:
if desc:
if desc.category not in descriptions:
descriptions[desc.category] = []
descriptions[desc.category].append(desc)
@ -187,7 +188,7 @@ def parse_args() -> argparse.Namespace:
# This function mirrors the PR description checks in ClickhousePullRequestTrigger.
# Returns False if the PR should not be mentioned changelog.
# Returns None if the PR should not be mentioned in changelog.
def generate_description(item: PullRequest, repo: Repository) -> Optional[Description]:
backport_number = item.number
if item.head.ref.startswith("backport/"):

View File

@ -1619,7 +1619,6 @@ generateRandom
generateRandomStructure
generateSeries
generateSnowflakeID
generateSnowflakeIDThreadMonotonic
generateULID
generateUUIDv
geoDistance
@ -2661,6 +2660,9 @@ toStartOfSecond
toStartOfTenMinutes
toStartOfWeek
toStartOfYear
toStartOfMicrosecond
toStartOfMillisecond
toStartOfNanosecond
toString
toStringCutToZero
toTime
@ -2789,6 +2791,7 @@ urls
usearch
userspace
userver
UTCTimestamp
utils
uuid
uuidv