Merge branch 'ClickHouse:master' into fix_pipeline_stuck_plansquashing

This commit is contained in:
Yarik Briukhovetskyi 2024-06-23 14:48:44 +02:00 committed by GitHub
commit a0c1f0a88d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
131 changed files with 1656 additions and 907 deletions

View File

@ -34,20 +34,18 @@ curl https://clickhouse.com/ | sh
Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know.
* [v24.5 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - May 30
* [v24.6 Community Call](https://clickhouse.com/company/events/v24-6-community-release-call) - Jul 2
## Upcoming Events
Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `<at>` clickhouse `<dot>` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc.
* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angeles](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22
* [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28
* [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3
* [ClickHouse Meetup @ Cloudflare - San Francisco](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/300523061/) - Jun 4
* [ClickHouse (クリックハウス) Meetup Tokyo](https://www.meetup.com/clickhouse-tokyo-user-group/events/300798053/) - Jun 5
* [AWS Summit in DC](https://clickhouse.com/company/events/2024-06-aws-summit-dc) - Jun 26
* [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27
* [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9
* [ClickHouse Cloud - Live Update Call](https://clickhouse.com/company/events/202407-cloud-update-live) - Jul 9
* [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9
* [AWS Summit in New York](https://clickhouse.com/company/events/2024-07-awssummit-nyc) - Jul 10
* [ClickHouse Meetup @ Klaviyo - Boston](https://www.meetup.com/clickhouse-boston-user-group/events/300907870) - Jul 11
## Recent Recordings

View File

@ -26,7 +26,7 @@ namespace Poco
{
namespace Net
{
constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 1024 * 1024;
constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024;
typedef Poco::BasicBufferedStreamBuf<char, std::char_traits<char>> HTTPBasicStreamBuf;

2
contrib/re2 vendored

@ -1 +1 @@
Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c
Subproject commit 85dd7ad833a73095ecf3e3baea608ba051bbe2c7

View File

@ -28,16 +28,20 @@ set(RE2_SOURCES
add_library(_re2 ${RE2_SOURCES})
target_include_directories(_re2 PUBLIC "${SRC_DIR}")
target_link_libraries(_re2 PRIVATE
absl::absl_check
absl::absl_log
absl::base
absl::core_headers
absl::fixed_array
absl::flags
absl::flat_hash_map
absl::flat_hash_set
absl::hash
absl::inlined_vector
absl::strings
absl::str_format
absl::synchronization
absl::optional
absl::span)
absl::span
absl::str_format
absl::strings
absl::synchronization)
add_library(ch_contrib::re2 ALIAS _re2)

View File

@ -285,7 +285,7 @@ stop_logs_replication
# Try to get logs while server is running
failed_to_save_logs=0
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log
do
err=$(clickhouse-client -q "select * from system.$table into outfile '/test_output/$table.tsv.gz' format TSVWithNamesAndTypes")
echo "$err"
@ -339,7 +339,7 @@ if [ $failed_to_save_logs -ne 0 ]; then
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log blob_storage_log error_log
do
clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then

View File

@ -954,6 +954,38 @@ Or it can be set in hex:
Everything mentioned above can be applied for `aes_256_gcm_siv` (but the key must be 32 bytes long).
## error_log {#error_log}
It is disabled by default.
**Enabling**
To manually turn on error history collection [`system.error_log`](../../operations/system-tables/error_log.md), create `/etc/clickhouse-server/config.d/error_log.xml` with the following content:
``` xml
<clickhouse>
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<flush_on_crash>false</flush_on_crash>
</error_log>
</clickhouse>
```
**Disabling**
To disable `error_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_error_log.xml` with the following content:
``` xml
<clickhouse>
<error_log remove="1" />
</clickhouse>
```
## custom_settings_prefixes {#custom_settings_prefixes}
@ -1917,7 +1949,7 @@ For more information, see the MergeTreeSettings.h header file.
## metric_log {#metric_log}
It is enabled by default. If it`s not, you can do this manually.
It is disabled by default.
**Enabling**

View File

@ -0,0 +1,39 @@
---
slug: /en/operations/system-tables/error_log
---
# error_log
Contains history of error values from table `system.errors`, periodically flushed to disk.
Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time.
- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — Code number of the error.
- `error` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) - Name of the error.
- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of times this error happened.
- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Remote exception (i.e. received during one of the distributed queries).
**Example**
``` sql
SELECT * FROM system.error_log LIMIT 1 FORMAT Vertical;
```
``` text
Row 1:
──────
hostname: clickhouse.eu-central1.internal
event_date: 2024-06-18
event_time: 2024-06-18 07:32:39
code: 999
error: KEEPER_EXCEPTION
value: 2
remote: 0
```
**See also**
- [error_log setting](../../operations/server-configuration-parameters/settings.md#error_log) — Enabling and disabling the setting.
- [system.errors](../../operations/system-tables/errors.md) — Contains error codes with the number of times they have been triggered.
- [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring.

View File

@ -25,7 +25,7 @@ stddevPop(x)
**Returned value**
Square root of standard deviation of `x`. [Float64](../../data-types/float.md).
- Square root of standard deviation of `x`. [Float64](../../data-types/float.md).
**Example**

View File

@ -4,30 +4,25 @@ slug: "/en/sql-reference/aggregate-functions/reference/varpop"
sidebar_position: 32
---
This page covers the `varPop` and `varPopStable` functions available in ClickHouse.
## varPop
Calculates the population covariance between two data columns. The population covariance measures the degree to which two variables vary together. Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`.
Calculates the population variance.
**Syntax**
```sql
covarPop(x, y)
varPop(x)
```
Alias: `VAR_POP`.
**Parameters**
- `x`: The first data column. [Numeric](../../../native-protocol/columns.md)
- `y`: The second data column. [Numeric](../../../native-protocol/columns.md)
- `x`: Population of values to find the population variance of. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md).
**Returned value**
Returns an integer of type `Float64`.
**Implementation details**
This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable`](#varpopstable) function.
- Returns the population variance of `x`. [`Float64`](../../data-types/float.md).
**Example**
@ -37,69 +32,21 @@ Query:
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data
(
x Int32,
y Int32
x UInt8,
)
ENGINE = Memory;
INSERT INTO test_data VALUES (1, 2), (2, 3), (3, 5), (4, 6), (5, 8);
INSERT INTO test_data VALUES (3), (3), (3), (4), (4), (5), (5), (7), (11), (15);
SELECT
covarPop(x, y) AS covar_pop
varPop(x) AS var_pop
FROM test_data;
```
Result:
```response
3
```
## varPopStable
Calculates population covariance between two data columns using a stable, numerically accurate method to calculate the variance. This function is designed to provide reliable results even with large datasets or values that might cause numerical instability in other implementations.
**Syntax**
```sql
covarPopStable(x, y)
```
**Parameters**
- `x`: The first data column. [String literal](../../syntax#syntax-string-literal)
- `y`: The second data column. [Expression](../../syntax#syntax-expressions)
**Returned value**
Returns an integer of type `Float64`.
**Implementation details**
Unlike [`varPop`](#varpop), this function uses a stable, numerically accurate algorithm to calculate the population variance to avoid issues like catastrophic cancellation or loss of precision. This function also handles `NaN` and `Inf` values correctly, excluding them from calculations.
**Example**
Query:
```sql
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data
(
x Int32,
y Int32
)
ENGINE = Memory;
INSERT INTO test_data VALUES (1, 2), (2, 9), (9, 5), (4, 6), (5, 8);
SELECT
covarPopStable(x, y) AS covar_pop_stable
FROM test_data;
```
Result:
```response
0.5999999999999999
┌─var_pop─┐
│ 14.4 │
└─────────┘
```

View File

@ -0,0 +1,52 @@
---
title: "varPopStable"
slug: "/en/sql-reference/aggregate-functions/reference/varpopstable"
sidebar_position: 32
---
## varPopStable
Returns the population variance. Unlike [`varPop`](../reference/varpop.md), this function uses a [numerically stable](https://en.wikipedia.org/wiki/Numerical_stability) algorithm. It works slower but provides a lower computational error.
**Syntax**
```sql
varPopStable(x)
```
Alias: `VAR_POP_STABLE`.
**Parameters**
- `x`: Population of values to find the population variance of. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md).
**Returned value**
- Returns the population variance of `x`. [Float64](../../data-types/float.md).
**Example**
Query:
```sql
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data
(
x UInt8,
)
ENGINE = Memory;
INSERT INTO test_data VALUES (3),(3),(3),(4),(4),(5),(5),(7),(11),(15);
SELECT
varPopStable(x) AS var_pop_stable
FROM test_data;
```
Result:
```response
┌─var_pop_stable─┐
│ 14.4 │
└────────────────┘
```

View File

@ -4,8 +4,6 @@ slug: /en/sql-reference/aggregate-functions/reference/varsamp
sidebar_position: 33
---
This page contains information on the `varSamp` and `varSampStable` ClickHouse functions.
## varSamp
Calculate the sample variance of a data set.
@ -13,24 +11,27 @@ Calculate the sample variance of a data set.
**Syntax**
```sql
varSamp(expr)
varSamp(x)
```
Alias: `VAR_SAMP`.
**Parameters**
- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../../syntax#syntax-expressions)
- `x`: The population for which you want to calculate the sample variance. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md).
**Returned value**
Returns a Float64 value representing the sample variance of the input data set.
- Returns the sample variance of the input data set `x`. [Float64](../../data-types/float.md).
**Implementation details**
The `varSamp()` function calculates the sample variance using the following formula:
The `varSamp` function calculates the sample variance using the following formula:
```plaintext
∑(x - mean(x))^2 / (n - 1)
```
$$
\sum\frac{(x - \text{mean}(x))^2}{(n - 1)}
$$
Where:
@ -38,91 +39,29 @@ Where:
- `mean(x)` is the arithmetic mean of the data set.
- `n` is the number of data points in the data set.
The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPop()` function](./varpop#varpop) instead.
This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable`](#varsampstable) function.
The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use [`varPop`](../reference/varpop.md) instead.
**Example**
Query:
```sql
CREATE TABLE example_table
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data
(
id UInt64,
value Float64
x Float64
)
ENGINE = MergeTree
ORDER BY id;
ENGINE = Memory;
INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7);
INSERT INTO test_data VALUES (10.5), (12.3), (9.8), (11.2), (10.7);
SELECT varSamp(value) FROM example_table;
SELECT round(varSamp(x),3) AS var_samp FROM test_data;
```
Response:
```response
0.8650000000000091
┌─var_samp─┐
│ 0.865 │
└──────────┘
```
## varSampStable
Calculate the sample variance of a data set using a numerically stable algorithm.
**Syntax**
```sql
varSampStable(expr)
```
**Parameters**
- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../../syntax#syntax-expressions)
**Returned value**
The `varSampStable` function returns a Float64 value representing the sample variance of the input data set.
**Implementation details**
The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](#varsamp) function:
```plaintext
∑(x - mean(x))^2 / (n - 1)
```
Where:
- `x` is each individual data point in the data set.
- `mean(x)` is the arithmetic mean of the data set.
- `n` is the number of data points in the data set.
The difference between `varSampStable` and `varSamp` is that `varSampStable` is designed to provide a more deterministic and stable result when dealing with floating-point arithmetic. It uses an algorithm that minimizes the accumulation of rounding errors, which can be particularly important when dealing with large data sets or data with a wide range of values.
Like `varSamp`, the `varSampStable` function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPopStable`](./varpop#varpopstable) function instead.
**Example**
Query:
```sql
CREATE TABLE example_table
(
id UInt64,
value Float64
)
ENGINE = MergeTree
ORDER BY id;
INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7);
SELECT varSampStable(value) FROM example_table;
```
Response:
```response
0.865
```
This query calculates the sample variance of the `value` column in the `example_table` using the `varSampStable()` function. The result shows that the sample variance of the values `[10.5, 12.3, 9.8, 11.2, 10.7]` is approximately 0.865, which may differ slightly from the result of `varSamp` due to the more precise handling of floating-point arithmetic.

View File

@ -0,0 +1,63 @@
---
title: "varSampStable"
slug: /en/sql-reference/aggregate-functions/reference/varsampstable
sidebar_position: 33
---
## varSampStable
Calculate the sample variance of a data set. Unlike [`varSamp`](../reference/varsamp.md), this function uses a numerically stable algorithm. It works slower but provides a lower computational error.
**Syntax**
```sql
varSampStable(x)
```
Alias: `VAR_SAMP_STABLE`
**Parameters**
- `x`: The population for which you want to calculate the sample variance. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md).
**Returned value**
- Returns the sample variance of the input data set. [Float64](../../data-types/float.md).
**Implementation details**
The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](../reference/varsamp.md):
$$
\sum\frac{(x - \text{mean}(x))^2}{(n - 1)}
$$
Where:
- `x` is each individual data point in the data set.
- `mean(x)` is the arithmetic mean of the data set.
- `n` is the number of data points in the data set.
**Example**
Query:
```sql
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data
(
x Float64
)
ENGINE = Memory;
INSERT INTO test_data VALUES (10.5), (12.3), (9.8), (11.2), (10.7);
SELECT round(varSampStable(x),3) AS var_samp_stable FROM test_data;
```
Response:
```response
┌─var_samp_stable─┐
│ 0.865 │
└─────────────────┘
```

View File

@ -732,11 +732,8 @@ void LocalServer::processConfig()
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
}
server_display_name = config().getString("display_name", getFQDNOrHostName());
prompt_by_server_display_name = config().getRawString("prompt_by_server_display_name.default", "{display_name} :) ");
std::map<String, String> prompt_substitutions{{"display_name", server_display_name}};
for (const auto & [key, value] : prompt_substitutions)
boost::replace_all(prompt_by_server_display_name, "{" + key + "}", value);
server_display_name = config().getString("display_name", "");
prompt_by_server_display_name = config().getRawString("prompt_by_server_display_name.default", ":) ");
global_context->setQueryKindInitial();
global_context->setQueryKind(query_kind);

View File

@ -1155,6 +1155,18 @@
<flush_on_crash>false</flush_on_crash>
</metric_log>
<!-- Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval. -->
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<max_size_rows>1048576</max_size_rows>
<reserved_size_rows>8192</reserved_size_rows>
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
<flush_on_crash>false</flush_on_crash>
</error_log>
<!--
Asynchronous metric log contains values of metrics from
system.asynchronous_metrics.

View File

@ -726,6 +726,13 @@ metric_log:
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000
# Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval.
error_log:
database: system
table: error_log
flush_interval_milliseconds: 7500
collect_interval_milliseconds: 1000
# Asynchronous metric log contains values of metrics from
# system.asynchronous_metrics.
asynchronous_metric_log:

View File

@ -261,7 +261,24 @@ AccessControl::AccessControl()
}
AccessControl::~AccessControl() = default;
AccessControl::~AccessControl()
{
try
{
AccessControl::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void AccessControl::shutdown()
{
MultipleAccessStorage::shutdown();
removeAllStorages();
}
void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,

View File

@ -53,6 +53,9 @@ public:
AccessControl();
~AccessControl() override;
/// Shutdown the access control and stops all background activity.
void shutdown() override;
/// Initializes access storage (user directories).
void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_);

View File

@ -194,11 +194,9 @@ DiskAccessStorage::DiskAccessStorage(const String & storage_name_, const String
DiskAccessStorage::~DiskAccessStorage()
{
stopListsWritingThread();
try
{
writeLists();
DiskAccessStorage::shutdown();
}
catch (...)
{
@ -207,6 +205,17 @@ DiskAccessStorage::~DiskAccessStorage()
}
void DiskAccessStorage::shutdown()
{
stopListsWritingThread();
{
std::lock_guard lock{mutex};
writeLists();
}
}
String DiskAccessStorage::getStorageParamsJSON() const
{
std::lock_guard lock{mutex};

View File

@ -18,6 +18,8 @@ public:
DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_);
~DiskAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
String getStorageParamsJSON() const override;

View File

@ -44,6 +44,11 @@ public:
explicit IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {}
virtual ~IAccessStorage() = default;
/// If the AccessStorage has to do some complicated work when destroying - do it in advance.
/// For example, if the AccessStorage contains any threads for background work - ask them to complete and wait for completion.
/// By default, does nothing.
virtual void shutdown() {}
/// Returns the name of this storage.
const String & getStorageName() const { return storage_name; }
virtual const char * getStorageType() const = 0;

View File

@ -34,11 +34,23 @@ MultipleAccessStorage::MultipleAccessStorage(const String & storage_name_)
MultipleAccessStorage::~MultipleAccessStorage()
{
/// It's better to remove the storages in the reverse order because they could depend on each other somehow.
try
{
MultipleAccessStorage::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void MultipleAccessStorage::shutdown()
{
/// It's better to shutdown the storages in the reverse order because they could depend on each other somehow.
const auto storages = getStoragesPtr();
for (const auto & storage : *storages | boost::adaptors::reversed)
{
removeStorage(storage);
storage->shutdown();
}
}
@ -72,6 +84,16 @@ void MultipleAccessStorage::removeStorage(const StoragePtr & storage_to_remove)
ids_cache.clear();
}
void MultipleAccessStorage::removeAllStorages()
{
/// It's better to remove the storages in the reverse order because they could depend on each other somehow.
const auto storages = getStoragesPtr();
for (const auto & storage : *storages | boost::adaptors::reversed)
{
removeStorage(storage);
}
}
std::vector<StoragePtr> MultipleAccessStorage::getStorages()
{
return *getStoragesPtr();

View File

@ -21,6 +21,8 @@ public:
explicit MultipleAccessStorage(const String & storage_name_ = STORAGE_TYPE);
~MultipleAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
bool isReadOnly() const override;
bool isReadOnly(const UUID & id) const override;
@ -32,6 +34,7 @@ public:
void setStorages(const std::vector<StoragePtr> & storages);
void addStorage(const StoragePtr & new_storage);
void removeStorage(const StoragePtr & storage_to_remove);
void removeAllStorages();
std::vector<StoragePtr> getStorages();
std::vector<ConstStoragePtr> getStorages() const;
std::shared_ptr<const std::vector<StoragePtr>> getStoragesPtr();

View File

@ -66,6 +66,18 @@ ReplicatedAccessStorage::ReplicatedAccessStorage(
}
ReplicatedAccessStorage::~ReplicatedAccessStorage()
{
try
{
ReplicatedAccessStorage::shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void ReplicatedAccessStorage::shutdown()
{
stopWatchingThread();
}

View File

@ -23,6 +23,8 @@ public:
ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper, AccessChangesNotifier & changes_notifier_, bool allow_backup);
~ReplicatedAccessStorage() override;
void shutdown() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
void startPeriodicReloading() override { startWatchingThread(); }

View File

@ -24,8 +24,6 @@
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/DOM/DOMParser.h>
#include <ranges>
namespace ProfileEvents
{
@ -93,6 +91,7 @@ BackupImpl::BackupImpl(
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupReader> reader_,
const ContextPtr & context_,
bool is_internal_backup_,
bool use_same_s3_credentials_for_base_backup_)
: backup_info(backup_info_)
, backup_name_for_logging(backup_info.toStringForLogging())
@ -101,7 +100,7 @@ BackupImpl::BackupImpl(
, open_mode(OpenMode::READ)
, reader(std::move(reader_))
, context(context_)
, is_internal_backup(false)
, is_internal_backup(is_internal_backup_)
, version(INITIAL_BACKUP_VERSION)
, base_backup_info(base_backup_info_)
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
@ -256,6 +255,7 @@ std::shared_ptr<const IBackup> BackupImpl::getBaseBackupUnlocked() const
params.backup_info = *base_backup_info;
params.open_mode = OpenMode::READ;
params.context = context;
params.is_internal_backup = is_internal_backup;
/// use_same_s3_credentials_for_base_backup should be inherited for base backups
params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup;

View File

@ -40,6 +40,7 @@ public:
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupReader> reader_,
const ContextPtr & context_,
bool is_internal_backup_,
bool use_same_s3_credentials_for_base_backup_);
BackupImpl(

View File

@ -153,6 +153,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory)
params.base_backup_info,
reader,
params.context,
params.is_internal_backup,
/* use_same_s3_credentials_for_base_backup*/ false);
}
else

View File

@ -119,6 +119,7 @@ void registerBackupEngineS3(BackupFactory & factory)
params.base_backup_info,
reader,
params.context,
params.is_internal_backup,
params.use_same_s3_credentials_for_base_backup);
}
else

View File

@ -177,6 +177,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory)
params.base_backup_info,
reader,
params.context,
params.is_internal_backup,
params.use_same_s3_credentials_for_base_backup);
}
else

View File

@ -1,6 +1,7 @@
#include <Common/CurrentMetrics.h>
// clang-format off
/// Available metrics. Add something here as you wish.
/// If the metric is generic (i.e. not server specific)
/// it should be also added to src/Coordination/KeeperConstant.cpp

View File

@ -202,7 +202,10 @@ uint64_t readU64(std::string_view & sp)
{
SAFE_CHECK(sp.size() >= N, "underflow");
uint64_t x = 0;
memcpy(&x, sp.data(), N);
if constexpr (std::endian::native == std::endian::little)
memcpy(&x, sp.data(), N);
else
memcpy(reinterpret_cast<char*>(&x) + sizeof(uint64_t) - N, sp.data(), N);
sp.remove_prefix(N);
return x;
}

View File

@ -1,8 +1,6 @@
#pragma once
#include <cstddef>
#include <cstdint>
#include <utility>
#include <mutex>
#include <string_view>
#include <vector>
@ -35,7 +33,7 @@ namespace ErrorCodes
struct Error
{
/// Number of times Exception with this ErrorCode had been throw.
/// Number of times Exception with this ErrorCode has been thrown.
Value count = 0;
/// Time of the last error.
UInt64 error_time_ms = 0;

View File

@ -3,6 +3,7 @@
#include <Common/TraceSender.h>
// clang-format off
/// Available events. Add something here as you wish.
/// If the event is generic (i.e. not server specific)
/// it should be also added to src/Coordination/KeeperConstant.cpp
@ -14,6 +15,7 @@
M(QueriesWithSubqueries, "Count queries with all subqueries") \
M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \
M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \
M(SelectQueriesWithPrimaryKeyUsage, "Count SELECT queries which use the primary key to evaluate the WHERE condition") \
M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \
M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \
M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \

View File

@ -1,5 +1,6 @@
#include <Interpreters/AsynchronousMetricLog.h>
#include <Interpreters/CrashLog.h>
#include <Interpreters/ErrorLog.h>
#include <Interpreters/MetricLog.h>
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Interpreters/PartLog.h>

View File

@ -1,9 +1,7 @@
#pragma once
#include <atomic>
#include <condition_variable>
#include <memory>
#include <thread>
#include <vector>
#include <base/types.h>
@ -32,7 +30,8 @@
M(FilesystemReadPrefetchesLogElement) \
M(AsynchronousInsertLogElement) \
M(BackupLogElement) \
M(BlobStorageLogElement)
M(BlobStorageLogElement) \
M(ErrorLogElement)
namespace Poco
{

View File

@ -14,6 +14,7 @@ class AbstractConfiguration;
namespace DB
{
// clang-format off
#define SERVER_SETTINGS(M, ALIAS) \
M(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \
M(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \

View File

@ -31,6 +31,7 @@ class IColumn;
* for tracking settings changes in different versions and for special `compatibility` setting to work correctly.
*/
// clang-format off
#define COMMON_SETTINGS(M, ALIAS) \
M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\
M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \

View File

@ -75,6 +75,7 @@ namespace SettingsChangesHistory
using SettingsChanges = std::vector<SettingChange>;
}
// clang-format off
/// History of settings changes that controls some backward incompatible changes
/// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done
/// in this version. This history contains both changes to existing settings and newly added settings.

View File

@ -786,9 +786,6 @@ Block ActionsDAG::updateHeader(const Block & header) const
for (auto & col : result_columns)
res.insert(std::move(col));
if (isInputProjected())
return res;
res.reserve(header.columns() - pos_to_remove.size());
for (size_t i = 0; i < header.columns(); i++)
{
@ -1150,8 +1147,33 @@ void ActionsDAG::project(const NamesWithAliases & projection)
}
removeUnusedActions();
projectInput();
projected_output = true;
}
void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block)
{
std::unordered_map<std::string_view, std::list<size_t>> names_map;
size_t num_columns = sample_block.columns();
for (size_t pos = 0; pos < num_columns; ++pos)
names_map[sample_block.getByPosition(pos).name].push_back(pos);
for (const auto * input : inputs)
{
auto & positions = names_map[input->result_name];
if (positions.empty())
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
"Not found column {} in block {}", input->result_name, sample_block.dumpStructure());
positions.pop_front();
}
for (const auto & [_, positions] : names_map)
{
for (auto pos : positions)
{
const auto & col = sample_block.getByPosition(pos);
addInput(col.name, col.type);
}
}
}
bool ActionsDAG::tryRestoreColumn(const std::string & column_name)
@ -1227,8 +1249,6 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name)
ActionsDAGPtr ActionsDAG::clone() const
{
auto actions = std::make_shared<ActionsDAG>();
actions->project_input = project_input;
actions->projected_output = projected_output;
std::unordered_map<const Node *, Node *> copy_map;
@ -1322,9 +1342,6 @@ std::string ActionsDAG::dumpDAG() const
out << ' ' << map[node];
out << '\n';
out << "Project input: " << project_input << '\n';
out << "Projected output: " << projected_output << '\n';
return out.str();
}
@ -1409,7 +1426,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
FunctionOverloadResolverPtr func_builder_materialize = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionMaterialize>());
std::map<std::string_view, std::list<size_t>> inputs;
std::unordered_map<std::string_view, std::list<size_t>> inputs;
if (mode == MatchColumnsMode::Name)
{
size_t input_nodes_size = actions_dag->inputs.size();
@ -1525,8 +1542,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions(
}
actions_dag->outputs.swap(projection);
actions_dag->removeUnusedActions();
actions_dag->projectInput();
actions_dag->removeUnusedActions(false);
return actions_dag;
}
@ -1584,10 +1600,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second)
auto it = first_result.find(input_node->result_name);
if (it == first_result.end() || it->second.empty())
{
if (first.project_input)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Cannot find column {} in ActionsDAG result", input_node->result_name);
first.inputs.push_back(input_node);
}
else
@ -1623,13 +1635,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second)
}
}
/// Update output nodes.
if (second.project_input)
{
first.outputs.swap(second.outputs);
first.project_input = true;
}
else
{
/// Add not removed result from first actions.
for (const auto * output_node : first.outputs)
@ -1645,8 +1650,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second)
}
first.nodes.splice(first.nodes.end(), std::move(second.nodes));
first.projected_output = second.projected_output;
}
void ActionsDAG::mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs)
@ -2042,7 +2045,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet &
}
auto res = split(split_nodes);
res.second->project_input = project_input;
return res;
}
@ -2086,7 +2088,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS
dumpDAG());
auto res = split(split_nodes);
res.second->project_input = project_input;
return res;
}
@ -2158,7 +2159,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co
std::unordered_set<const Node *> split_nodes = {node};
auto res = split(split_nodes);
res.second->project_input = project_input;
return res;
}
@ -2745,11 +2745,7 @@ void ActionsDAG::removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions
std::unordered_set<const Node *> used_inputs;
for (const auto * input : inputs)
{
if (removes_filter && input == predicate)
continue;
used_inputs.insert(input);
}
removeUnusedActions(used_inputs);
}

View File

@ -103,13 +103,11 @@ private:
NodeRawConstPtrs inputs;
NodeRawConstPtrs outputs;
bool project_input = false;
bool projected_output = false;
public:
ActionsDAG() = default;
ActionsDAG(ActionsDAG &&) = default;
ActionsDAG(const ActionsDAG &) = delete;
ActionsDAG & operator=(ActionsDAG &&) = default;
ActionsDAG & operator=(const ActionsDAG &) = delete;
explicit ActionsDAG(const NamesAndTypesList & inputs_);
explicit ActionsDAG(const ColumnsWithTypeAndName & inputs_);
@ -168,9 +166,12 @@ public:
/// Call addAlias several times.
void addAliases(const NamesWithAliases & aliases);
/// Add alias actions and remove unused columns from outputs. Also specify result columns order in outputs.
/// Add alias actions. Also specify result columns order in outputs.
void project(const NamesWithAliases & projection);
/// Add input for every column from sample_block which is not mapped to existing input.
void appendInputsForUnusedColumns(const Block & sample_block);
/// If column is not in outputs, try to find it in nodes and insert back into outputs.
bool tryRestoreColumn(const std::string & column_name);
@ -179,10 +180,6 @@ public:
/// Return true if column was removed from inputs.
bool removeUnusedResult(const std::string & column_name);
void projectInput(bool project = true) { project_input = project; }
bool isInputProjected() const { return project_input; }
bool isOutputProjected() const { return projected_output; }
/// Remove actions that are not needed to compute output nodes
void removeUnusedActions(bool allow_remove_inputs = true, bool allow_constant_folding = true);
@ -510,4 +507,15 @@ struct ActionDAGNodes
ActionsDAG::NodeRawConstPtrs nodes;
};
/// Helper for query analysis.
/// If project_input is set, all columns not found in inputs should be removed.
/// Now, we do it before adding a step to query plan by calling appendInputsForUnusedColumns.
struct ActionsAndProjectInputsFlag
{
ActionsDAG dag;
bool project_input = false;
};
using ActionsAndProjectInputsFlagPtr = std::shared_ptr<ActionsAndProjectInputsFlag>;
}

View File

@ -405,6 +405,9 @@ Block createBlockForSet(
}
ScopeStack::Level::Level() = default;
ScopeStack::Level::~Level() = default;
ScopeStack::Level::Level(Level &&) noexcept = default;
FutureSetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets)
@ -499,16 +502,12 @@ public:
}
};
ScopeStack::Level::~Level() = default;
ScopeStack::Level::Level() = default;
ScopeStack::Level::Level(Level &&) noexcept = default;
ActionsMatcher::Data::Data(
ContextPtr context_,
SizeLimits set_size_limit_,
size_t subquery_depth_,
std::reference_wrapper<const NamesAndTypesList> source_columns_,
ActionsDAGPtr actions_dag,
ActionsDAG actions_dag,
PreparedSetsPtr prepared_sets_,
bool no_subqueries_,
bool no_makeset_,
@ -544,13 +543,13 @@ std::vector<std::string_view> ActionsMatcher::Data::getAllColumnNames() const
return index.getAllNames();
}
ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_) : WithContext(context_)
ScopeStack::ScopeStack(ActionsDAG actions_dag, ContextPtr context_) : WithContext(context_)
{
auto & level = stack.emplace_back();
level.actions_dag = std::move(actions_dag);
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getOutputs());
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag.getOutputs());
for (const auto & node : level.actions_dag->getOutputs())
for (const auto & node : level.actions_dag.getOutputs())
if (node->type == ActionsDAG::ActionType::INPUT)
level.inputs.emplace(node->result_name);
}
@ -558,22 +557,21 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_) : WithCon
void ScopeStack::pushLevel(const NamesAndTypesList & input_columns)
{
auto & level = stack.emplace_back();
level.actions_dag = std::make_shared<ActionsDAG>();
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag->getOutputs());
level.index = std::make_unique<ScopeStack::Index>(level.actions_dag.getOutputs());
const auto & prev = stack[stack.size() - 2];
for (const auto & input_column : input_columns)
{
const auto & node = level.actions_dag->addInput(input_column.name, input_column.type);
const auto & node = level.actions_dag.addInput(input_column.name, input_column.type);
level.index->addNode(&node);
level.inputs.emplace(input_column.name);
}
for (const auto & node : prev.actions_dag->getOutputs())
for (const auto & node : prev.actions_dag.getOutputs())
{
if (!level.index->contains(node->result_name))
{
const auto & input = level.actions_dag->addInput({node->column, node->result_type, node->result_name});
const auto & input = level.actions_dag.addInput({node->column, node->result_type, node->result_name});
level.index->addNode(&input);
}
}
@ -598,12 +596,12 @@ size_t ScopeStack::getColumnLevel(const std::string & name)
void ScopeStack::addColumn(ColumnWithTypeAndName column)
{
const auto & node = stack[0].actions_dag->addColumn(std::move(column));
const auto & node = stack[0].actions_dag.addColumn(std::move(column));
stack[0].index->addNode(&node);
for (size_t j = 1; j < stack.size(); ++j)
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
}
@ -612,12 +610,12 @@ void ScopeStack::addAlias(const std::string & name, std::string alias)
{
auto level = getColumnLevel(name);
const auto & source = stack[level].index->getNode(name);
const auto & node = stack[level].actions_dag->addAlias(source, std::move(alias));
const auto & node = stack[level].actions_dag.addAlias(source, std::move(alias));
stack[level].index->addNode(&node);
for (size_t j = level + 1; j < stack.size(); ++j)
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
}
@ -631,12 +629,12 @@ void ScopeStack::addArrayJoin(const std::string & source_name, std::string resul
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression with arrayJoin cannot depend on lambda argument: {}",
source_name);
const auto & node = stack.front().actions_dag->addArrayJoin(*source_node, std::move(result_name));
const auto & node = stack.front().actions_dag.addArrayJoin(*source_node, std::move(result_name));
stack.front().index->addNode(&node);
for (size_t j = 1; j < stack.size(); ++j)
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
}
@ -655,17 +653,17 @@ void ScopeStack::addFunction(
for (const auto & argument : argument_names)
children.push_back(&stack[level].index->getNode(argument));
const auto & node = stack[level].actions_dag->addFunction(function, std::move(children), std::move(result_name));
const auto & node = stack[level].actions_dag.addFunction(function, std::move(children), std::move(result_name));
stack[level].index->addNode(&node);
for (size_t j = level + 1; j < stack.size(); ++j)
{
const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name});
const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name});
stack[j].index->addNode(&input);
}
}
ActionsDAGPtr ScopeStack::popLevel()
ActionsDAG ScopeStack::popLevel()
{
auto res = std::move(stack.back().actions_dag);
stack.pop_back();
@ -674,12 +672,12 @@ ActionsDAGPtr ScopeStack::popLevel()
std::string ScopeStack::dumpNames() const
{
return stack.back().actions_dag->dumpNames();
return stack.back().actions_dag.dumpNames();
}
const ActionsDAG & ScopeStack::getLastActions() const
{
return *stack.back().actions_dag;
return stack.back().actions_dag;
}
const ScopeStack::Index & ScopeStack::getLastActionsIndex() const
@ -1002,7 +1000,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
data.set_size_limit,
data.subquery_depth,
data.source_columns,
std::make_shared<ActionsDAG>(data.source_columns),
ActionsDAG(data.source_columns),
data.prepared_sets,
data.no_subqueries,
data.no_makeset,
@ -1021,10 +1019,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
}
auto dag = index_hint_data.getActions();
dag->project(args);
dag.project(args);
auto index_hint = std::make_shared<FunctionIndexHint>();
index_hint->setActions(std::move(dag));
index_hint->setActions(std::make_shared<ActionsDAG>(std::move(dag)));
// Arguments are removed. We add function instead of constant column to avoid constant folding.
data.addFunction(std::make_unique<FunctionToOverloadResolverAdaptor>(index_hint), {}, column_name);
@ -1284,10 +1282,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
auto lambda_dag = data.actions_stack.popLevel();
String result_name = lambda->arguments->children.at(1)->getColumnName();
lambda_dag->removeUnusedActions(Names(1, result_name));
lambda_dag.removeUnusedActions(Names(1, result_name));
auto lambda_actions = std::make_shared<ExpressionActions>(
lambda_dag,
std::make_shared<ActionsDAG>(std::move(lambda_dag)),
ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes));
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;

View File

@ -1,5 +1,6 @@
#pragma once
#include <deque>
#include <string_view>
#include <Core/ColumnNumbers.h>
#include <Core/ColumnWithTypeAndName.h>
@ -9,6 +10,7 @@
#include <Interpreters/PreparedSets.h>
#include <Parsers/IAST.h>
#include <QueryPipeline/SizeLimits.h>
#include <Interpreters/ActionsDAG.h>
namespace DB
{
@ -43,20 +45,20 @@ struct ScopeStack : WithContext
struct Level
{
ActionsDAGPtr actions_dag;
ActionsDAG actions_dag;
IndexPtr index;
NameSet inputs;
~Level();
Level();
Level(Level &&) noexcept;
~Level();
};
using Levels = std::vector<Level>;
using Levels = std::deque<Level>;
Levels stack;
ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_);
ScopeStack(ActionsDAG actions_dag, ContextPtr context_);
void pushLevel(const NamesAndTypesList & input_columns);
@ -67,7 +69,7 @@ struct ScopeStack : WithContext
void addArrayJoin(const std::string & source_name, std::string result_name);
void addFunction(const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name);
ActionsDAGPtr popLevel();
ActionsDAG popLevel();
const ActionsDAG & getLastActions() const;
const Index & getLastActionsIndex() const;
@ -147,7 +149,7 @@ public:
SizeLimits set_size_limit_,
size_t subquery_depth_,
std::reference_wrapper<const NamesAndTypesList> source_columns_,
ActionsDAGPtr actions_dag,
ActionsDAG actions_dag,
PreparedSetsPtr prepared_sets_,
bool no_subqueries_,
bool no_makeset_,
@ -182,7 +184,7 @@ public:
actions_stack.addFunction(function, argument_names, std::move(result_name));
}
ActionsDAGPtr getActions()
ActionsDAG getActions()
{
return actions_stack.popLevel();
}

View File

@ -8,8 +8,6 @@
#include <Core/NamesAndAliases.h>
#include <Storages/ColumnsDescription.h>
#include <vector>
#include <atomic>
#include <ctime>

View File

@ -679,6 +679,9 @@ struct ContextSharedPart : boost::noncopyable
}
}
LOG_TRACE(log, "Shutting down AccessControl");
access_control->shutdown();
{
std::lock_guard lock(mutex);

View File

@ -0,0 +1,123 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/ErrorLog.h>
#include <base/getFQDNOrHostName.h>
#include <Common/DateLUTImpl.h>
#include <Common/ThreadPool.h>
#include <Common/ErrorCodes.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/parseQuery.h>
#include <vector>
namespace DB
{
ColumnsDescription ErrorLogElement::getColumnsDescription()
{
ParserCodec codec_parser;
return ColumnsDescription {
{
"hostname",
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Hostname of the server executing the query."
},
{
"event_date",
std::make_shared<DataTypeDate>(),
parseQuery(codec_parser, "(Delta(2), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Event date."
},
{
"event_time",
std::make_shared<DataTypeDateTime>(),
parseQuery(codec_parser, "(Delta(4), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Event time."
},
{
"code",
std::make_shared<DataTypeInt32>(),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Error code."
},
{
"error",
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Error name."
},
{
"value",
std::make_shared<DataTypeUInt64>(),
parseQuery(codec_parser, "(ZSTD(3))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Number of errors happened in time interval."
},
{
"remote",
std::make_shared<DataTypeUInt8>(),
parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS),
"Remote exception (i.e. received during one of the distributed queries)."
}
};
}
void ErrorLogElement::appendToBlock(MutableColumns & columns) const
{
size_t column_idx = 0;
columns[column_idx++]->insert(getFQDNOrHostName());
columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[column_idx++]->insert(event_time);
columns[column_idx++]->insert(code);
columns[column_idx++]->insert(ErrorCodes::getName(code));
columns[column_idx++]->insert(value);
columns[column_idx++]->insert(remote);
}
struct ValuePair
{
UInt64 local = 0;
UInt64 remote = 0;
};
void ErrorLog::stepFunction(TimePoint current_time)
{
/// Static lazy initialization to avoid polluting the header with implementation details
static std::vector<ValuePair> previous_values(ErrorCodes::end());
auto event_time = std::chrono::system_clock::to_time_t(current_time);
for (ErrorCodes::ErrorCode code = 0, end = ErrorCodes::end(); code < end; ++code)
{
const auto & error = ErrorCodes::values[code].get();
if (error.local.count != previous_values.at(code).local)
{
ErrorLogElement local_elem {
.event_time=event_time,
.code=code,
.value=error.local.count - previous_values.at(code).local,
.remote=false
};
this->add(std::move(local_elem));
previous_values[code].local = error.local.count;
}
if (error.remote.count != previous_values.at(code).remote)
{
ErrorLogElement remote_elem {
.event_time=event_time,
.code=code,
.value=error.remote.count - previous_values.at(code).remote,
.remote=true
};
this->add(std::move(remote_elem));
previous_values[code].remote = error.remote.count;
}
}
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Interpreters/SystemLog.h>
#include <Interpreters/PeriodicLog.h>
#include <Common/ErrorCodes.h>
#include <Core/NamesAndTypes.h>
#include <Core/NamesAndAliases.h>
#include <Storages/ColumnsDescription.h>
namespace DB
{
/** ErrorLog is a log of error values measured at regular time interval.
*/
struct ErrorLogElement
{
time_t event_time{};
ErrorCodes::ErrorCode code{};
ErrorCodes::Value value{};
bool remote{};
static std::string name() { return "ErrorLog"; }
static ColumnsDescription getColumnsDescription();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const;
};
class ErrorLog : public PeriodicLog<ErrorLogElement>
{
using PeriodicLog<ErrorLogElement>::PeriodicLog;
protected:
void stepFunction(TimePoint current_time) override;
};
}

View File

@ -49,8 +49,9 @@ namespace ErrorCodes
static std::unordered_set<const ActionsDAG::Node *> processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation);
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_)
: settings(settings_)
ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_)
: project_inputs(project_inputs_)
, settings(settings_)
{
actions_dag = actions_dag_->clone();
@ -757,7 +758,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run,
}
}
if (actions_dag->isInputProjected())
if (project_inputs)
{
block.clear();
}
@ -862,7 +863,7 @@ std::string ExpressionActions::dumpActions() const
for (const auto & output_column : output_columns)
ss << output_column.name << " " << output_column.type->getName() << "\n";
ss << "\nproject input: " << actions_dag->isInputProjected() << "\noutput positions:";
ss << "\noutput positions:";
for (auto pos : result_positions)
ss << " " << pos;
ss << "\n";
@ -926,7 +927,6 @@ JSONBuilder::ItemPtr ExpressionActions::toTree() const
map->add("Actions", std::move(actions_array));
map->add("Outputs", std::move(outputs_array));
map->add("Positions", std::move(positions_array));
map->add("Project Input", actions_dag->isInputProjected());
return map;
}
@ -980,7 +980,7 @@ void ExpressionActionsChain::addStep(NameSet non_constant_inputs)
if (column.column && isColumnConst(*column.column) && non_constant_inputs.contains(column.name))
column.column = nullptr;
steps.push_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsDAG>(columns)));
steps.push_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsAndProjectInputsFlag>(ActionsDAG(columns), false)));
}
void ExpressionActionsChain::finalize()
@ -1129,14 +1129,14 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_
std::swap(result_columns, new_result_columns);
}
ActionsDAGPtr & ExpressionActionsChain::Step::actions()
ActionsAndProjectInputsFlagPtr & ExpressionActionsChain::Step::actions()
{
return typeid_cast<ExpressionActionsStep &>(*this).actions_dag;
return typeid_cast<ExpressionActionsStep &>(*this).actions_and_flags;
}
const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const
const ActionsAndProjectInputsFlagPtr & ExpressionActionsChain::Step::actions() const
{
return typeid_cast<const ExpressionActionsStep &>(*this).actions_dag;
return typeid_cast<const ExpressionActionsStep &>(*this).actions_and_flags;
}
}

View File

@ -79,11 +79,13 @@ private:
ColumnNumbers result_positions;
Block sample_block;
bool project_inputs = false;
ExpressionActionsSettings settings;
public:
ExpressionActions() = delete;
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {});
explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false);
ExpressionActions(const ExpressionActions &) = default;
ExpressionActions & operator=(const ExpressionActions &) = default;
@ -173,48 +175,49 @@ struct ExpressionActionsChain : WithContext
/// Remove unused result and update required columns
virtual void finalize(const NameSet & required_output_) = 0;
/// Add projections to expression
virtual void prependProjectInput() const = 0;
virtual void prependProjectInput() = 0;
virtual std::string dump() const = 0;
/// Only for ExpressionActionsStep
ActionsDAGPtr & actions();
const ActionsDAGPtr & actions() const;
ActionsAndProjectInputsFlagPtr & actions();
const ActionsAndProjectInputsFlagPtr & actions() const;
};
struct ExpressionActionsStep : public Step
{
ActionsDAGPtr actions_dag;
ActionsAndProjectInputsFlagPtr actions_and_flags;
bool is_final_projection = false;
explicit ExpressionActionsStep(ActionsDAGPtr actions_dag_, Names required_output_ = Names())
explicit ExpressionActionsStep(ActionsAndProjectInputsFlagPtr actiactions_and_flags_, Names required_output_ = Names())
: Step(std::move(required_output_))
, actions_dag(std::move(actions_dag_))
, actions_and_flags(std::move(actiactions_and_flags_))
{
}
NamesAndTypesList getRequiredColumns() const override
{
return actions_dag->getRequiredColumns();
return actions_and_flags->dag.getRequiredColumns();
}
ColumnsWithTypeAndName getResultColumns() const override
{
return actions_dag->getResultColumns();
return actions_and_flags->dag.getResultColumns();
}
void finalize(const NameSet & required_output_) override
{
if (!actions_dag->isOutputProjected())
actions_dag->removeUnusedActions(required_output_);
if (!is_final_projection)
actions_and_flags->dag.removeUnusedActions(required_output_);
}
void prependProjectInput() const override
void prependProjectInput() override
{
actions_dag->projectInput();
actions_and_flags->project_input = true;
}
std::string dump() const override
{
return actions_dag->dumpDAG();
return actions_and_flags->dag.dumpDAG();
}
};
@ -229,7 +232,7 @@ struct ExpressionActionsChain : WithContext
NamesAndTypesList getRequiredColumns() const override { return required_columns; }
ColumnsWithTypeAndName getResultColumns() const override { return result_columns; }
void finalize(const NameSet & required_output_) override;
void prependProjectInput() const override {} /// TODO: remove unused columns before ARRAY JOIN ?
void prependProjectInput() override {} /// TODO: remove unused columns before ARRAY JOIN ?
std::string dump() const override { return "ARRAY JOIN"; }
};
@ -245,7 +248,7 @@ struct ExpressionActionsChain : WithContext
NamesAndTypesList getRequiredColumns() const override { return required_columns; }
ColumnsWithTypeAndName getResultColumns() const override { return result_columns; }
void finalize(const NameSet & required_output_) override;
void prependProjectInput() const override {} /// TODO: remove unused columns before JOIN ?
void prependProjectInput() override {} /// TODO: remove unused columns before JOIN ?
std::string dump() const override { return "JOIN"; }
};
@ -263,7 +266,7 @@ struct ExpressionActionsChain : WithContext
steps.clear();
}
ActionsDAGPtr getLastActions(bool allow_empty = false)
ExpressionActionsStep * getLastExpressionStep(bool allow_empty = false)
{
if (steps.empty())
{
@ -272,7 +275,15 @@ struct ExpressionActionsChain : WithContext
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty ExpressionActionsChain");
}
return typeid_cast<ExpressionActionsStep *>(steps.back().get())->actions_dag;
return typeid_cast<ExpressionActionsStep *>(steps.back().get());
}
ActionsAndProjectInputsFlagPtr getLastActions(bool allow_empty = false)
{
if (auto * step = getLastExpressionStep(allow_empty))
return step->actions_and_flags;
return nullptr;
}
Step & getLastStep()
@ -286,10 +297,15 @@ struct ExpressionActionsChain : WithContext
Step & lastStep(const NamesAndTypesList & columns)
{
if (steps.empty())
steps.emplace_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsDAG>(columns)));
return addStep(columns);
return *steps.back();
}
Step & addStep(const NamesAndTypesList & columns)
{
return *steps.emplace_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsAndProjectInputsFlag>(ActionsDAG(columns), false)));
}
std::string dumpChain() const;
};

View File

@ -186,7 +186,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
initGlobalSubqueriesAndExternalTables(do_global, is_explain);
auto temp_actions = std::make_shared<ActionsDAG>(sourceColumns());
ActionsDAG temp_actions(sourceColumns());
columns_after_array_join = getColumnsAfterArrayJoin(temp_actions, sourceColumns());
columns_after_join = analyzeJoin(temp_actions, columns_after_array_join);
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
@ -199,7 +199,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
analyzeAggregation(temp_actions);
}
NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns)
NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns)
{
const auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
@ -213,14 +213,14 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & a
getRootActionsNoMakeSet(array_join_expression_list, actions, false);
auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left);
auto sample_columns = actions->getResultColumns();
auto sample_columns = actions.getResultColumns();
array_join->prepare(sample_columns);
actions = std::make_shared<ActionsDAG>(sample_columns);
actions = ActionsDAG(sample_columns);
NamesAndTypesList new_columns_after_array_join;
NameSet added_columns;
for (auto & column : actions->getResultColumns())
for (auto & column : actions.getResultColumns())
{
if (syntax->array_join_result_to_source.contains(column.name))
{
@ -236,7 +236,7 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & a
return new_columns_after_array_join;
}
NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns)
NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns)
{
const auto * select_query = query->as<ASTSelectQuery>();
if (!select_query)
@ -246,9 +246,9 @@ NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const
if (join)
{
getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), actions, false);
auto sample_columns = actions->getNamesAndTypesList();
auto sample_columns = actions.getNamesAndTypesList();
syntax->analyzed_join->addJoinedColumnsAndCorrectTypes(sample_columns, true);
actions = std::make_shared<ActionsDAG>(sample_columns);
actions = ActionsDAG(sample_columns);
}
NamesAndTypesList result_columns = src_columns;
@ -256,7 +256,7 @@ NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const
return result_columns;
}
void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
void ExpressionAnalyzer::analyzeAggregation(ActionsDAG & temp_actions)
{
/** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions),
* as well as a set of columns obtained after the aggregation, if any,
@ -272,7 +272,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
if (!has_aggregation)
{
aggregated_columns = temp_actions->getNamesAndTypesList();
aggregated_columns = temp_actions.getNamesAndTypesList();
return;
}
@ -321,7 +321,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
ssize_t group_size = group_elements_ast.size();
const auto & column_name = group_elements_ast[j]->getColumnName();
const auto * node = temp_actions->tryFindInOutputs(column_name);
const auto * node = temp_actions.tryFindInOutputs(column_name);
if (!node)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier (in GROUP BY): {}", column_name);
@ -375,7 +375,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
getRootActionsNoMakeSet(group_asts[i], temp_actions, false);
const auto & column_name = group_asts[i]->getColumnName();
const auto * node = temp_actions->tryFindInOutputs(column_name);
const auto * node = temp_actions.tryFindInOutputs(column_name);
if (!node)
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier (in GROUP BY): {}", column_name);
@ -434,7 +434,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions)
has_const_aggregation_keys = select_query->group_by_with_constant_keys;
}
else
aggregated_columns = temp_actions->getNamesAndTypesList();
aggregated_columns = temp_actions.getNamesAndTypesList();
for (const auto & desc : aggregate_descriptions)
aggregated_columns.emplace_back(desc.column_name, desc.function->getResultType());
@ -465,7 +465,7 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o
return storage_set->getSet();
}
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(
@ -485,7 +485,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_
actions = visitor_data.getActions();
}
void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts)
void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAG & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(
@ -507,7 +507,7 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGP
void ExpressionAnalyzer::getRootActionsForHaving(
const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts)
{
LogAST log;
ActionsVisitor::Data visitor_data(
@ -528,7 +528,7 @@ void ExpressionAnalyzer::getRootActionsForHaving(
}
void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions)
void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions)
{
LogAST log;
ActionsVisitor::Data visitor_data(
@ -548,7 +548,7 @@ void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bo
}
void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions)
void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAG & actions, AggregateDescriptions & descriptions)
{
for (const ASTPtr & ast : aggregates())
{
@ -567,7 +567,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
for (size_t i = 0; i < arguments.size(); ++i)
{
const std::string & name = arguments[i]->getColumnName();
const auto * dag_node = actions->tryFindInOutputs(name);
const auto * dag_node = actions.tryFindInOutputs(name);
if (!dag_node)
{
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
@ -659,7 +659,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
1 /* nulls_direction */));
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
getRootActions(column_ast, false, actions_dag);
getRootActions(column_ast, false, *actions_dag);
desc.partition_by_actions.push_back(std::move(actions_dag));
}
}
@ -680,7 +680,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
order_by_element.nulls_direction));
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
getRootActions(column_ast, false, actions_dag);
getRootActions(column_ast, false, *actions_dag);
desc.order_by_actions.push_back(std::move(actions_dag));
}
}
@ -720,7 +720,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_,
}
}
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions)
{
auto current_context = getContext();
@ -737,13 +737,13 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
desc, elem.definition.get());
auto [it, inserted] = window_descriptions.insert(
{desc.window_name, desc});
{elem.name, std::move(desc)});
if (!inserted)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Window '{}' is defined twice in the WINDOW clause",
desc.window_name);
elem.name);
}
}
}
@ -776,7 +776,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
for (size_t i = 0; i < arguments.size(); ++i)
{
const std::string & name = arguments[i]->getColumnName();
const auto * node = actions->tryFindInOutputs(name);
const auto * node = actions.tryFindInOutputs(name);
if (!node)
{
@ -817,13 +817,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
{
const auto & definition = function_node.window_definition->as<
const ASTWindowDefinition &>();
auto default_window_name = definition.getDefaultWindowName();
WindowDescription desc;
desc.window_name = definition.getDefaultWindowName();
desc.window_name = default_window_name;
makeWindowDescriptionFromAST(*current_context, window_descriptions,
desc, &definition);
auto [it, inserted] = window_descriptions.insert(
{desc.window_name, desc});
{default_window_name, desc});
if (!inserted)
{
@ -871,7 +872,7 @@ const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() cons
}
/// "Big" ARRAY JOIN.
ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool array_join_is_left) const
ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & actions, bool array_join_is_left) const
{
NameSet result_columns;
for (const auto & result_source : syntax->array_join_result_to_source)
@ -879,8 +880,8 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr
/// Assign new names to columns, if needed.
if (result_source.first != result_source.second)
{
const auto & node = actions->findInOutputs(result_source.second);
actions->getOutputs().push_back(&actions->addAlias(node, result_source.first));
const auto & node = actions.findInOutputs(result_source.second);
actions.getOutputs().push_back(&actions.addAlias(node, result_source.first));
}
/// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names.
@ -890,7 +891,7 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr
return std::make_shared<ArrayJoinAction>(result_columns, array_join_is_left, getContext());
}
ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types)
ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types)
{
const auto * select_query = getSelectQuery();
@ -900,9 +901,9 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
getRootActions(array_join_expression_list, only_types, step.actions());
getRootActions(array_join_expression_list, only_types, step.actions()->dag);
auto array_join = addMultipleArrayJoinAction(step.actions(), is_array_join_left);
auto array_join = addMultipleArrayJoinAction(step.actions()->dag, is_array_join_left);
before_array_join = chain.getLastActions();
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ArrayJoinStep>(array_join, step.getResultColumns()));
@ -916,20 +917,23 @@ bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain &
{
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join);
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions());
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions()->dag);
return true;
}
JoinPtr SelectQueryExpressionAnalyzer::appendJoin(
ExpressionActionsChain & chain,
ActionsDAGPtr & converting_join_columns)
ActionsAndProjectInputsFlagPtr & converting_join_columns)
{
const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns();
JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_join_columns);
ActionsDAGPtr converting_actions;
JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions);
if (converting_join_columns)
if (converting_actions)
{
converting_join_columns = std::make_shared<ActionsAndProjectInputsFlag>();
converting_join_columns->dag = std::move(*converting_actions);
chain.steps.push_back(std::make_unique<ExpressionActionsChain::ExpressionActionsStep>(converting_join_columns));
chain.addStep();
}
@ -1065,7 +1069,7 @@ static std::unique_ptr<QueryPlan> buildJoinedPlan(
rename_dag->getOutputs()[pos] = &alias;
}
}
rename_dag->projectInput();
rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header);
auto rename_step = std::make_unique<ExpressionStep>(joined_plan->getCurrentDataStream(), std::move(rename_dag));
rename_step->setStepDescription("Rename joined columns");
joined_plan->addStep(std::move(rename_step));
@ -1166,45 +1170,45 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin(
return join;
}
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendPrewhere(
ExpressionActionsChain & chain, bool only_types)
{
const auto * select_query = getSelectQuery();
if (!select_query->prewhere())
return nullptr;
return {};
Names first_action_names;
if (!chain.steps.empty())
first_action_names = chain.steps.front()->getRequiredColumns().getNames();
auto & step = chain.lastStep(sourceColumns());
getRootActions(select_query->prewhere(), only_types, step.actions());
getRootActions(select_query->prewhere(), only_types, step.actions()->dag);
String prewhere_column_name = select_query->prewhere()->getColumnName();
step.addRequiredOutput(prewhere_column_name);
const auto & node = step.actions()->findInOutputs(prewhere_column_name);
const auto & node = step.actions()->dag.findInOutputs(prewhere_column_name);
auto filter_type = node.result_type;
if (!filter_type->canBeUsedInBooleanContext())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}",
filter_type->getName());
ActionsDAGPtr prewhere_actions;
ActionsAndProjectInputsFlagPtr prewhere_actions;
{
/// Remove unused source_columns from prewhere actions.
auto tmp_actions_dag = std::make_shared<ActionsDAG>(sourceColumns());
ActionsDAG tmp_actions_dag(sourceColumns());
getRootActions(select_query->prewhere(), only_types, tmp_actions_dag);
/// Constants cannot be removed since they can be used in other parts of the query.
/// And if they are not used anywhere, except PREWHERE, they will be removed on the next step.
tmp_actions_dag->removeUnusedActions(
tmp_actions_dag.removeUnusedActions(
NameSet{prewhere_column_name},
/* allow_remove_inputs= */ true,
/* allow_constant_folding= */ false);
auto required_columns = tmp_actions_dag->getRequiredColumnsNames();
auto required_columns = tmp_actions_dag.getRequiredColumnsNames();
NameSet required_source_columns(required_columns.begin(), required_columns.end());
required_source_columns.insert(first_action_names.begin(), first_action_names.end());
auto names = step.actions()->getNames();
auto names = step.actions()->dag.getNames();
NameSet name_set(names.begin(), names.end());
for (const auto & column : sourceColumns())
@ -1213,13 +1217,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
Names required_output(name_set.begin(), name_set.end());
prewhere_actions = chain.getLastActions();
prewhere_actions->removeUnusedActions(required_output);
prewhere_actions->dag.removeUnusedActions(required_output);
}
{
ActionsDAGPtr actions;
auto actions = std::make_shared<ActionsAndProjectInputsFlag>();
auto required_columns = prewhere_actions->getRequiredColumns();
auto required_columns = prewhere_actions->dag.getRequiredColumns();
NameSet prewhere_input_names;
for (const auto & col : required_columns)
prewhere_input_names.insert(col.name);
@ -1263,11 +1267,11 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
}
}
actions = std::make_shared<ActionsDAG>(std::move(required_columns));
actions->dag = ActionsDAG(required_columns);
}
else
{
ColumnsWithTypeAndName columns = prewhere_actions->getResultColumns();
ColumnsWithTypeAndName columns = prewhere_actions->dag.getResultColumns();
for (const auto & column : sourceColumns())
{
@ -1278,7 +1282,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere(
}
}
actions = std::make_shared<ActionsDAG>(std::move(columns));
actions->dag = ActionsDAG(columns);
}
chain.steps.emplace_back(
@ -1300,12 +1304,12 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain,
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join);
getRootActions(select_query->where(), only_types, step.actions());
getRootActions(select_query->where(), only_types, step.actions()->dag);
auto where_column_name = select_query->where()->getColumnName();
step.addRequiredOutput(where_column_name);
const auto & node = step.actions()->findInOutputs(where_column_name);
const auto & node = step.actions()->dag.findInOutputs(where_column_name);
auto filter_type = node.result_type;
if (!filter_type->canBeUsedInBooleanContext())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in WHERE: {}",
@ -1332,7 +1336,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
for (const auto & ast_element : ast->children)
{
step.addRequiredOutput(ast_element->getColumnName());
getRootActions(ast_element, only_types, step.actions());
getRootActions(ast_element, only_types, step.actions()->dag);
}
}
}
@ -1341,7 +1345,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
for (const auto & ast : asts)
{
step.addRequiredOutput(ast->getColumnName());
getRootActions(ast, only_types, step.actions());
getRootActions(ast, only_types, step.actions()->dag);
}
}
@ -1350,7 +1354,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
for (auto & child : asts)
{
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
getRootActions(child, only_types, actions_dag);
getRootActions(child, only_types, *actions_dag);
group_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
}
@ -1387,7 +1391,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
if (node.arguments)
for (auto & argument : node.arguments->children)
getRootActions(argument, only_types, step.actions());
getRootActions(argument, only_types, step.actions()->dag);
}
}
@ -1409,7 +1413,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
// recursively together with (1b) as ASTFunction::window_definition.
if (getSelectQuery()->window())
{
getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions());
getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions()->dag);
}
for (const auto & [_, w] : window_descriptions)
@ -1420,7 +1424,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
// definitions (1a).
// Requiring a constant reference to a shared pointer to non-const AST
// doesn't really look sane, but the visitor does indeed require it.
getRootActionsNoMakeSet(f.function_node->clone(), step.actions());
getRootActionsNoMakeSet(f.function_node->clone(), step.actions()->dag);
// (2b) Required function argument columns.
for (const auto & a : f.function_node->arguments->children)
@ -1442,17 +1446,17 @@ void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(Expres
ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window);
for (const auto & expression : syntax->expressions_with_window_function)
getRootActionsForWindowFunctions(expression->clone(), true, step.actions());
getRootActionsForWindowFunctions(expression->clone(), true, step.actions()->dag);
}
void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */)
void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */)
{
const auto * select_query = getAggregatingQuery();
if (!select_query->groupBy() || !(select_query->group_by_with_rollup || select_query->group_by_with_cube))
return;
auto source_columns = before_aggregation->getResultColumns();
auto source_columns = before_aggregation.getResultColumns();
ColumnsWithTypeAndName result_columns;
for (const auto & source_column : source_columns)
@ -1462,9 +1466,11 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & befor
else
result_columns.push_back(source_column);
}
ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation->getNamesAndTypesList());
auto required_output = chain.getLastStep().required_output;
ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList());
step.required_output = std::move(required_output);
step.actions() = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position);
step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position));
}
void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node)
@ -1495,7 +1501,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain,
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
getRootActionsForHaving(select_query->having(), only_types, step.actions());
getRootActionsForHaving(select_query->having(), only_types, step.actions()->dag);
step.addRequiredOutput(select_query->having()->getColumnName());
@ -1508,13 +1514,13 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain,
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
getRootActions(select_query->select(), only_types, step.actions());
getRootActions(select_query->select(), only_types, step.actions()->dag);
for (const auto & child : select_query->select()->children)
appendSelectSkipWindowExpressions(step, child);
}
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order,
ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order,
ManyExpressionActions & order_by_elements_actions)
{
const auto * select_query = getSelectQuery();
@ -1538,7 +1544,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
replaceForPositionalArguments(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY);
}
getRootActions(select_query->orderBy(), only_types, step.actions());
getRootActions(select_query->orderBy(), only_types, step.actions()->dag);
bool with_fill = false;
@ -1601,7 +1607,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai
for (const auto & child : select_query->orderBy()->children)
{
auto actions_dag = std::make_shared<ActionsDAG>(columns_after_join);
getRootActions(child, only_types, actions_dag);
getRootActions(child, only_types, *actions_dag);
order_by_elements_actions.emplace_back(
std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)));
}
@ -1628,7 +1634,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain
ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns);
getRootActions(select_query->limitBy(), only_types, step.actions());
getRootActions(select_query->limitBy(), only_types, step.actions()->dag);
NameSet existing_column_names;
for (const auto & column : aggregated_columns)
@ -1657,7 +1663,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain
return true;
}
ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
{
const auto * select_query = getSelectQuery();
@ -1705,17 +1711,20 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
}
}
auto actions = chain.getLastActions();
actions->project(result_columns);
auto * last_step = chain.getLastExpressionStep();
auto & actions = last_step->actions_and_flags;
actions->dag.project(result_columns);
if (!required_result_columns.empty())
{
result_columns.clear();
for (const auto & column : required_result_columns)
result_columns.emplace_back(column, std::string{});
actions->project(result_columns);
actions->dag.project(result_columns);
}
actions->project_input = true;
last_step->is_final_projection = true;
return actions;
}
@ -1723,14 +1732,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio
void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types)
{
ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns());
getRootActions(expr, only_types, step.actions());
getRootActions(expr, only_types, step.actions()->dag);
step.addRequiredOutput(expr->getColumnName());
}
ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result)
ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result)
{
auto actions_dag = std::make_shared<ActionsDAG>(aggregated_columns);
ActionsDAG actions_dag(aggregated_columns);
NamesWithAliases result_columns;
Names result_names;
@ -1756,13 +1764,15 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
if (add_aliases)
{
if (project_result)
actions_dag->project(result_columns);
if (remove_unused_result)
{
actions_dag.project(result_columns);
}
else
actions_dag->addAliases(result_columns);
actions_dag.addAliases(result_columns);
}
if (!(add_aliases && project_result))
if (!(add_aliases && remove_unused_result))
{
NameSet name_set(result_names.begin(), result_names.end());
/// We will not delete the original columns.
@ -1775,22 +1785,22 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r
}
}
actions_dag->removeUnusedActions(name_set);
actions_dag.removeUnusedActions(name_set);
}
return actions_dag;
return std::make_unique<ActionsDAG>(std::move(actions_dag));
}
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result, CompileExpressions compile_expressions)
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions)
{
return std::make_shared<ExpressionActions>(
getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions));
getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result);
}
ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs)
{
auto actions = std::make_shared<ActionsDAG>(constant_inputs);
getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */);
getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */);
return actions;
}
@ -1805,7 +1815,7 @@ std::unique_ptr<QueryPlan> SelectQueryExpressionAnalyzer::getJoinedPlan()
return std::move(joined_plan);
}
ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::simpleSelectActions()
{
ExpressionActionsChain new_chain(getContext());
appendSelect(new_chain, false);
@ -1845,14 +1855,16 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
ssize_t where_step_num = -1;
ssize_t having_step_num = -1;
ActionsAndProjectInputsFlagPtr prewhere_dag_and_flags;
auto finalize_chain = [&](ExpressionActionsChain & chain) -> ColumnsWithTypeAndName
{
if (prewhere_step_num >= 0)
{
ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num);
auto required_columns_ = prewhere_info->prewhere_actions->getRequiredColumnsNames();
NameSet required_source_columns(required_columns_.begin(), required_columns_.end());
auto prewhere_required_columns = prewhere_dag_and_flags->dag.getRequiredColumnsNames();
NameSet required_source_columns(prewhere_required_columns.begin(), prewhere_required_columns.end());
/// Add required columns to required output in order not to remove them after prewhere execution.
/// TODO: add sampling and final execution to common chain.
for (const auto & column : additional_required_columns_after_prewhere)
@ -1864,6 +1876,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
chain.finalize();
if (prewhere_dag_and_flags)
{
auto dag = std::make_shared<ActionsDAG>(std::move(prewhere_dag_and_flags->dag));
prewhere_info = std::make_shared<PrewhereInfo>(std::move(dag), query.prewhere()->getColumnName());
prewhere_dag_and_flags.reset();
}
finalize(chain, prewhere_step_num, where_step_num, having_step_num, query);
auto res = chain.getLastStep().getResultColumns();
@ -1914,19 +1933,19 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
filter_info->do_remove_column = true;
}
if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage))
if (prewhere_dag_and_flags = query_analyzer.appendPrewhere(chain, !first_stage); prewhere_dag_and_flags)
{
/// Prewhere is always the first one.
prewhere_step_num = 0;
prewhere_info = std::make_shared<PrewhereInfo>(actions, query.prewhere()->getColumnName());
if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings))
if (allowEarlyConstantFolding(prewhere_dag_and_flags->dag, settings))
{
Block before_prewhere_sample = source_header;
if (sanitizeBlock(before_prewhere_sample))
{
auto dag = prewhere_dag_and_flags->dag.clone();
ExpressionActions(
prewhere_info->prewhere_actions,
dag,
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample);
auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName());
/// If the filter column is a constant, record it.
@ -1950,7 +1969,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
{
where_step_num = chain.steps.size() - 1;
before_where = chain.getLastActions();
if (allowEarlyConstantFolding(*before_where, settings))
if (allowEarlyConstantFolding(before_where->dag, settings))
{
Block before_where_sample;
if (chain.steps.size() > 1)
@ -1960,7 +1979,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (sanitizeBlock(before_where_sample))
{
ExpressionActions(
before_where,
before_where->dag.clone(),
ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample);
auto & column_elem
@ -1986,7 +2005,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
before_aggregation = chain.getLastActions();
if (settings.group_by_use_nulls)
query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types);
query_analyzer.appendGroupByModifiers(before_aggregation->dag, chain, only_types);
auto columns_before_aggregation = finalize_chain(chain);
@ -2033,8 +2052,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
true);
auto & step = chain.lastStep(query_analyzer.aggregated_columns);
auto & actions = step.actions();
actions = ActionsDAG::merge(std::move(*actions), std::move(*converting));
auto & actions = step.actions()->dag;
actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting)));
}
}
@ -2070,13 +2089,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
// the main SELECT, similar to what we do for aggregate functions.
if (has_window)
{
query_analyzer.makeWindowDescriptions(chain.getLastActions());
query_analyzer.makeWindowDescriptions(chain.getLastActions()->dag);
query_analyzer.appendWindowFunctionsArguments(chain, only_types || !first_stage);
// Build a list of output columns of the window step.
// 1) We need the columns that are the output of ExpressionActions.
for (const auto & x : chain.getLastActions()->getNamesAndTypesList())
for (const auto & x : chain.getLastActions()->dag.getNamesAndTypesList())
{
query_analyzer.columns_after_window.push_back(x);
}
@ -2113,7 +2132,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
finalize_chain(chain);
query_analyzer.appendExpressionsAfterWindowFunctions(chain, only_types || !first_stage);
for (const auto & x : chain.getLastActions()->getNamesAndTypesList())
for (const auto & x : chain.getLastActions()->dag.getNamesAndTypesList())
{
query_analyzer.columns_after_window.push_back(x);
}
@ -2173,7 +2192,6 @@ void ExpressionAnalysisResult::finalize(
if (prewhere_step_num >= 0)
{
const ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num);
prewhere_info->prewhere_actions->projectInput(false);
NameSet columns_to_remove;
for (const auto & [name, can_remove] : step.required_output)
@ -2206,9 +2224,9 @@ void ExpressionAnalysisResult::finalize(
void ExpressionAnalysisResult::removeExtraColumns() const
{
if (hasWhere())
before_where->projectInput();
before_where->project_input = true;
if (hasHaving())
before_having->projectInput();
before_having->project_input = true;
}
void ExpressionAnalysisResult::checkActions() const
@ -2238,7 +2256,7 @@ std::string ExpressionAnalysisResult::dump() const
if (before_array_join)
{
ss << "before_array_join " << before_array_join->dumpDAG() << "\n";
ss << "before_array_join " << before_array_join->dag.dumpDAG() << "\n";
}
if (array_join)
@ -2248,12 +2266,12 @@ std::string ExpressionAnalysisResult::dump() const
if (before_join)
{
ss << "before_join " << before_join->dumpDAG() << "\n";
ss << "before_join " << before_join->dag.dumpDAG() << "\n";
}
if (before_where)
{
ss << "before_where " << before_where->dumpDAG() << "\n";
ss << "before_where " << before_where->dag.dumpDAG() << "\n";
}
if (prewhere_info)
@ -2268,32 +2286,32 @@ std::string ExpressionAnalysisResult::dump() const
if (before_aggregation)
{
ss << "before_aggregation " << before_aggregation->dumpDAG() << "\n";
ss << "before_aggregation " << before_aggregation->dag.dumpDAG() << "\n";
}
if (before_having)
{
ss << "before_having " << before_having->dumpDAG() << "\n";
ss << "before_having " << before_having->dag.dumpDAG() << "\n";
}
if (before_window)
{
ss << "before_window " << before_window->dumpDAG() << "\n";
ss << "before_window " << before_window->dag.dumpDAG() << "\n";
}
if (before_order_by)
{
ss << "before_order_by " << before_order_by->dumpDAG() << "\n";
ss << "before_order_by " << before_order_by->dag.dumpDAG() << "\n";
}
if (before_limit_by)
{
ss << "before_limit_by " << before_limit_by->dumpDAG() << "\n";
ss << "before_limit_by " << before_limit_by->dag.dumpDAG() << "\n";
}
if (final_projection)
{
ss << "final_projection " << final_projection->dumpDAG() << "\n";
ss << "final_projection " << final_projection->dag.dumpDAG() << "\n";
}
if (!selected_columns.empty())

View File

@ -115,10 +115,10 @@ public:
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
/// If add_aliases, only the calculated values in the desired order and add aliases.
/// If also project_result, than only aliases remain in the output block.
/// If also remove_unused_result, than only aliases remain in the output block.
/// Otherwise, only temporary columns will be deleted from the block.
ActionsDAGPtr getActionsDAG(bool add_aliases, bool project_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true);
ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no);
/// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants.
/// Does not execute subqueries.
@ -139,7 +139,7 @@ public:
const WindowDescriptions & windowDescriptions() const { return window_descriptions; }
void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast);
void makeWindowDescriptions(ActionsDAGPtr actions);
void makeWindowDescriptions(ActionsDAG & actions);
/** Checks if subquery is not a plain StorageSet.
* Because while making set we will read data from StorageSet which is not allowed.
@ -172,34 +172,34 @@ protected:
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain);
ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool is_left) const;
ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAG & actions, bool is_left) const;
void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts = false);
/** Similar to getRootActions but do not make sets when analyzing IN functions. It's used in
* analyzeAggregation which happens earlier than analyzing PREWHERE and WHERE. If we did, the
* prepared sets would not be applicable for MergeTree index optimization.
*/
void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAG & actions, bool only_consts = false);
void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false);
void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts = false);
void getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions);
void getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions);
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
* Create a set of columns aggregated_columns resulting after the aggregation, if any,
* or after all the actions that are normally performed before aggregation.
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
*/
void analyzeAggregation(ActionsDAGPtr & temp_actions);
void makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions);
void analyzeAggregation(ActionsDAG & temp_actions);
void makeAggregateDescriptions(ActionsDAG & actions, AggregateDescriptions & descriptions);
const ASTSelectQuery * getSelectQuery() const;
bool isRemoteStorage() const;
NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns);
NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns);
NamesAndTypesList analyzeJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns);
AggregationKeysInfo getAggregationKeysInfo() const noexcept
{
@ -231,20 +231,20 @@ struct ExpressionAnalysisResult
bool use_grouping_set_key = false;
ActionsDAGPtr before_array_join;
ActionsAndProjectInputsFlagPtr before_array_join;
ArrayJoinActionPtr array_join;
ActionsDAGPtr before_join;
ActionsDAGPtr converting_join_columns;
ActionsAndProjectInputsFlagPtr before_join;
ActionsAndProjectInputsFlagPtr converting_join_columns;
JoinPtr join;
ActionsDAGPtr before_where;
ActionsDAGPtr before_aggregation;
ActionsDAGPtr before_having;
ActionsAndProjectInputsFlagPtr before_where;
ActionsAndProjectInputsFlagPtr before_aggregation;
ActionsAndProjectInputsFlagPtr before_having;
String having_column_name;
bool remove_having_filter = false;
ActionsDAGPtr before_window;
ActionsDAGPtr before_order_by;
ActionsDAGPtr before_limit_by;
ActionsDAGPtr final_projection;
ActionsAndProjectInputsFlagPtr before_window;
ActionsAndProjectInputsFlagPtr before_order_by;
ActionsAndProjectInputsFlagPtr before_limit_by;
ActionsAndProjectInputsFlagPtr final_projection;
/// Columns from the SELECT list, before renaming them to aliases. Used to
/// perform SELECT DISTINCT.
@ -351,12 +351,12 @@ public:
/// Tables that will need to be sent to remote servers for distributed query processing.
const TemporaryTablesMapping & getExternalTables() const { return external_tables; }
ActionsDAGPtr simpleSelectActions();
ActionsAndProjectInputsFlagPtr simpleSelectActions();
/// These appends are public only for tests
void appendSelect(ExpressionActionsChain & chain, bool only_types);
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const;
ActionsAndProjectInputsFlagPtr appendProjectResult(ExpressionActionsChain & chain) const;
private:
StorageMetadataPtr metadata_snapshot;
@ -386,13 +386,13 @@ private:
*/
/// Before aggregation:
ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types);
ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types);
bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types);
JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns);
JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & converting_join_columns);
/// remove_filter is set in ExpressionActionsChain::finalize();
/// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier).
ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types);
ActionsAndProjectInputsFlagPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types);
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &);
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
@ -401,12 +401,12 @@ private:
void appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool only_types);
void appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node);
void appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool only_types);
void appendGroupByModifiers(ActionsDAG & before_aggregation, ExpressionActionsChain & chain, bool only_types);
/// After aggregation:
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
/// appendSelect
ActionsDAGPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &);
ActionsAndProjectInputsFlagPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &);
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
/// appendProjectResult
};

View File

@ -175,11 +175,10 @@ FilterDAGInfoPtr generateFilterActions(
/// Using separate expression analyzer to prevent any possible alias injection
auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot));
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets);
filter_info->actions = analyzer.simpleSelectActions();
filter_info->actions = std::make_unique<ActionsDAG>(std::move(analyzer.simpleSelectActions()->dag));
filter_info->column_name = expr_list->children.at(0)->getColumnName();
filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name});
filter_info->actions->projectInput(false);
for (const auto * node : filter_info->actions->getInputs())
filter_info->actions->getOutputs().push_back(node);
@ -1078,15 +1077,15 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
// with this code. See
// https://github.com/ClickHouse/ClickHouse/issues/19857 for details.
if (analysis_result.before_window)
return analysis_result.before_window->getResultColumns();
return analysis_result.before_window->dag.getResultColumns();
// NOTE: should not handle before_limit_by specially since
// WithMergeableState does not process LIMIT BY
return analysis_result.before_order_by->getResultColumns();
return analysis_result.before_order_by->dag.getResultColumns();
}
Block header = analysis_result.before_aggregation->getResultColumns();
Block header = analysis_result.before_aggregation->dag.getResultColumns();
Block res;
@ -1124,18 +1123,18 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
// It's different from selected_columns, see the comment above for
// WithMergeableState stage.
if (analysis_result.before_window)
return analysis_result.before_window->getResultColumns();
return analysis_result.before_window->dag.getResultColumns();
// In case of query on remote shards executed up to
// WithMergeableStateAfterAggregation*, they can process LIMIT BY,
// since the initiator will not apply LIMIT BY again.
if (analysis_result.before_limit_by)
return analysis_result.before_limit_by->getResultColumns();
return analysis_result.before_limit_by->dag.getResultColumns();
return analysis_result.before_order_by->getResultColumns();
return analysis_result.before_order_by->dag.getResultColumns();
}
return analysis_result.final_projection->getResultColumns();
return analysis_result.final_projection->dag.getResultColumns();
}
@ -1639,12 +1638,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
add_filter_step(parallel_replicas_custom_filter_info, "Parallel replica custom key filter");
if (expressions.before_array_join)
{
QueryPlanStepPtr before_array_join_step
= std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expressions.before_array_join);
before_array_join_step->setStepDescription("Before ARRAY JOIN");
query_plan.addStep(std::move(before_array_join_step));
}
executeExpression(query_plan, expressions.before_array_join, "Before ARRAY JOIN");
if (expressions.array_join)
{
@ -1656,23 +1650,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
}
if (expressions.before_join)
{
QueryPlanStepPtr before_join_step = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
expressions.before_join);
before_join_step->setStepDescription("Before JOIN");
query_plan.addStep(std::move(before_join_step));
}
executeExpression(query_plan, expressions.before_join, "Before JOIN");
/// Optional step to convert key columns to common supertype.
if (expressions.converting_join_columns)
{
QueryPlanStepPtr convert_join_step = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
expressions.converting_join_columns);
convert_join_step->setStepDescription("Convert JOIN columns");
query_plan.addStep(std::move(convert_join_step));
}
executeExpression(query_plan, expressions.converting_join_columns, "Convert JOIN columns");
if (expressions.hasJoin())
{
@ -2116,7 +2098,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
{
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
analysis.prewhere_info = std::make_shared<PrewhereInfo>(analysis.filter_info->actions, analysis.filter_info->column_name);
analysis.prewhere_info->prewhere_actions->projectInput(false);
analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column;
analysis.prewhere_info->need_filter = true;
analysis.filter_info = nullptr;
@ -2127,7 +2108,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis
/// Add row level security actions to prewhere.
analysis.prewhere_info->row_level_filter = analysis.filter_info->actions;
analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name;
analysis.prewhere_info->row_level_filter->projectInput(false);
analysis.filter_info = nullptr;
}
}
@ -2336,7 +2316,7 @@ std::optional<UInt64> InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle
}
if (analysis_result.hasWhere())
{
filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name));
filter_nodes.push_back(&analysis_result.before_where->dag.findInOutputs(analysis_result.where_column_name));
}
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes);
@ -2402,7 +2382,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
auto column = ColumnAggregateFunction::create(func);
column->insertFrom(place);
Block header = analysis_result.before_aggregation->getResultColumns();
Block header = analysis_result.before_aggregation->dag.getResultColumns();
size_t arguments_size = desc.argument_names.size();
DataTypes argument_types(arguments_size);
for (size_t j = 0; j < arguments_size; ++j)
@ -2557,6 +2537,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
query_info.storage_limits = std::make_shared<StorageLimitsList>(storage_limits);
query_info.settings_limit_offset_done = options.settings_limit_offset_done;
storage->read(query_plan, required_columns, storage_snapshot, query_info, context, processing_stage, max_block_size, max_streams);
if (context->hasQueryContext() && !options.is_internal)
@ -2598,10 +2579,14 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
}
}
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter)
{
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter);
query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter);
where_step->setStepDescription("WHERE");
query_plan.addStep(std::move(where_step));
@ -2675,11 +2660,9 @@ static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryE
return result;
}
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
{
auto expression_before_aggregation = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
expression_before_aggregation->setStepDescription("Before GROUP BY");
query_plan.addStep(std::move(expression_before_aggregation));
executeExpression(query_plan, expression, "Before GROUP BY");
AggregateDescriptions aggregates = query_analyzer->aggregates();
const Settings & settings = context->getSettingsRef();
@ -2770,10 +2753,14 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
}
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter)
{
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto having_step
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), remove_filter);
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter);
having_step->setStepDescription("HAVING");
query_plan.addStep(std::move(having_step));
@ -2781,15 +2768,23 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
void InterpreterSelectQuery::executeTotalsAndHaving(
QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final)
QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final)
{
ActionsDAGPtr dag;
if (expression)
{
dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
}
const Settings & settings = context->getSettingsRef();
auto totals_having_step = std::make_unique<TotalsHavingStep>(
query_plan.getCurrentDataStream(),
query_analyzer->aggregates(),
overflow_row,
expression,
std::move(dag),
has_having ? getSelectQuery().having()->getColumnName() : "",
remove_filter,
settings.totals_mode,
@ -2822,12 +2817,16 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific
query_plan.addStep(std::move(step));
}
void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description)
void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, const std::string & description)
{
if (!expression)
return;
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
auto dag = expression->dag.clone();
if (expression->project_input)
dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), std::move(dag));
expression_step->setStepDescription(description);
query_plan.addStep(std::move(expression_step));
@ -2997,11 +2996,9 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const st
}
void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression)
void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression)
{
auto projection_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
projection_step->setStepDescription("Projection");
query_plan.addStep(std::move(projection_step));
executeExpression(query_plan, expression, "Projection");
}

View File

@ -174,13 +174,13 @@ private:
/// Different stages of query execution.
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan);
void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
void executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter);
void executeAggregation(
QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final, bool has_grouping_sets);
void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final);
void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description);
void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final);
void executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter);
static void executeExpression(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, const std::string & description);
/// FIXME should go through ActionsDAG to behave as a proper function
void executeWindow(QueryPlan & query_plan);
void executeOrder(QueryPlan & query_plan, InputOrderInfoPtr sorting_info);
@ -191,7 +191,7 @@ private:
void executeLimitBy(QueryPlan & query_plan);
void executeLimit(QueryPlan & query_plan);
void executeOffset(QueryPlan & query_plan);
static void executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression);
static void executeProjection(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression);
void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct);
void executeExtremes(QueryPlan & query_plan);
void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan);

View File

@ -56,78 +56,32 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const
columns[column_idx++]->insert(current_metrics[i].toUnderType());
}
void MetricLog::startCollectMetric(size_t collect_interval_milliseconds_)
void MetricLog::stepFunction(const std::chrono::system_clock::time_point current_time)
{
collect_interval_milliseconds = collect_interval_milliseconds_;
is_shutdown_metric_thread = false;
metric_flush_thread = std::make_unique<ThreadFromGlobalPool>([this] { metricThreadFunction(); });
}
void MetricLog::stopCollectMetric()
{
bool old_val = false;
if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true))
return;
if (metric_flush_thread)
metric_flush_thread->join();
}
void MetricLog::shutdown()
{
stopCollectMetric();
stopFlushThread();
}
void MetricLog::metricThreadFunction()
{
auto desired_timepoint = std::chrono::system_clock::now();
/// Static lazy initialization to avoid polluting the header with implementation details
/// For differentiation of ProfileEvents counters.
std::vector<ProfileEvents::Count> prev_profile_events(ProfileEvents::end());
static std::vector<ProfileEvents::Count> prev_profile_events(ProfileEvents::end());
while (!is_shutdown_metric_thread)
MetricLogElement elem;
elem.event_time = std::chrono::system_clock::to_time_t(current_time);
elem.event_time_microseconds = timeInMicroseconds(current_time);
elem.profile_events.resize(ProfileEvents::end());
for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i)
{
try
{
const auto current_time = std::chrono::system_clock::now();
MetricLogElement elem;
elem.event_time = std::chrono::system_clock::to_time_t(current_time);
elem.event_time_microseconds = timeInMicroseconds(current_time);
elem.profile_events.resize(ProfileEvents::end());
for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i)
{
const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed);
auto & old_value = prev_profile_events[i];
elem.profile_events[i] = new_value - old_value;
old_value = new_value;
}
elem.current_metrics.resize(CurrentMetrics::end());
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
{
elem.current_metrics[i] = CurrentMetrics::values[i];
}
this->add(std::move(elem));
/// We will record current time into table but align it to regular time intervals to avoid time drift.
/// We may drop some time points if the server is overloaded and recording took too much time.
while (desired_timepoint <= current_time)
desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds);
std::this_thread::sleep_until(desired_timepoint);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed);
auto & old_value = prev_profile_events[i];
elem.profile_events[i] = new_value - old_value;
old_value = new_value;
}
elem.current_metrics.resize(CurrentMetrics::end());
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
{
elem.current_metrics[i] = CurrentMetrics::values[i];
}
this->add(std::move(elem));
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Interpreters/SystemLog.h>
#include <Interpreters/PeriodicLog.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Common/ThreadPool_fwd.h>
@ -9,7 +10,6 @@
#include <Storages/ColumnsDescription.h>
#include <vector>
#include <atomic>
#include <ctime>
@ -33,26 +33,12 @@ struct MetricLogElement
void appendToBlock(MutableColumns & columns) const;
};
class MetricLog : public SystemLog<MetricLogElement>
class MetricLog : public PeriodicLog<MetricLogElement>
{
using SystemLog<MetricLogElement>::SystemLog;
using PeriodicLog<MetricLogElement>::PeriodicLog;
public:
void shutdown() override;
/// Launches a background thread to collect metrics with interval
void startCollectMetric(size_t collect_interval_milliseconds_);
/// Stop background thread. Call before shutdown.
void stopCollectMetric();
private:
void metricThreadFunction();
std::unique_ptr<ThreadFromGlobalPool> metric_flush_thread;
size_t collect_interval_milliseconds;
std::atomic<bool> is_shutdown_metric_thread{false};
protected:
void stepFunction(TimePoint current_time) override;
};
}

View File

@ -1137,9 +1137,9 @@ void MutationsInterpreter::prepareMutationStages(std::vector<Stage> & prepared_s
for (const auto & kv : stage.column_to_updated)
{
auto column_name = kv.second->getColumnName();
const auto & dag_node = actions->findInOutputs(column_name);
const auto & alias = actions->addAlias(dag_node, kv.first);
actions->addOrReplaceInOutputs(alias);
const auto & dag_node = actions->dag.findInOutputs(column_name);
const auto & alias = actions->dag.addAlias(dag_node, kv.first);
actions->dag.addOrReplaceInOutputs(alias);
}
}
@ -1202,7 +1202,7 @@ void MutationsInterpreter::Source::read(
{
ActionsDAG::NodeRawConstPtrs nodes(num_filters);
for (size_t i = 0; i < num_filters; ++i)
nodes[i] = &steps[i]->actions()->findInOutputs(names[i]);
nodes[i] = &steps[i]->actions()->dag.findInOutputs(names[i]);
filter = ActionsDAG::buildFilterActionsDAG(nodes);
}
@ -1273,18 +1273,24 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v
for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i)
{
const auto & step = stage.expressions_chain.steps[i];
if (step->actions()->hasArrayJoin())
if (step->actions()->dag.hasArrayJoin())
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "arrayJoin is not allowed in mutations");
if (i < stage.filter_column_names.size())
{
auto dag = step->actions()->dag.clone();
if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute DELETEs.
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false));
plan.addStep(std::make_unique<FilterStep>(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false));
}
else
{
auto dag = step->actions()->dag.clone();
if (step->actions()->project_input)
dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
/// Execute UPDATE or final projection.
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), step->actions()));
plan.addStep(std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), dag));
}
}

View File

@ -0,0 +1,62 @@
#include <Interpreters/PeriodicLog.h>
#include <Interpreters/ErrorLog.h>
#include <Interpreters/MetricLog.h>
namespace DB
{
template <typename LogElement>
void PeriodicLog<LogElement>::startCollect(size_t collect_interval_milliseconds_)
{
collect_interval_milliseconds = collect_interval_milliseconds_;
is_shutdown_metric_thread = false;
flush_thread = std::make_unique<ThreadFromGlobalPool>([this] { threadFunction(); });
}
template <typename LogElement>
void PeriodicLog<LogElement>::stopCollect()
{
bool old_val = false;
if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true))
return;
if (flush_thread)
flush_thread->join();
}
template <typename LogElement>
void PeriodicLog<LogElement>::shutdown()
{
stopCollect();
this->stopFlushThread();
}
template <typename LogElement>
void PeriodicLog<LogElement>::threadFunction()
{
auto desired_timepoint = std::chrono::system_clock::now();
while (!is_shutdown_metric_thread)
{
try
{
const auto current_time = std::chrono::system_clock::now();
stepFunction(current_time);
/// We will record current time into table but align it to regular time intervals to avoid time drift.
/// We may drop some time points if the server is overloaded and recording took too much time.
while (desired_timepoint <= current_time)
desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds);
std::this_thread::sleep_until(desired_timepoint);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
}
#define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class PeriodicLog<ELEMENT>;
SYSTEM_PERIODIC_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG)
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Interpreters/SystemLog.h>
#include <Common/ThreadPool.h>
#include <atomic>
#include <chrono>
#define SYSTEM_PERIODIC_LOG_ELEMENTS(M) \
M(ErrorLogElement) \
M(MetricLogElement)
namespace DB
{
template <typename LogElement>
class PeriodicLog : public SystemLog<LogElement>
{
using SystemLog<LogElement>::SystemLog;
public:
using TimePoint = std::chrono::system_clock::time_point;
/// Launches a background thread to collect metrics with interval
void startCollect(size_t collect_interval_milliseconds_);
/// Stop background thread
void stopCollect();
void shutdown() final;
protected:
virtual void stepFunction(TimePoint current_time) = 0;
private:
void threadFunction();
std::unique_ptr<ThreadFromGlobalPool> flush_thread;
size_t collect_interval_milliseconds;
std::atomic<bool> is_shutdown_metric_thread{false};
};
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <base/types.h>
#include <Core/UUID.h>
#include <tuple>
#include <Parsers/IAST_fwd.h>
#include <Core/QualifiedTableName.h>
#include <Common/Exception.h>

View File

@ -12,6 +12,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/CrashLog.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/ErrorLog.h>
#include <Interpreters/FilesystemCacheLog.h>
#include <Interpreters/FilesystemReadPrefetchesLog.h>
#include <Interpreters/InterpreterCreateQuery.h>
@ -116,6 +117,7 @@ namespace
{
constexpr size_t DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000;
constexpr size_t DEFAULT_ERROR_LOG_COLLECT_INTERVAL_MILLISECONDS = 1000;
/// Creates a system log with MergeTree engine using parameters from config
template <typename TSystemLog>
@ -286,6 +288,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
crash_log = createSystemLog<CrashLog>(global_context, "system", "crash_log", config, "crash_log", "Contains information about stack traces for fatal errors. The table does not exist in the database by default, it is created only when fatal errors occur.");
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log", "Contains logging entries which are normally written to a log file or to stdout.");
metric_log = createSystemLog<MetricLog>(global_context, "system", "metric_log", config, "metric_log", "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.");
error_log = createSystemLog<ErrorLog>(global_context, "system", "error_log", config, "error_log", "Contains history of error values from table system.errors, periodically flushed to disk.");
filesystem_cache_log = createSystemLog<FilesystemCacheLog>(global_context, "system", "filesystem_cache_log", config, "filesystem_cache_log", "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.");
filesystem_read_prefetches_log = createSystemLog<FilesystemReadPrefetchesLog>(
global_context, "system", "filesystem_read_prefetches_log", config, "filesystem_read_prefetches_log", "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.");
@ -320,6 +323,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
logs.emplace_back(text_log.get());
if (metric_log)
logs.emplace_back(metric_log.get());
if (error_log)
logs.emplace_back(error_log.get());
if (asynchronous_metric_log)
logs.emplace_back(asynchronous_metric_log.get());
if (opentelemetry_span_log)
@ -366,7 +371,14 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
{
size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds",
DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS);
metric_log->startCollectMetric(collect_interval_milliseconds);
metric_log->startCollect(collect_interval_milliseconds);
}
if (error_log)
{
size_t collect_interval_milliseconds = config.getUInt64("error_log.collect_interval_milliseconds",
DEFAULT_ERROR_LOG_COLLECT_INTERVAL_MILLISECONDS);
error_log->startCollect(collect_interval_milliseconds);
}
if (crash_log)

View File

@ -40,6 +40,7 @@ class PartLog;
class TextLog;
class TraceLog;
class CrashLog;
class ErrorLog;
class MetricLog;
class AsynchronousMetricLog;
class OpenTelemetrySpanLog;
@ -72,6 +73,7 @@ struct SystemLogs
std::shared_ptr<CrashLog> crash_log; /// Used to log server crashes.
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
std::shared_ptr<ErrorLog> error_log; /// Used to log errors.
std::shared_ptr<FilesystemCacheLog> filesystem_cache_log;
std::shared_ptr<FilesystemReadPrefetchesLog> filesystem_read_prefetches_log;
std::shared_ptr<S3QueueLog> s3_queue_log;

View File

@ -31,7 +31,7 @@ TEST(ActionsVisitor, VisitLiteral)
size_limits_for_set,
size_t(0),
name_and_types,
std::make_shared<ActionsDAG>(name_and_types),
ActionsDAG(name_and_types),
std::make_shared<PreparedSets>(),
false /* no_subqueries */,
false /* no_makeset */,
@ -39,7 +39,7 @@ TEST(ActionsVisitor, VisitLiteral)
info);
ActionsVisitor(visitor_data).visit(ast);
auto actions = visitor_data.getActions();
ASSERT_EQ(actions->getResultColumns().back().type->getTypeId(), expect_type->getTypeId());
ASSERT_EQ(actions.getResultColumns().back().type->getTypeId(), expect_type->getTypeId());
}
TEST(ActionsVisitor, VisitLiteralWithType)
@ -61,7 +61,7 @@ TEST(ActionsVisitor, VisitLiteralWithType)
size_limits_for_set,
size_t(0),
name_and_types,
std::make_shared<ActionsDAG>(name_and_types),
ActionsDAG(name_and_types),
std::make_shared<PreparedSets>(),
false /* no_subqueries */,
false /* no_makeset */,
@ -69,5 +69,5 @@ TEST(ActionsVisitor, VisitLiteralWithType)
info);
ActionsVisitor(visitor_data).visit(ast);
auto actions = visitor_data.getActions();
ASSERT_EQ(actions->getResultColumns().back().type->getTypeId(), date_type->getTypeId());
ASSERT_EQ(actions.getResultColumns().back().type->getTypeId(), date_type->getTypeId());
}

View File

@ -60,6 +60,8 @@ ASTPtr ASTAlterCommand::clone() const
res->settings_resets = res->children.emplace_back(settings_resets->clone()).get();
if (select)
res->select = res->children.emplace_back(select->clone()).get();
if (sql_security)
res->sql_security = res->children.emplace_back(sql_security->clone()).get();
if (rename_to)
res->rename_to = res->children.emplace_back(rename_to->clone()).get();
@ -522,6 +524,7 @@ void ASTAlterCommand::forEachPointerToChild(std::function<void(void**)> f)
f(reinterpret_cast<void **>(&settings_changes));
f(reinterpret_cast<void **>(&settings_resets));
f(reinterpret_cast<void **>(&select));
f(reinterpret_cast<void **>(&sql_security));
f(reinterpret_cast<void **>(&rename_to));
}

View File

@ -7,7 +7,7 @@ namespace DB
void ASTSQLSecurity::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!type.has_value())
if (!type)
return;
if (definer || is_definer_current_user)

View File

@ -11,7 +11,7 @@
namespace DB
{
ActionsChainStep::ActionsChainStep(ActionsDAGPtr actions_,
ActionsChainStep::ActionsChainStep(ActionsAndProjectInputsFlagPtr actions_,
bool use_actions_nodes_as_output_columns_,
ColumnsWithTypeAndName additional_output_columns_)
: actions(std::move(actions_))
@ -28,12 +28,12 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input
auto child_input_columns_copy = child_input_columns;
std::unordered_set<std::string_view> output_nodes_names;
output_nodes_names.reserve(actions->getOutputs().size());
output_nodes_names.reserve(actions->dag.getOutputs().size());
for (auto & output_node : actions->getOutputs())
for (auto & output_node : actions->dag.getOutputs())
output_nodes_names.insert(output_node->result_name);
for (const auto & node : actions->getNodes())
for (const auto & node : actions->dag.getNodes())
{
auto it = child_input_columns_copy.find(node.result_name);
if (it == child_input_columns_copy.end())
@ -45,20 +45,20 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input
if (output_nodes_names.contains(node.result_name))
continue;
actions->getOutputs().push_back(&node);
actions->dag.getOutputs().push_back(&node);
output_nodes_names.insert(node.result_name);
}
actions->removeUnusedActions();
actions->dag.removeUnusedActions();
/// TODO: Analyzer fix ActionsDAG input and constant nodes with same name
actions->projectInput();
actions->project_input = true;
initialize();
}
void ActionsChainStep::dump(WriteBuffer & buffer) const
{
buffer << "DAG" << '\n';
buffer << actions->dumpDAG();
buffer << actions->dag.dumpDAG();
if (!available_output_columns.empty())
{
@ -84,7 +84,7 @@ String ActionsChainStep::dump() const
void ActionsChainStep::initialize()
{
auto required_columns_names = actions->getRequiredColumnsNames();
auto required_columns_names = actions->dag.getRequiredColumnsNames();
input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end());
available_output_columns.clear();
@ -93,7 +93,7 @@ void ActionsChainStep::initialize()
{
std::unordered_set<std::string_view> available_output_columns_names;
for (const auto & node : actions->getNodes())
for (const auto & node : actions->dag.getNodes())
{
if (available_output_columns_names.contains(node.result_name))
continue;

View File

@ -48,18 +48,18 @@ public:
* If use_actions_nodes_as_output_columns = true output columns are initialized using actions dag nodes.
* If additional output columns are specified they are added to output columns.
*/
explicit ActionsChainStep(ActionsDAGPtr actions_,
explicit ActionsChainStep(ActionsAndProjectInputsFlagPtr actions_,
bool use_actions_nodes_as_output_columns = true,
ColumnsWithTypeAndName additional_output_columns_ = {});
/// Get actions
ActionsDAGPtr & getActions()
ActionsAndProjectInputsFlagPtr & getActions()
{
return actions;
}
/// Get actions
const ActionsDAGPtr & getActions() const
const ActionsAndProjectInputsFlagPtr & getActions() const
{
return actions;
}
@ -98,7 +98,7 @@ public:
private:
void initialize();
ActionsDAGPtr actions;
ActionsAndProjectInputsFlagPtr actions;
bool use_actions_nodes_as_output_columns = true;

View File

@ -90,7 +90,7 @@ public:
ActionsDAGPtr alias_column_actions_dag = std::make_shared<ActionsDAG>();
PlannerActionsVisitor actions_visitor(planner_context, false);
auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression());
auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression());
if (outputs.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size());
@ -340,7 +340,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr
QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere();
PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node);
auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node);
if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::ILLEGAL_PREWHERE,
"Invalid PREWHERE. Expected single boolean expression. In query {}",

View File

@ -329,12 +329,16 @@ public:
};
void addExpressionStep(QueryPlan & query_plan,
const ActionsDAGPtr & expression_actions,
const ActionsAndProjectInputsFlagPtr & expression_actions,
const std::string & step_description,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
{
result_actions_to_execute.push_back(expression_actions);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression_actions);
auto actions = expression_actions->dag.clone();
if (expression_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), actions);
expression_step->setStepDescription(step_description);
query_plan.addStep(std::move(expression_step));
}
@ -344,9 +348,13 @@ void addFilterStep(QueryPlan & query_plan,
const std::string & step_description,
std::vector<ActionsDAGPtr> & result_actions_to_execute)
{
result_actions_to_execute.push_back(filter_analysis_result.filter_actions);
auto actions = filter_analysis_result.filter_actions->dag.clone();
if (filter_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
auto where_step = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(),
filter_analysis_result.filter_actions,
actions,
filter_analysis_result.filter_column_name,
filter_analysis_result.remove_filter_column);
where_step->setStepDescription(step_description);
@ -545,14 +553,21 @@ void addTotalsHavingStep(QueryPlan & query_plan,
const auto & having_analysis_result = expression_analysis_result.getHaving();
bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube();
ActionsDAGPtr actions;
if (having_analysis_result.filter_actions)
result_actions_to_execute.push_back(having_analysis_result.filter_actions);
{
actions = having_analysis_result.filter_actions->dag.clone();
if (having_analysis_result.filter_actions->project_input)
actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header);
result_actions_to_execute.push_back(actions);
}
auto totals_having_step = std::make_unique<TotalsHavingStep>(
query_plan.getCurrentDataStream(),
aggregation_analysis_result.aggregate_descriptions,
query_analysis_result.aggregate_overflow_row,
having_analysis_result.filter_actions,
actions,
having_analysis_result.filter_column_name,
having_analysis_result.remove_filter_column,
settings.totals_mode,
@ -728,12 +743,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan,
auto & interpolate_node_typed = interpolate_node->as<InterpolateNode &>();
PlannerActionsVisitor planner_actions_visitor(planner_context);
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
interpolate_node_typed.getExpression());
if (expression_to_interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node");
auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag,
auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag,
interpolate_node_typed.getInterpolateExpression());
if (interpolate_expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node");

View File

@ -413,11 +413,11 @@ private:
class ActionsScopeNode
{
public:
explicit ActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_)
: actions_dag(std::move(actions_dag_))
explicit ActionsScopeNode(ActionsDAG & actions_dag_, QueryTreeNodePtr scope_node_)
: actions_dag(actions_dag_)
, scope_node(std::move(scope_node_))
{
for (const auto & node : actions_dag->getNodes())
for (const auto & node : actions_dag.getNodes())
node_name_to_node[node.result_name] = &node;
}
@ -456,7 +456,7 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No node with name {}. There are only nodes {}",
node_name,
actions_dag->dumpNames());
actions_dag.dumpNames());
return it->second;
}
@ -467,7 +467,7 @@ public:
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addInput(node_name, column_type);
const auto * node = &actions_dag.addInput(node_name, column_type);
node_name_to_node[node->result_name] = node;
return node;
@ -479,7 +479,7 @@ public:
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addInput(column);
const auto * node = &actions_dag.addInput(column);
node_name_to_node[node->result_name] = node;
return node;
@ -491,7 +491,7 @@ public:
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addColumn(column);
const auto * node = &actions_dag.addColumn(column);
node_name_to_node[node->result_name] = node;
return node;
@ -504,7 +504,7 @@ public:
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addFunction(function, children, node_name);
const auto * node = &actions_dag.addFunction(function, children, node_name);
node_name_to_node[node->result_name] = node;
return node;
@ -516,7 +516,7 @@ public:
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addArrayJoin(*child, node_name);
const auto * node = &actions_dag.addArrayJoin(*child, node_name);
node_name_to_node[node->result_name] = node;
return node;
@ -524,14 +524,14 @@ public:
private:
std::unordered_map<std::string_view, const ActionsDAG::Node *> node_name_to_node;
ActionsDAGPtr actions_dag;
ActionsDAG & actions_dag;
QueryTreeNodePtr scope_node;
};
class PlannerActionsVisitorImpl
{
public:
PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_);
@ -595,14 +595,14 @@ private:
bool use_column_identifier_as_action_node_name;
};
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag,
PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag,
const PlannerContextPtr & planner_context_,
bool use_column_identifier_as_action_node_name_)
: planner_context(planner_context_)
, action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_)
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
{
actions_stack.emplace_back(std::move(actions_dag), nullptr);
actions_stack.emplace_back(actions_dag, nullptr);
}
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitorImpl::visit(QueryTreeNodePtr expression_node)
@ -758,7 +758,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
}
auto lambda_actions_dag = std::make_shared<ActionsDAG>();
actions_stack.emplace_back(lambda_actions_dag, node);
actions_stack.emplace_back(*lambda_actions_dag, node);
auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression());
lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name));
@ -886,7 +886,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
for (const auto & argument : function_node.getArguments())
{
auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument);
auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument);
for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes)
{
@ -1013,7 +1013,7 @@ PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_c
, use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_)
{}
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node)
ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node)
{
PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name);
return actions_visitor_impl.visit(expression_node);

View File

@ -37,7 +37,7 @@ public:
* Necessary actions are not added in actions dag output.
* Returns query tree expression node actions dag nodes.
*/
ActionsDAG::NodeRawConstPtrs visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node);
ActionsDAG::NodeRawConstPtrs visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node);
private:
const PlannerContextPtr planner_context;

View File

@ -45,8 +45,10 @@ std::optional<FilterAnalysisResult> analyzeFilter(const QueryTreeNodePtr & filte
{
FilterAnalysisResult result;
result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context);
const auto * output = result.filter_actions->getOutputs().at(0);
result.filter_actions = std::make_shared<ActionsAndProjectInputsFlag>();
result.filter_actions->dag = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context);
const auto * output = result.filter_actions->dag.getOutputs().at(0);
if (output->column && ConstantFilterDescription(*output->column).always_true)
return {};
@ -116,8 +118,9 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
Names aggregation_keys;
ActionsDAGPtr before_aggregation_actions = std::make_shared<ActionsDAG>(input_columns);
before_aggregation_actions->getOutputs().clear();
ActionsAndProjectInputsFlagPtr before_aggregation_actions = std::make_shared<ActionsAndProjectInputsFlag>();
before_aggregation_actions->dag = ActionsDAG(input_columns);
before_aggregation_actions->dag.getOutputs().clear();
std::unordered_set<std::string_view> before_aggregation_actions_output_node_names;
@ -152,7 +155,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key)))
continue;
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node);
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, grouping_set_key_node);
aggregation_keys.reserve(expression_dag_nodes.size());
for (auto & expression_dag_node : expression_dag_nodes)
@ -165,7 +168,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column;
available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name);
aggregation_keys.push_back(expression_dag_node->result_name);
before_aggregation_actions->getOutputs().push_back(expression_dag_node);
before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node);
before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name);
}
}
@ -204,7 +207,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key)))
continue;
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node);
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, group_by_key_node);
aggregation_keys.reserve(expression_dag_nodes.size());
for (auto & expression_dag_node : expression_dag_nodes)
@ -216,7 +219,7 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column;
available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name);
aggregation_keys.push_back(expression_dag_node->result_name);
before_aggregation_actions->getOutputs().push_back(expression_dag_node);
before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node);
before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name);
}
}
@ -230,13 +233,13 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
auto & aggregate_function_node_typed = aggregate_function_node->as<FunctionNode &>();
for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes())
{
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, aggregate_function_node_argument);
auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, aggregate_function_node_argument);
for (auto & expression_dag_node : expression_dag_nodes)
{
if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name))
continue;
before_aggregation_actions->getOutputs().push_back(expression_dag_node);
before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node);
before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name);
}
}
@ -283,8 +286,9 @@ std::optional<WindowAnalysisResult> analyzeWindow(const QueryTreeNodePtr & query
PlannerActionsVisitor actions_visitor(planner_context);
ActionsDAGPtr before_window_actions = std::make_shared<ActionsDAG>(input_columns);
before_window_actions->getOutputs().clear();
ActionsAndProjectInputsFlagPtr before_window_actions = std::make_shared<ActionsAndProjectInputsFlag>();
before_window_actions->dag = ActionsDAG(input_columns);
before_window_actions->dag.getOutputs().clear();
std::unordered_set<std::string_view> before_window_actions_output_node_names;
@ -293,25 +297,25 @@ std::optional<WindowAnalysisResult> analyzeWindow(const QueryTreeNodePtr & query
auto & window_function_node_typed = window_function_node->as<FunctionNode &>();
auto & window_node = window_function_node_typed.getWindowNode()->as<WindowNode &>();
auto expression_dag_nodes = actions_visitor.visit(before_window_actions, window_function_node_typed.getArgumentsNode());
auto expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, window_function_node_typed.getArgumentsNode());
for (auto & expression_dag_node : expression_dag_nodes)
{
if (before_window_actions_output_node_names.contains(expression_dag_node->result_name))
continue;
before_window_actions->getOutputs().push_back(expression_dag_node);
before_window_actions->dag.getOutputs().push_back(expression_dag_node);
before_window_actions_output_node_names.insert(expression_dag_node->result_name);
}
expression_dag_nodes = actions_visitor.visit(before_window_actions, window_node.getPartitionByNode());
expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, window_node.getPartitionByNode());
for (auto & expression_dag_node : expression_dag_nodes)
{
if (before_window_actions_output_node_names.contains(expression_dag_node->result_name))
continue;
before_window_actions->getOutputs().push_back(expression_dag_node);
before_window_actions->dag.getOutputs().push_back(expression_dag_node);
before_window_actions_output_node_names.insert(expression_dag_node->result_name);
}
@ -322,14 +326,14 @@ std::optional<WindowAnalysisResult> analyzeWindow(const QueryTreeNodePtr & query
for (auto & sort_node : order_by_node_list.getNodes())
{
auto & sort_node_typed = sort_node->as<SortNode &>();
expression_dag_nodes = actions_visitor.visit(before_window_actions, sort_node_typed.getExpression());
expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, sort_node_typed.getExpression());
for (auto & expression_dag_node : expression_dag_nodes)
{
if (before_window_actions_output_node_names.contains(expression_dag_node->result_name))
continue;
before_window_actions->getOutputs().push_back(expression_dag_node);
before_window_actions->dag.getOutputs().push_back(expression_dag_node);
before_window_actions_output_node_names.insert(expression_dag_node->result_name);
}
}
@ -362,7 +366,8 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node,
const PlannerContextPtr & planner_context,
ActionsChain & actions_chain)
{
auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context);
auto projection_actions = std::make_shared<ActionsAndProjectInputsFlag>();
projection_actions->dag = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context);
auto projection_columns = query_node.getProjectionColumns();
size_t projection_columns_size = projection_columns.size();
@ -371,7 +376,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node,
NamesWithAliases projection_column_names_with_display_aliases;
projection_column_names_with_display_aliases.reserve(projection_columns_size);
auto & projection_actions_outputs = projection_actions->getOutputs();
auto & projection_actions_outputs = projection_actions->dag.getOutputs();
size_t projection_outputs_size = projection_actions_outputs.size();
if (projection_columns_size != projection_outputs_size)
@ -409,8 +414,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
const PlannerContextPtr & planner_context,
ActionsChain & actions_chain)
{
ActionsDAGPtr before_sort_actions = std::make_shared<ActionsDAG>(input_columns);
auto & before_sort_actions_outputs = before_sort_actions->getOutputs();
auto before_sort_actions = std::make_shared<ActionsAndProjectInputsFlag>();
before_sort_actions->dag = ActionsDAG(input_columns);
auto & before_sort_actions_outputs = before_sort_actions->dag.getOutputs();
before_sort_actions_outputs.clear();
PlannerActionsVisitor actions_visitor(planner_context);
@ -424,7 +430,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
for (const auto & sort_node : order_by_node_list.getNodes())
{
auto & sort_node_typed = sort_node->as<SortNode &>();
auto expression_dag_nodes = actions_visitor.visit(before_sort_actions, sort_node_typed.getExpression());
auto expression_dag_nodes = actions_visitor.visit(before_sort_actions->dag, sort_node_typed.getExpression());
has_with_fill |= sort_node_typed.withFill();
for (auto & action_dag_node : expression_dag_nodes)
@ -440,7 +446,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
if (has_with_fill)
{
for (auto & output_node : before_sort_actions_outputs)
output_node = &before_sort_actions->materializeNode(*output_node);
output_node = &before_sort_actions->dag.materializeNode(*output_node);
}
/// We add only INPUT columns necessary for INTERPOLATE expression in before ORDER BY actions DAG
@ -449,7 +455,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
auto & interpolate_list_node = query_node.getInterpolate()->as<ListNode &>();
PlannerActionsVisitor interpolate_actions_visitor(planner_context);
auto interpolate_actions_dag = std::make_shared<ActionsDAG>();
ActionsDAG interpolate_actions_dag;
for (auto & interpolate_node : interpolate_list_node.getNodes())
{
@ -458,10 +464,10 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
}
std::unordered_map<std::string_view, const ActionsDAG::Node *> before_sort_actions_inputs_name_to_node;
for (const auto & node : before_sort_actions->getInputs())
for (const auto & node : before_sort_actions->dag.getInputs())
before_sort_actions_inputs_name_to_node.emplace(node->result_name, node);
for (const auto & node : interpolate_actions_dag->getNodes())
for (const auto & node : interpolate_actions_dag.getNodes())
{
if (before_sort_actions_dag_output_node_names.contains(node.result_name) ||
node.type != ActionsDAG::ActionType::INPUT)
@ -471,7 +477,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node,
if (input_node_it == before_sort_actions_inputs_name_to_node.end())
{
auto input_column = ColumnWithTypeAndName{node.column, node.result_type, node.result_name};
const auto * input_node = &before_sort_actions->addInput(std::move(input_column));
const auto * input_node = &before_sort_actions->dag.addInput(std::move(input_column));
auto [it, _] = before_sort_actions_inputs_name_to_node.emplace(node.result_name, input_node);
input_node_it = it;
}
@ -496,22 +502,23 @@ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node,
const NameSet & required_output_nodes_names,
ActionsChain & actions_chain)
{
auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context);
auto before_limit_by_actions = std::make_shared<ActionsAndProjectInputsFlag>();
before_limit_by_actions->dag = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context);
NameSet limit_by_column_names_set;
Names limit_by_column_names;
limit_by_column_names.reserve(before_limit_by_actions->getOutputs().size());
for (auto & output_node : before_limit_by_actions->getOutputs())
limit_by_column_names.reserve(before_limit_by_actions->dag.getOutputs().size());
for (auto & output_node : before_limit_by_actions->dag.getOutputs())
{
limit_by_column_names_set.insert(output_node->result_name);
limit_by_column_names.push_back(output_node->result_name);
}
for (const auto & node : before_limit_by_actions->getNodes())
for (const auto & node : before_limit_by_actions->dag.getNodes())
{
if (required_output_nodes_names.contains(node.result_name) &&
!limit_by_column_names_set.contains(node.result_name))
before_limit_by_actions->getOutputs().push_back(&node);
before_limit_by_actions->dag.getOutputs().push_back(&node);
}
auto actions_step_before_limit_by = std::make_unique<ActionsChainStep>(before_limit_by_actions);
@ -605,7 +612,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo
if (sort_analysis_result_optional.has_value() && planner_query_processing_info.isFirstStage() && planner_query_processing_info.getToStage() != QueryProcessingStage::Complete)
{
const auto & before_order_by_actions = sort_analysis_result_optional->before_order_by_actions;
for (const auto & output_node : before_order_by_actions->getOutputs())
for (const auto & output_node : before_order_by_actions->dag.getOutputs())
required_output_nodes_names.insert(output_node->result_name);
}
@ -661,8 +668,10 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo
}
}
auto project_names_actions = std::make_shared<ActionsDAG>(project_names_input);
project_names_actions->project(projection_analysis_result.projection_column_names_with_display_aliases);
auto project_names_actions = std::make_shared<ActionsAndProjectInputsFlag>();
project_names_actions->dag = ActionsDAG(project_names_input);
project_names_actions->dag.project(projection_analysis_result.projection_column_names_with_display_aliases);
project_names_actions->project_input = true;
actions_chain.addStep(std::make_unique<ActionsChainStep>(project_names_actions));
actions_chain.finalize();

View File

@ -17,22 +17,22 @@ namespace DB
struct ProjectionAnalysisResult
{
ActionsDAGPtr projection_actions;
ActionsAndProjectInputsFlagPtr projection_actions;
Names projection_column_names;
NamesWithAliases projection_column_names_with_display_aliases;
ActionsDAGPtr project_names_actions;
ActionsAndProjectInputsFlagPtr project_names_actions;
};
struct FilterAnalysisResult
{
ActionsDAGPtr filter_actions;
ActionsAndProjectInputsFlagPtr filter_actions;
std::string filter_column_name;
bool remove_filter_column = false;
};
struct AggregationAnalysisResult
{
ActionsDAGPtr before_aggregation_actions;
ActionsAndProjectInputsFlagPtr before_aggregation_actions;
Names aggregation_keys;
AggregateDescriptions aggregate_descriptions;
GroupingSetsParamsList grouping_sets_parameters_list;
@ -41,19 +41,19 @@ struct AggregationAnalysisResult
struct WindowAnalysisResult
{
ActionsDAGPtr before_window_actions;
ActionsAndProjectInputsFlagPtr before_window_actions;
std::vector<WindowDescription> window_descriptions;
};
struct SortAnalysisResult
{
ActionsDAGPtr before_order_by_actions;
ActionsAndProjectInputsFlagPtr before_order_by_actions;
bool has_with_fill = false;
};
struct LimitByAnalysisResult
{
ActionsDAGPtr before_limit_by_actions;
ActionsAndProjectInputsFlagPtr before_limit_by_actions;
Names limit_by_column_names;
};

View File

@ -1132,7 +1132,7 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP
}
}
cast_actions_dag->projectInput();
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step = std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable");
plan_to_add_cast.addStep(std::move(cast_join_columns_step));
@ -1178,12 +1178,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
join_table_expression,
planner_context);
join_clauses_and_actions.left_join_expressions_actions->projectInput();
join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header);
auto left_join_expressions_actions_step = std::make_unique<ExpressionStep>(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions);
left_join_expressions_actions_step->setStepDescription("JOIN actions");
left_plan.addStep(std::move(left_join_expressions_actions_step));
join_clauses_and_actions.right_join_expressions_actions->projectInput();
join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header);
auto right_join_expressions_actions_step = std::make_unique<ExpressionStep>(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions);
right_join_expressions_actions_step->setStepDescription("JOIN actions");
right_plan.addStep(std::move(right_join_expressions_actions_step));
@ -1235,7 +1235,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_
output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name);
}
cast_actions_dag->projectInput();
cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header);
auto cast_join_columns_step
= std::make_unique<ExpressionStep>(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag));
cast_join_columns_step->setStepDescription("Cast JOIN USING columns");
@ -1630,7 +1630,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
array_join_column_names.insert(array_join_column_identifier);
auto & array_join_expression_column = array_join_expression->as<ColumnNode &>();
auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow());
for (auto & expression_dag_index_node : expression_dag_index_nodes)
{
@ -1640,7 +1640,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_
}
}
array_join_action_dag->projectInput();
array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header);
join_tree_query_plan.actions_dags.push_back(array_join_action_dag);

View File

@ -183,7 +183,7 @@ const ActionsDAG::Node * appendExpression(
const JoinNode & join_node)
{
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"JOIN {} ON clause contains multiple expressions",
@ -603,7 +603,7 @@ JoinClausesAndActions buildJoinClausesAndActions(
{
auto mixed_join_expressions_actions = std::make_shared<ActionsDAG>(mixed_table_expression_columns);
PlannerActionsVisitor join_expression_visitor(planner_context);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression);
auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression);
if (join_expression_dag_node_raw_pointers.size() != 1)
throw Exception(
ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage());

View File

@ -213,14 +213,14 @@ StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptio
return {limits, leaf_limits};
}
ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node,
ActionsDAG buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node,
const ColumnsWithTypeAndName & input_columns,
const PlannerContextPtr & planner_context)
{
ActionsDAGPtr action_dag = std::make_shared<ActionsDAG>(input_columns);
ActionsDAG action_dag(input_columns);
PlannerActionsVisitor actions_visitor(planner_context);
auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node);
action_dag->getOutputs() = std::move(expression_dag_index_nodes);
action_dag.getOutputs() = std::move(expression_dag_index_nodes);
return action_dag;
}
@ -443,7 +443,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree,
auto filter_actions_dag = std::make_shared<ActionsDAG>();
PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/);
auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree);
auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree);
if (expression_nodes.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Filter actions must return single output node. Actual {}",

View File

@ -47,7 +47,7 @@ StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptio
* Inputs are not used for actions dag outputs.
* Only root query tree expression node is used as actions dag output.
*/
ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node,
ActionsDAG buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node,
const ColumnsWithTypeAndName & input_columns,
const PlannerContextPtr & planner_context);

View File

@ -107,7 +107,7 @@ struct Frame
using Stack = std::vector<Frame>;
/// Second pass optimizations
void optimizePrimaryKeyCondition(const Stack & stack);
void optimizePrimaryKeyConditionAndLimit(const Stack & stack);
void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes);
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);

View File

@ -1,13 +1,13 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/QueryPlan/LimitStep.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
namespace DB::QueryPlanOptimizations
{
void optimizePrimaryKeyCondition(const Stack & stack)
void optimizePrimaryKeyConditionAndLimit(const Stack & stack)
{
const auto & frame = stack.back();
@ -26,15 +26,25 @@ void optimizePrimaryKeyCondition(const Stack & stack)
for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter)
{
if (auto * filter_step = typeid_cast<FilterStep *>(iter->node->step.get()))
{
source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName());
/// Note: actually, plan optimizations merge Filter and Expression steps.
/// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage,
/// So this is likely not needed.
else if (typeid_cast<ExpressionStep *>(iter->node->step.get()))
continue;
else
}
else if (auto * limit_step = typeid_cast<LimitStep *>(iter->node->step.get()))
{
source_step_with_filter->setLimit(limit_step->getLimitForSorting());
break;
}
else if (typeid_cast<ExpressionStep *>(iter->node->step.get()))
{
/// Note: actually, plan optimizations merge Filter and Expression steps.
/// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage,
/// So this is likely not needed.
continue;
}
else
{
break;
}
}
source_step_with_filter->applyFilters();

View File

@ -176,8 +176,6 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi
dag->mergeInplace(std::move(*expression->clone()));
else
dag = expression->clone();
dag->projectInput(false);
}
/// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain.

View File

@ -115,7 +115,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
while (!stack.empty())
{
optimizePrimaryKeyCondition(stack);
optimizePrimaryKeyConditionAndLimit(stack);
/// NOTE: optimizePrewhere can modify the stack.
/// Prewhere optimization relies on PK optimization (getConditionSelectivityEstimatorByPredicate)

View File

@ -77,7 +77,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo(
AggregateProjectionInfo info;
info.context = interpreter.getContext();
info.before_aggregation = analysis_result.before_aggregation;
info.before_aggregation = analysis_result.before_aggregation->dag.clone();
info.keys = query_analyzer->aggregationKeys().getNames();
info.aggregates = query_analyzer->aggregates();

View File

@ -120,6 +120,7 @@ namespace ProfileEvents
extern const Event SelectedParts;
extern const Event SelectedRanges;
extern const Event SelectedMarks;
extern const Event SelectQueriesWithPrimaryKeyUsage;
}
namespace DB
@ -500,11 +501,11 @@ Pipe ReadFromMergeTree::readInOrder(
Names required_columns,
PoolSettings pool_settings,
ReadType read_type,
UInt64 limit)
UInt64 read_limit)
{
/// For reading in order it makes sense to read only
/// one range per task to reduce number of read rows.
bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < block_size.max_block_size_rows;
bool has_limit_below_one_block = read_type != ReadType::Default && read_limit && read_limit < block_size.max_block_size_rows;
MergeTreeReadPoolPtr pool;
if (is_parallel_reading_from_replicas)
@ -849,10 +850,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
static ActionsDAGPtr createProjection(const Block & header)
{
auto projection = std::make_shared<ActionsDAG>(header.getNamesAndTypesList());
projection->removeUnusedActions(header.getNames());
projection->projectInput();
return projection;
return std::make_shared<ActionsDAG>(header.getNamesAndTypesList());
}
Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
@ -1572,11 +1570,17 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
if (indexes->part_values && indexes->part_values->empty())
return std::make_shared<AnalysisResult>(std::move(result));
if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue())
if (indexes->key_condition.alwaysUnknownOrTrue())
{
throw Exception(ErrorCodes::INDEX_NOT_USED,
"Primary key ({}) is not used and setting 'force_primary_key' is set",
fmt::join(primary_key_column_names, ", "));
if (settings.force_primary_key)
{
throw Exception(ErrorCodes::INDEX_NOT_USED,
"Primary key ({}) is not used and setting 'force_primary_key' is set",
fmt::join(primary_key_column_names, ", "));
}
} else
{
ProfileEvents::increment(ProfileEvents::SelectQueriesWithPrimaryKeyUsage);
}
LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString());
@ -1672,7 +1676,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
return std::make_shared<AnalysisResult>(std::move(result));
}
bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, size_t limit)
bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, size_t read_limit)
{
/// if dirction is not set, use current one
if (!direction)
@ -1683,7 +1687,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction,
if (direction != 1 && query_info.isFinal())
return false;
query_info.input_order_info = std::make_shared<InputOrderInfo>(SortDescription{}, prefix_size, direction, limit);
query_info.input_order_info = std::make_shared<InputOrderInfo>(SortDescription{}, prefix_size, direction, read_limit);
reader_settings.read_in_order = true;
/// In case or read-in-order, don't create too many reading streams.

View File

@ -393,7 +393,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep(
, num_streams{num_streams_}
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery &>(), context))
, should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first))
, limit(query_info.limit)
, query_info_limit(query_info.limit)
, storage_limits(query_info.storage_limits)
{
storage_snapshot->check(column_names);
@ -563,7 +563,7 @@ Pipe ReadFromSystemNumbersStep::makePipe()
{
auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1;
if (limit > 0 && limit < rows_appr)
rows_appr = limit;
rows_appr = query_info_limit;
source->addTotalRowsApprox(rows_appr);
}

View File

@ -41,7 +41,7 @@ private:
size_t num_streams;
std::pair<UInt64, UInt64> limit_length_and_offset;
bool should_pushdown_limit;
UInt64 limit;
UInt64 query_info_limit;
std::shared_ptr<const StorageLimitsList> storage_limits;
};

View File

@ -8,8 +8,9 @@
namespace DB
{
/** Source step that can use filters for more efficient pipeline initialization.
/** Source step that can use filters and limit for more efficient pipeline initialization.
* Filters must be added before pipeline initialization.
* Limit must be set before pipeline initialization.
*/
class SourceStepWithFilter : public ISourceStep
{
@ -49,6 +50,11 @@ public:
filter_dags.push_back(std::move(filter_dag));
}
void setLimit(size_t limit_value)
{
limit = limit_value;
}
/// Apply filters that can optimize reading from storage.
void applyFilters()
{
@ -72,6 +78,7 @@ protected:
PrewhereInfoPtr prewhere_info;
StorageSnapshotPtr storage_snapshot;
ContextPtr context;
std::optional<size_t> limit;
ActionsDAGPtr filter_actions_dag;

View File

@ -178,7 +178,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk)
auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false);
if (dag)
{
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes));
auto actions = std::make_shared<ExpressionActions>(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true);
actions->execute(evaluate_block);
}

View File

@ -7061,7 +7061,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS
filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name));
if (before_where)
filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name));
filter_nodes.nodes.push_back(&before_where->dag.findInOutputs(where_column_name));
return filter_nodes;
}

View File

@ -273,7 +273,7 @@ void StorageMaterializedView::read(
* They may be added in case of distributed query with JOIN.
* In that case underlying table returns joined columns as well.
*/
converting_actions->projectInput(false);
converting_actions->removeUnusedActions();
auto converting_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), converting_actions);
converting_step->setStepDescription("Convert target table structure to MaterializedView structure");
query_plan.addStep(std::move(converting_step));

View File

@ -976,7 +976,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo
}
PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/);
actions_visitor.visit(filter_actions_dag, column_node);
actions_visitor.visit(*filter_actions_dag, column_node);
}
column_names_as_aliases = filter_actions_dag->getRequiredColumnsNames();
if (column_names_as_aliases.empty())
@ -1485,7 +1485,7 @@ void ReadFromMerge::convertAndFilterSourceStream(
query_analysis_pass.run(query_tree, local_context);
PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/);
const auto & nodes = actions_visitor.visit(actions_dag, query_tree);
const auto & nodes = actions_visitor.visit(*actions_dag, query_tree);
if (nodes.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size());

View File

@ -35,9 +35,12 @@
#include <Parsers/getInsertQuery.h>
#include <Parsers/ASTFunction.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/StorageFactory.h>
#include <Storages/transformQueryForExternalDatabase.h>
@ -106,28 +109,79 @@ ColumnsDescription StoragePostgreSQL::getTableStructureFromData(
return ColumnsDescription{columns_info->columns};
}
Pipe StoragePostgreSQL::read(
const Names & column_names_,
namespace
{
class ReadFromPostgreSQL : public SourceStepWithFilter
{
public:
ReadFromPostgreSQL(
const Names & column_names_,
const SelectQueryInfo & query_info_,
const StorageSnapshotPtr & storage_snapshot_,
const ContextPtr & context_,
Block sample_block,
size_t max_block_size_,
String remote_table_schema_,
String remote_table_name_,
postgres::ConnectionHolderPtr connection_)
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_)
, logger(getLogger("ReadFromPostgreSQL"))
, max_block_size(max_block_size_)
, remote_table_schema(remote_table_schema_)
, remote_table_name(remote_table_name_)
, connection(std::move(connection_))
{
}
std::string getName() const override { return "ReadFromPostgreSQL"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override
{
std::optional<size_t> transform_query_limit;
if (limit && !filter_actions_dag)
transform_query_limit = limit;
/// Connection is already made to the needed database, so it should not be present in the query;
/// remote_table_schema is empty if it is not specified, will access only table_name.
String query = transformQueryForExternalDatabase(
query_info,
required_source_columns,
storage_snapshot->metadata->getColumns().getOrdinary(),
IdentifierQuotingStyle::DoubleQuotes,
LiteralEscapingStyle::PostgreSQL,
remote_table_schema,
remote_table_name,
context,
transform_query_limit);
LOG_TRACE(logger, "Query: {}", query);
pipeline.init(Pipe(std::make_shared<PostgreSQLSource<>>(std::move(connection), query, getOutputStream().header, max_block_size)));
}
LoggerPtr logger;
size_t max_block_size;
String remote_table_schema;
String remote_table_name;
postgres::ConnectionHolderPtr connection;
};
}
void StoragePostgreSQL::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info_,
ContextPtr context_,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size_,
size_t max_block_size,
size_t /*num_streams*/)
{
storage_snapshot->check(column_names_);
/// Connection is already made to the needed database, so it should not be present in the query;
/// remote_table_schema is empty if it is not specified, will access only table_name.
String query = transformQueryForExternalDatabase(
query_info_,
column_names_,
storage_snapshot->metadata->getColumns().getOrdinary(),
IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_);
LOG_TRACE(log, "Query: {}", query);
storage_snapshot->check(column_names);
Block sample_block;
for (const String & column_name : column_names_)
for (const String & column_name : column_names)
{
auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name);
WhichDataType which(column_data.type);
@ -136,7 +190,17 @@ Pipe StoragePostgreSQL::read(
sample_block.insert({ column_data.type, column_data.name });
}
return Pipe(std::make_shared<PostgreSQLSource<>>(pool->get(), query, sample_block, max_block_size_));
auto reading = std::make_unique<ReadFromPostgreSQL>(
column_names,
query_info,
storage_snapshot,
local_context,
sample_block,
max_block_size,
remote_table_schema,
remote_table_name,
pool->get());
query_plan.addStep(std::move(reading));
}

View File

@ -37,11 +37,12 @@ public:
String getName() const override { return "PostgreSQL"; }
Pipe read(
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;

View File

@ -288,7 +288,8 @@ String transformQueryForExternalDatabaseImpl(
LiteralEscapingStyle literal_escaping_style,
const String & database,
const String & table,
ContextPtr context)
ContextPtr context,
std::optional<size_t> limit)
{
bool strict = context->getSettingsRef().external_table_strict_query;
@ -374,6 +375,9 @@ String transformQueryForExternalDatabaseImpl(
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where));
}
if (limit)
select->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::make_shared<ASTLiteral>(*limit));
ASTPtr select_ptr = select;
dropAliases(select_ptr);
@ -399,7 +403,8 @@ String transformQueryForExternalDatabase(
LiteralEscapingStyle literal_escaping_style,
const String & database,
const String & table,
ContextPtr context)
ContextPtr context,
std::optional<size_t> limit)
{
if (!query_info.syntax_analyzer_result)
{
@ -424,7 +429,8 @@ String transformQueryForExternalDatabase(
literal_escaping_style,
database,
table,
context);
context,
limit);
}
auto clone_query = query_info.query->clone();
@ -436,7 +442,8 @@ String transformQueryForExternalDatabase(
literal_escaping_style,
database,
table,
context);
context,
limit);
}
}

View File

@ -21,6 +21,8 @@ class IAST;
* and WHERE contains subset of (AND-ed) conditions from original query,
* that contain only compatible expressions.
*
* If limit is passed additionally apply LIMIT in result query.
*
* Compatible expressions are comparisons of identifiers, constants, and logical operations on them.
*
* Throws INCORRECT_QUERY if external_table_strict_query (from context settings)
@ -34,6 +36,7 @@ String transformQueryForExternalDatabase(
LiteralEscapingStyle literal_escaping_style,
const String & database,
const String & table,
ContextPtr context);
ContextPtr context,
std::optional<size_t> limit = {});
}

View File

@ -1065,18 +1065,19 @@ def main() -> int:
)
# rerun helper check
# FIXME: remove rerun_helper check and rely on ci cache only
if check_name not in (
CI.JobNames.BUILD_CHECK,
): # we might want to rerun build report job
rerun_helper = RerunHelper(commit, check_name_with_group)
if rerun_helper.is_already_finished_by_status():
print("WARNING: Rerunning job with GH status ")
status = rerun_helper.get_finished_status()
assert status
previous_status = status.state
print("::group::Commit Status")
print(status)
print("::endgroup::")
# FIXME: try rerun, even if status is present. To enable manual restart via GH interface
# previous_status = status.state
# ci cache check
if not previous_status and not ci_settings.no_ci_cache:

View File

@ -5,6 +5,7 @@
<query_log remove="remove" />
<query_views_log remove="remove" />
<metric_log remove="remove"/>
<error_log remove="remove"/>
<text_log remove="remove"/>
<trace_log remove="remove"/>
<asynchronous_metric_log remove="remove" />

View File

@ -1,5 +1,4 @@
import pytest
import asyncio
import glob
import re
import random
@ -1486,6 +1485,7 @@ def test_backup_all(exclude_system_log_tables):
"processors_profile_log",
"asynchronous_insert_log",
"backup_log",
"error_log",
]
exclude_from_backup += ["system." + table_name for table_name in log_tables]

View File

@ -627,67 +627,126 @@ def test_user_specific_auth(start_cluster):
create_user("superuser2")
create_user("regularuser")
node.query("CREATE TABLE specific_auth (col UInt64) ENGINE=Memory")
node.query("CREATE TABLE specific_auth (col UInt64) ENGINE=MergeTree ORDER BY col")
node.query("INSERT INTO specific_auth VALUES (1)")
assert "Access" in node.query_and_get_error(
"BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')"
def backup_restore(backup, user, should_fail, on_cluster=False, base_backup=None):
on_cluster_clause = "ON CLUSTER 'cluster'" if on_cluster else ""
base_backup = (
f" SETTINGS base_backup = {base_backup}" if base_backup is not None else ""
)
backup_query = (
f"BACKUP TABLE specific_auth {on_cluster_clause} TO {backup} {base_backup}"
)
restore_query = f"RESTORE TABLE specific_auth {on_cluster_clause} FROM {backup}"
if should_fail:
assert "Access" in node.query_and_get_error(backup_query, user=user)
else:
node.query(backup_query, user=user)
node.query("DROP TABLE specific_auth SYNC")
node.query(restore_query, user=user)
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup1/')",
user=None,
should_fail=True,
)
assert "Access" in node.query_and_get_error(
"BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')",
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup1/')",
user="regularuser",
should_fail=True,
)
node.query(
"BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')",
user="superuser1",
)
node.query(
"RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1.zip')",
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup1/')",
user="superuser1",
should_fail=False,
)
node.query(
"BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup2.zip')",
user="superuser2",
)
node.query(
"RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup2.zip')",
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup2/')",
user="superuser2",
should_fail=False,
)
assert "Access" in node.query_and_get_error(
"RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1.zip')",
"RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1/')",
user="regularuser",
)
assert "HTTP response code: 403" in node.query_and_get_error(
"SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1.zip', 'RawBLOB')",
node.query("INSERT INTO specific_auth VALUES (2)")
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')",
user="regularuser",
should_fail=True,
base_backup="S3('http://minio1:9001/root/data/backups/limited/backup1/')",
)
node.query(
"SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1.zip', 'RawBLOB')",
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')",
user="superuser1",
should_fail=False,
base_backup="S3('http://minio1:9001/root/data/backups/limited/backup1/')",
)
assert "Access" in node.query_and_get_error(
"RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')",
user="regularuser",
)
assert "Access Denied" in node.query_and_get_error(
"BACKUP TABLE specific_auth ON CLUSTER 'cluster' TO S3('http://minio1:9001/root/data/backups/limited/backup3/')",
"SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1/*', 'RawBLOB')",
user="regularuser",
)
node.query(
"BACKUP TABLE specific_auth ON CLUSTER 'cluster' TO S3('http://minio1:9001/root/data/backups/limited/backup3/')",
"SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1/*', 'RawBLOB')",
user="superuser1",
)
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup3/')",
user="regularuser",
should_fail=True,
on_cluster=True,
)
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup3/')",
user="superuser1",
should_fail=False,
on_cluster=True,
)
assert "Access Denied" in node.query_and_get_error(
"RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3/')",
user="regularuser",
)
node.query(
"RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3/')",
node.query("INSERT INTO specific_auth VALUES (3)")
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')",
user="regularuser",
should_fail=True,
on_cluster=True,
base_backup="S3('http://minio1:9001/root/data/backups/limited/backup3/')",
)
backup_restore(
"S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')",
user="superuser1",
should_fail=False,
on_cluster=True,
base_backup="S3('http://minio1:9001/root/data/backups/limited/backup3/')",
)
assert "Access Denied" in node.query_and_get_error(
"RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')",
user="regularuser",
)
assert "Access Denied" in node.query_and_get_error(

View File

@ -300,7 +300,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3):
LIMIT 1000000
SETTINGS
s3_max_single_part_upload_size=100,
s3_min_upload_part_size=100000,
s3_min_upload_part_size=1000000,
s3_check_objects_after_upload=0
""",
query_id=insert_query_id,
@ -311,7 +311,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3):
)
assert create_multipart == 1
assert upload_parts == 69
assert upload_parts == 7
assert s3_errors == 3
broken_s3.setup_at_part_upload(

View File

@ -0,0 +1,8 @@
<clickhouse>
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</error_log>
</clickhouse>

View File

@ -756,6 +756,14 @@
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</metric_log>
<!-- Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval. -->
<error_log>
<database>system</database>
<table>error_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
</error_log>
<!--
Asynchronous metric log contains values of metrics from
system.asynchronous_metrics.

View File

@ -1,10 +1,3 @@
import time
import threading
from os import path as p, unlink
from tempfile import NamedTemporaryFile
import helpers
import pytest
from helpers.cluster import ClickHouseCluster
@ -16,6 +9,7 @@ def test_xml_full_conf():
all_confd = [
"configs/config.d/access_control.xml",
"configs/config.d/error_log.xml",
"configs/config.d/keeper_port.xml",
"configs/config.d/logging_no_rotate.xml",
"configs/config.d/log_to_console.xml",

Some files were not shown because too many files have changed in this diff Show More