diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 08ee52e4f1b..638a2408748 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -249,10 +249,13 @@ stop_logs_replication successfuly_saved=0 for table in query_log zookeeper_log trace_log transactions_info_log do - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst + successfuly_saved=$? if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) - clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst || successfuly_saved=$((successfuly_saved+$?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst + successfuly_saved=$((successfuly_saved | $?)) + clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst + successfuly_saved=$((successfuly_saved | $?)) fi done diff --git a/docs/changelogs/v23.9.1.1854-stable.md b/docs/changelogs/v23.9.1.1854-stable.md index 655dd54d81b..aa27cd34478 100644 --- a/docs/changelogs/v23.9.1.1854-stable.md +++ b/docs/changelogs/v23.9.1.1854-stable.md @@ -11,6 +11,7 @@ sidebar_label: 2023 * Remove the `status_info` configuration option and dictionaries status from the default Prometheus handler. [#54090](https://github.com/ClickHouse/ClickHouse/pull/54090) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * The experimental parts metadata cache is removed from the codebase. [#54215](https://github.com/ClickHouse/ClickHouse/pull/54215) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Disable setting `input_format_json_try_infer_numbers_from_strings` by default, so we don't try to infer numbers from strings in JSON formats by default to avoid possible parsing errors when sample data contains strings that looks like a number. [#55099](https://github.com/ClickHouse/ClickHouse/pull/55099) ([Kruglov Pavel](https://github.com/Avogar)). +* IPv6 bloom filter indexes created prior to March 2023 are not compatible with current version and have to be rebuilt. [#54200](https://github.com/ClickHouse/ClickHouse/pull/54200) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). #### New Feature * Added new type of authentication based on SSH keys. It works only for Native TCP protocol. [#41109](https://github.com/ClickHouse/ClickHouse/pull/41109) ([George Gamezardashvili](https://github.com/InfJoker)). diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 9c9caee0e47..f185c11bab3 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -39,8 +39,8 @@ If you need to update rows frequently, we recommend using the [`ReplacingMergeTr ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY], - name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY], + name1 [type1] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr1] [COMMENT ...] [CODEC(codec1)] [STATISTIC(stat1)] [TTL expr1] [PRIMARY KEY] [SETTINGS (name = value, ...)], + name2 [type2] [[NOT] NULL] [DEFAULT|MATERIALIZED|ALIAS|EPHEMERAL expr2] [COMMENT ...] [CODEC(codec2)] [STATISTIC(stat2)] [TTL expr2] [PRIMARY KEY] [SETTINGS (name = value, ...)], ... INDEX index_name1 expr1 TYPE type1(...) [GRANULARITY value1], INDEX index_name2 expr2 TYPE type2(...) [GRANULARITY value2], @@ -56,7 +56,7 @@ ORDER BY expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] [WHERE conditions] [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] -[SETTINGS name=value, ...] +[SETTINGS name = value, ...] ``` For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). @@ -620,7 +620,7 @@ The `TTL` clause can’t be used for key columns. #### Creating a table with `TTL`: ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( d DateTime, a Int TTL d + INTERVAL 1 MONTH, @@ -635,7 +635,7 @@ ORDER BY d; #### Adding TTL to a column of an existing table ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY COLUMN c String TTL d + INTERVAL 1 DAY; ``` @@ -643,7 +643,7 @@ ALTER TABLE example_table #### Altering TTL of the column ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY COLUMN c String TTL d + INTERVAL 1 MONTH; ``` @@ -681,7 +681,7 @@ If a column is not part of the `GROUP BY` expression and is not set explicitly i #### Creating a table with `TTL`: ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( d DateTime, a Int @@ -697,7 +697,7 @@ TTL d + INTERVAL 1 MONTH DELETE, #### Altering `TTL` of the table: ``` sql -ALTER TABLE example_table +ALTER TABLE tab MODIFY TTL d + INTERVAL 1 DAY; ``` @@ -1366,7 +1366,7 @@ In this sample configuration: The statistic declaration is in the columns section of the `CREATE` query for tables from the `*MergeTree*` Family when we enable `set allow_experimental_statistic = 1`. ``` sql -CREATE TABLE example_table +CREATE TABLE tab ( a Int64 STATISTIC(tdigest), b Float64 @@ -1378,8 +1378,8 @@ ORDER BY a We can also manipulate statistics with `ALTER` statements. ```sql -ALTER TABLE example_table ADD STATISTIC b TYPE tdigest; -ALTER TABLE example_table DROP STATISTIC a TYPE tdigest; +ALTER TABLE tab ADD STATISTIC b TYPE tdigest; +ALTER TABLE tab DROP STATISTIC a TYPE tdigest; ``` These lightweight statistics aggregate information about distribution of values in columns. @@ -1390,3 +1390,42 @@ They can be used for query optimization when we enable `set allow_statistic_opti - `tdigest` Stores distribution of values from numeric columns in [TDigest](https://github.com/tdunning/t-digest) sketch. + +## Column-level Settings {#column-level-settings} + +Certain MergeTree settings can be override at column level: + +- `max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. +- `min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. + +Example: + +```sql +CREATE TABLE tab +( + id Int64, + document String SETTINGS (min_compress_block_size = 16777216, max_compress_block_size = 16777216) +) +ENGINE = MergeTree +ORDER BY id +``` + +Column-level settings can be modified or removed using [ALTER MODIFY COLUMN](/docs/en/sql-reference/statements/alter/column.md), for example: + +- Remove `SETTINGS` from column declaration: + +```sql +ALTER TABLE tab MODIFY COLUMN document REMOVE SETTINGS; +``` + +- Modify a setting: + +```sql +ALTER TABLE tab MODIFY COLUMN document MODIFY SETTING min_compress_block_size = 8192; +``` + +- Reset one or more settings, also removes the setting declaration in the column expression of the table's CREATE query. + +```sql +ALTER TABLE tab MODIFY COLUMN document RESET SETTING min_compress_block_size; +``` diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index 65b2e349707..e63ab65ba07 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -10,7 +10,7 @@ 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. -- `name` ([String](../../sql-reference/data-types/string.md)) — Metric name. +- `metric` ([String](../../sql-reference/data-types/string.md)) — Metric name. - `value` ([Float64](../../sql-reference/data-types/float.md)) — Metric value. **Example** diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 2cb802c863b..676d30f5e44 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -23,10 +23,11 @@ The following actions are supported: - [RENAME COLUMN](#rename-column) — Renames an existing column. - [CLEAR COLUMN](#clear-column) — Resets column values. - [COMMENT COLUMN](#comment-column) — Adds a text comment to the column. -- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression and TTL. +- [MODIFY COLUMN](#modify-column) — Changes column’s type, default expression, TTL, and column settings. - [MODIFY COLUMN REMOVE](#modify-column-remove) — Removes one of the column properties. +- [MODIFY COLUMN MODIFY SETTING](#modify-column-modify-setting) - Changes column settings. +- [MODIFY COLUMN RESET SETTING](#modify-column-reset-setting) - Reset column settings. - [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing. - These actions are described in detail below. ## ADD COLUMN @@ -75,7 +76,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. -:::tip +:::tip You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: @@ -208,7 +209,7 @@ The `ALTER` query for changing columns is replicated. The instructions are saved ## MODIFY COLUMN REMOVE -Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`. +Removes one of the column properties: `DEFAULT`, `ALIAS`, `MATERIALIZED`, `CODEC`, `COMMENT`, `TTL`, `SETTING`. Syntax: @@ -228,6 +229,43 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; - [REMOVE TTL](ttl.md). + +## MODIFY COLUMN MODIFY SETTING + +Modify a column setting. + +Syntax: + +```sql +ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING name=value,...; +``` + +**Example** + +Modify column's `max_compress_block_size` to `1MB`: + +```sql +ALTER TABLE table_name MODIFY COLUMN MODIFY SETTING max_compress_block_size = 1048576; +``` + +## MODIFY COLUMN RESET SETTING + +Reset a column setting, also removes the setting declaration in the column expression of the table's CREATE query. + +Syntax: + +```sql +ALTER TABLE table_name MODIFY COLUMN RESET SETTING name,...; +``` + +**Example** + +Remove column setting `max_compress_block_size` to `1MB`: + +```sql +ALTER TABLE table_name MODIFY COLUMN REMOVE SETTING max_compress_block_size; +``` + ## MATERIALIZE COLUMN Materializes or updates a column with an expression for a default value (`DEFAULT` or `MATERIALIZED`). diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 5659a0565c5..114b8d5ffe3 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -112,7 +112,7 @@ Note that: For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. -- Both tables must have the same order by key and the same primary key. +- Both tables must have the same partition key, the same order by key and the same primary key. - Both tables must have the same indices and projections. - Both tables must have the same storage policy. diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6220009cd21..96f3119e073 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1900,6 +1900,7 @@ try /// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread. async_metrics.start(); + global_context->setAsynchronousMetrics(&async_metrics); main_config_reloader->start(); access_control.startPeriodicReloading(); diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 6bbb9acc0c1..b305b6fca86 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -170,6 +170,7 @@ enum class AccessType M(SYSTEM_RELOAD_MODEL, "SYSTEM RELOAD MODELS, RELOAD MODEL, RELOAD MODELS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_FUNCTION, "SYSTEM RELOAD FUNCTIONS, RELOAD FUNCTION, RELOAD FUNCTIONS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_EMBEDDED_DICTIONARIES, "RELOAD EMBEDDED DICTIONARIES", GLOBAL, SYSTEM_RELOAD) /* implicitly enabled by the grant SYSTEM_RELOAD_DICTIONARY ON *.* */\ + M(SYSTEM_RELOAD_ASYNCHRONOUS_METRICS, "RELOAD ASYNCHRONOUS METRICS", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD, "", GROUP, SYSTEM) \ M(SYSTEM_RESTART_DISK, "SYSTEM RESTART DISK", GLOBAL, SYSTEM) \ M(SYSTEM_MERGES, "SYSTEM STOP MERGES, SYSTEM START MERGES, STOP MERGES, START MERGES", TABLE, SYSTEM) \ diff --git a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp index 7887a1b7175..24ddb7522c8 100644 --- a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp +++ b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.cpp @@ -10,14 +10,6 @@ namespace DB { -/** Rewrites `sum(column +/- literal)` into two individual functions - * `sum(column)` and `literal * count(column)`. - * sum(column + literal) -> sum(column) + literal * count(column) - * sum(literal + column) -> literal * count(column) + sum(column) - * sum(column - literal) -> sum(column) - literal * count(column) - * sum(literal - column) -> literal * count(column) - sum(column) - */ - namespace { @@ -29,6 +21,9 @@ public: void enterImpl(QueryTreeNodePtr & node) { + if (!getSettings().optimize_arithmetic_operations_in_aggregate_functions) + return; + static const std::unordered_set func_supported = { "plus", "minus" diff --git a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h index cb76448b34a..e878a2c0e7a 100644 --- a/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h +++ b/src/Analyzer/Passes/RewriteSumFunctionWithSumAndCountPass.h @@ -5,6 +5,14 @@ namespace DB { +/** + * Rewrites `sum(column +/- literal)` into two individual functions + * `sum(column)` and `literal * count(column)`. + * sum(column + literal) -> sum(column) + literal * count(column) + * sum(literal + column) -> literal * count(column) + sum(column) + * sum(column - literal) -> sum(column) - literal * count(column) + * sum(literal - column) -> literal * count(column) - sum(column) + */ class RewriteSumFunctionWithSumAndCountPass final : public IQueryTreePass { public: diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 31cf1962251..ec94d17f590 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -98,7 +98,7 @@ AsynchronousMetrics::AsynchronousMetrics( } #if defined(OS_LINUX) -void AsynchronousMetrics::openSensors() +void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/class/thermal"); @@ -136,7 +136,7 @@ void AsynchronousMetrics::openSensors() } } -void AsynchronousMetrics::openBlockDevices() +void AsynchronousMetrics::openBlockDevices() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/block"); @@ -163,7 +163,7 @@ void AsynchronousMetrics::openBlockDevices() } } -void AsynchronousMetrics::openEDAC() +void AsynchronousMetrics::openEDAC() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/devices/system/edac"); @@ -194,7 +194,7 @@ void AsynchronousMetrics::openEDAC() } } -void AsynchronousMetrics::openSensorsChips() +void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex) { LOG_TRACE(log, "Scanning /sys/class/hwmon"); @@ -281,7 +281,7 @@ void AsynchronousMetrics::stop() try { { - std::lock_guard lock{mutex}; + std::lock_guard lock(thread_mutex); quit = true; } @@ -306,11 +306,14 @@ AsynchronousMetrics::~AsynchronousMetrics() AsynchronousMetricValues AsynchronousMetrics::getValues() const { - std::lock_guard lock{mutex}; + std::lock_guard lock(data_mutex); return values; } -static auto get_next_update_time(std::chrono::seconds update_period) +namespace +{ + +auto get_next_update_time(std::chrono::seconds update_period) { using namespace std::chrono; @@ -334,6 +337,8 @@ static auto get_next_update_time(std::chrono::seconds update_period) return time_next; } +} + void AsynchronousMetrics::run() { setThreadName("AsyncMetrics"); @@ -344,9 +349,9 @@ void AsynchronousMetrics::run() { // Wait first, so that the first metric collection is also on even time. - std::unique_lock lock{mutex}; + std::unique_lock lock(thread_mutex); if (wait_cond.wait_until(lock, next_update_time, - [this] { return quit; })) + [this] TSA_REQUIRES(thread_mutex) { return quit; })) { break; } @@ -364,6 +369,9 @@ void AsynchronousMetrics::run() } #if USE_JEMALLOC +namespace +{ + uint64_t updateJemallocEpoch() { uint64_t value = 0; @@ -373,7 +381,7 @@ uint64_t updateJemallocEpoch() } template -static Value saveJemallocMetricImpl( +Value saveJemallocMetricImpl( AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) @@ -386,7 +394,7 @@ static Value saveJemallocMetricImpl( } template -static Value saveJemallocMetric(AsynchronousMetricValues & values, +Value saveJemallocMetric(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, @@ -395,13 +403,15 @@ static Value saveJemallocMetric(AsynchronousMetricValues & values, } template -static Value saveAllArenasMetric(AsynchronousMetricValues & values, +Value saveAllArenasMetric(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, fmt::format("stats.arenas.{}.{}", MALLCTL_ARENAS_ALL, metric_name), fmt::format("jemalloc.arenas.all.{}", metric_name)); } + +} #endif @@ -547,21 +557,23 @@ AsynchronousMetrics::NetworkInterfaceStatValues::operator-(const AsynchronousMet #endif -void AsynchronousMetrics::update(TimePoint update_time) +void AsynchronousMetrics::update(TimePoint update_time, bool force_update) { Stopwatch watch; AsynchronousMetricValues new_values; + std::lock_guard lock(data_mutex); + auto current_time = std::chrono::system_clock::now(); - auto time_after_previous_update = current_time - previous_update_time; + auto time_since_previous_update = current_time - previous_update_time; previous_update_time = update_time; double update_interval = 0.; if (first_run) update_interval = update_period.count(); else - update_interval = std::chrono::duration_cast(time_after_previous_update).count() / 1e6; + update_interval = std::chrono::duration_cast(time_since_previous_update).count() / 1e6; new_values["AsynchronousMetricsUpdateInterval"] = { update_interval, "Metrics update interval" }; /// This is also a good indicator of system responsiveness. @@ -815,7 +827,7 @@ void AsynchronousMetrics::update(TimePoint update_time) if (-1 == hz) throw ErrnoException(ErrorCodes::CANNOT_SYSCONF, "Cannot call 'sysconf' to obtain system HZ"); - double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_after_previous_update).count() / 1e9); + double multiplier = 1.0 / hz / (std::chrono::duration_cast(time_since_previous_update).count() / 1e9); size_t num_cpus = 0; ProcStatValuesOther current_other_values{}; @@ -1572,7 +1584,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - updateImpl(new_values, update_time, current_time); + updateImpl(update_time, current_time, force_update, first_run, new_values); new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; @@ -1581,7 +1593,6 @@ void AsynchronousMetrics::update(TimePoint update_time) first_run = false; // Finally, update the current metrics. - std::lock_guard lock(mutex); values = new_values; } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index e3b5142553b..b9a5862dbff 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -56,8 +56,13 @@ struct ProtocolServerMetrics */ class AsynchronousMetrics { +protected: + using Duration = std::chrono::seconds; + using TimePoint = std::chrono::system_clock::time_point; + public: using ProtocolServerMetricsFunc = std::function()>; + AsynchronousMetrics( int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); @@ -69,62 +74,66 @@ public: void stop(); + void update(TimePoint update_time, bool force_update = false); + /// Returns copy of all values. AsynchronousMetricValues getValues() const; protected: - using Duration = std::chrono::seconds; - using TimePoint = std::chrono::system_clock::time_point; - const Duration update_period; - /// Some values are incremental and we have to calculate the difference. - /// On first run we will only collect the values to subtract later. - bool first_run = true; - TimePoint previous_update_time; - Poco::Logger * log; private: - virtual void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) = 0; + virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0; virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; - mutable std::mutex mutex; + std::unique_ptr thread; + + mutable std::mutex thread_mutex; std::condition_variable wait_cond; - bool quit {false}; - AsynchronousMetricValues values; + bool quit TSA_GUARDED_BY(thread_mutex) = false; + + mutable std::mutex data_mutex; + + /// Some values are incremental and we have to calculate the difference. + /// On first run we will only collect the values to subtract later. + bool first_run TSA_GUARDED_BY(data_mutex) = true; + TimePoint previous_update_time TSA_GUARDED_BY(data_mutex); + + AsynchronousMetricValues values TSA_GUARDED_BY(data_mutex); #if defined(OS_LINUX) || defined(OS_FREEBSD) - MemoryStatisticsOS memory_stat; + MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif #if defined(OS_LINUX) - std::optional meminfo; - std::optional loadavg; - std::optional proc_stat; - std::optional cpuinfo; - std::optional file_nr; - std::optional uptime; - std::optional net_dev; + std::optional meminfo TSA_GUARDED_BY(data_mutex); + std::optional loadavg TSA_GUARDED_BY(data_mutex); + std::optional proc_stat TSA_GUARDED_BY(data_mutex); + std::optional cpuinfo TSA_GUARDED_BY(data_mutex); + std::optional file_nr TSA_GUARDED_BY(data_mutex); + std::optional uptime TSA_GUARDED_BY(data_mutex); + std::optional net_dev TSA_GUARDED_BY(data_mutex); - std::optional cgroupmem_limit_in_bytes; - std::optional cgroupmem_usage_in_bytes; - std::optional cgroupcpu_cfs_period; - std::optional cgroupcpu_cfs_quota; - std::optional cgroupcpu_max; + std::optional cgroupmem_limit_in_bytes TSA_GUARDED_BY(data_mutex); + std::optional cgroupmem_usage_in_bytes TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_cfs_period TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_cfs_quota TSA_GUARDED_BY(data_mutex); + std::optional cgroupcpu_max TSA_GUARDED_BY(data_mutex); - std::vector> thermal; + std::vector> thermal TSA_GUARDED_BY(data_mutex); std::unordered_map>> hwmon_devices; + std::unique_ptr>> hwmon_devices TSA_GUARDED_BY(data_mutex); std::vector /* correctable errors */, - std::unique_ptr /* uncorrectable errors */>> edac; + std::unique_ptr /* uncorrectable errors */>> edac TSA_GUARDED_BY(data_mutex); - std::unordered_map> block_devs; + std::unordered_map> block_devs TSA_GUARDED_BY(data_mutex); /// TODO: socket statistics. @@ -154,9 +163,9 @@ private: ProcStatValuesOther operator-(const ProcStatValuesOther & other) const; }; - ProcStatValuesCPU proc_stat_values_all_cpus{}; - ProcStatValuesOther proc_stat_values_other{}; - std::vector proc_stat_values_per_cpu; + ProcStatValuesCPU proc_stat_values_all_cpus TSA_GUARDED_BY(data_mutex) {}; + ProcStatValuesOther proc_stat_values_other TSA_GUARDED_BY(data_mutex) {}; + std::vector proc_stat_values_per_cpu TSA_GUARDED_BY(data_mutex); /// https://www.kernel.org/doc/Documentation/block/stat.txt struct BlockDeviceStatValues @@ -181,7 +190,7 @@ private: BlockDeviceStatValues operator-(const BlockDeviceStatValues & other) const; }; - std::unordered_map block_device_stats; + std::unordered_map block_device_stats TSA_GUARDED_BY(data_mutex); struct NetworkInterfaceStatValues { @@ -197,9 +206,9 @@ private: NetworkInterfaceStatValues operator-(const NetworkInterfaceStatValues & other) const; }; - std::unordered_map network_interface_stats; + std::unordered_map network_interface_stats TSA_GUARDED_BY(data_mutex); - Stopwatch block_devices_rescan_delay; + Stopwatch block_devices_rescan_delay TSA_GUARDED_BY(data_mutex); void openSensors(); void openBlockDevices(); @@ -207,10 +216,7 @@ private: void openEDAC(); #endif - std::unique_ptr thread; - void run(); - void update(TimePoint update_time); }; } diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index e3e91e6bd07..8f6e1dec6c1 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -121,7 +121,7 @@ KeeperAsynchronousMetrics::~KeeperAsynchronousMetrics() stop(); } -void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint /*update_time*/, TimePoint /*current_time*/) +void KeeperAsynchronousMetrics::updateImpl(TimePoint /*update_time*/, TimePoint /*current_time*/, bool /*force_update*/, bool /*first_run*/, AsynchronousMetricValues & new_values) { #if USE_NURAFT { diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index 457a7112507..33e8d6818d7 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -19,7 +19,7 @@ public: private: ContextPtr context; - void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override; }; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 0ca88fafba7..45619ab38a1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -208,6 +208,11 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server return sm_commit_exec_in_progress_; } + void setServingRequest(bool value) + { + serving_req_ = value; + } + using nuraft::raft_server::raft_server; // peers are initially marked as responding because at least one cycle @@ -687,6 +692,14 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ if (req.get_type() != nuraft::msg_type::append_entries_request) break; + if (req.log_entries().empty()) + break; + + /// committing/preprocessing of local logs can take some time + /// and we don't want election to start during that time so we + /// set serving requests to avoid elections on timeout + raft_instance->setServingRequest(true); + SCOPE_EXIT(raft_instance->setServingRequest(false)); /// maybe we got snapshot installed if (state_machine->last_commit_index() >= last_log_idx_on_disk && !raft_instance->isCommitInProgress()) preprocess_logs(); diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 9c8c46d5ce6..ef37aa5fb47 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -97,7 +97,8 @@ static std::map sett {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}}}, + {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}}}, {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 68fd9012857..fe00fdd64d6 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -4,9 +4,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -374,7 +374,7 @@ std::unique_ptr DiskEncrypted::readFile( { /// File is empty, that's a normal case, see DiskEncrypted::truncateFile(). /// There is no header so we just return `ReadBufferFromString("")`. - return std::make_unique(std::make_unique(std::string_view{}), wrapped_path); + return std::make_unique(wrapped_path); } auto encryption_settings = current_settings.get(); FileEncryption::Header header = readHeader(*buffer); diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index daeab7aae6c..3da2e6f925a 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index c035349bb18..ac87305270f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -560,8 +560,9 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) ProfileEvents::FileSegmentPredownloadMicroseconds, predownload_watch.elapsedMicroseconds()); }); - OpenTelemetry::SpanHolder span{ - fmt::format("CachedOnDiskReadBufferFromFile::predownload(key={}, size={})", file_segment.key().toString(), bytes_to_predownload)}; + OpenTelemetry::SpanHolder span("CachedOnDiskReadBufferFromFile::predownload"); + span.addAttribute("clickhouse.key", file_segment.key().toString()); + span.addAttribute("clickhouse.size", bytes_to_predownload); if (bytes_to_predownload) { diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 676337e0373..3bc86f86250 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -1,6 +1,6 @@ #include "ReadBufferFromRemoteFSGather.h" -#include +#include #include #include @@ -61,7 +61,7 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( current_object = blobs_to_read.front(); } -SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) +std::unique_ptr ReadBufferFromRemoteFSGather::createImplementationBuffer(const StoredObject & object) { if (current_buf && !with_cache) { @@ -78,7 +78,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c if (with_cache) { auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); - return std::make_shared( + return std::make_unique( object_path, cache_key, settings.remote_fs_cache, diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 2764ed2d021..c5886dea603 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -53,7 +53,7 @@ public: bool isContentCached(size_t offset, size_t size) override; private: - SeekableReadBufferPtr createImplementationBuffer(const StoredObject & object); + std::unique_ptr createImplementationBuffer(const StoredObject & object); bool nextImpl() override; @@ -80,7 +80,7 @@ private: StoredObject current_object; size_t current_buf_idx = 0; - SeekableReadBufferPtr current_buf; + std::unique_ptr current_buf; Poco::Logger * log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index a9d451496ff..d4cb6b83223 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -39,7 +39,7 @@ std::unique_ptr createReadBufferFromFileBase( size_t alignment) { if (file_size.has_value() && !*file_size) - return std::make_unique(); + return std::make_unique(filename); size_t estimated_size = 0; if (read_hint.has_value()) diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index b6445d5763a..3c39fa2a8ff 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -531,7 +531,7 @@ std::unique_ptr DiskObjectStorage::readFile( const bool file_can_be_empty = !file_size.has_value() || *file_size == 0; if (storage_objects.empty() && file_can_be_empty) - return std::make_unique(); + return std::make_unique(path); return object_storage->readObjects( storage_objects, diff --git a/src/IO/Archives/LibArchiveReader.cpp b/src/IO/Archives/LibArchiveReader.cpp index 94e68045575..763cd3b171b 100644 --- a/src/IO/Archives/LibArchiveReader.cpp +++ b/src/IO/Archives/LibArchiveReader.cpp @@ -228,7 +228,12 @@ public: off_t getPosition() override { - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition not supported when reading from archive"); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getPosition is not supported when reading from archive"); + } + + size_t getFileOffsetOfBufferEnd() const override + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from archive"); } String getFileName() const override { return handle.getFileName(); } diff --git a/src/IO/Archives/ZipArchiveReader.cpp b/src/IO/Archives/ZipArchiveReader.cpp index 8c9c37e4ae0..636042ec586 100644 --- a/src/IO/Archives/ZipArchiveReader.cpp +++ b/src/IO/Archives/ZipArchiveReader.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int CANNOT_UNPACK_ARCHIVE; extern const int LOGICAL_ERROR; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int UNSUPPORTED_METHOD; extern const int CANNOT_SEEK_THROUGH_FILE; } @@ -252,6 +253,11 @@ public: checkResult(err); } + size_t getFileOffsetOfBufferEnd() const override + { + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "getFileOffsetOfBufferEnd is not supported when reading from zip archive"); + } + off_t seek(off_t off, int whence) override { off_t current_pos = getPosition(); diff --git a/src/IO/BoundedReadBuffer.cpp b/src/IO/BoundedReadBuffer.cpp index bda79d82ad3..f3b176a963c 100644 --- a/src/IO/BoundedReadBuffer.cpp +++ b/src/IO/BoundedReadBuffer.cpp @@ -4,8 +4,7 @@ namespace DB { -BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr impl_) - : ReadBufferFromFileDecorator(std::move(impl_)) +BoundedReadBuffer::BoundedReadBuffer(std::unique_ptr impl_) : impl(std::move(impl_)) { } diff --git a/src/IO/BoundedReadBuffer.h b/src/IO/BoundedReadBuffer.h index eb65857e83a..22a6471a9a1 100644 --- a/src/IO/BoundedReadBuffer.h +++ b/src/IO/BoundedReadBuffer.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB @@ -7,10 +7,10 @@ namespace DB /// A buffer which allows to make an underlying buffer as right bounded, /// e.g. the buffer cannot return data beyond offset specified in `setReadUntilPosition`. -class BoundedReadBuffer : public ReadBufferFromFileDecorator +class BoundedReadBuffer : public ReadBufferFromFileBase { public: - explicit BoundedReadBuffer(std::unique_ptr impl_); + explicit BoundedReadBuffer(std::unique_ptr impl_); bool supportsRightBoundedReads() const override { return true; } @@ -23,6 +23,8 @@ public: off_t seek(off_t off, int whence) override; size_t getFileOffsetOfBufferEnd() const override { return file_offset_of_buffer_end; } + String getFileName() const override { return impl->getFileName(); } + size_t getFileSize() override { return impl->getFileSize(); } /// file_offset_of_buffer_end can differ from impl's file_offset_of_buffer_end /// because of resizing of the tail. => Need to also override getPosition() as @@ -30,6 +32,8 @@ public: off_t getPosition() override; private: + std::unique_ptr impl; + std::optional read_until_position; /// atomic because can be used in log or exception messages while being updated. std::atomic file_offset_of_buffer_end = 0; diff --git a/src/IO/LimitSeekableReadBuffer.h b/src/IO/LimitSeekableReadBuffer.h index 61b307c522c..5624388dd7e 100644 --- a/src/IO/LimitSeekableReadBuffer.h +++ b/src/IO/LimitSeekableReadBuffer.h @@ -18,7 +18,6 @@ public: /// Returns adjusted position, i.e. returns `3` if the position in the nested buffer is `start_offset + 3`. off_t getPosition() override; - off_t seek(off_t off, int whence) override; private: diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 9b1c132cc01..56a094bb1a3 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -92,6 +92,11 @@ size_t MMapReadBufferFromFileDescriptor::getFileSize() return getSizeFromFileDescriptor(getFD(), getFileName()); } +size_t MMapReadBufferFromFileDescriptor::getFileOffsetOfBufferEnd() const +{ + return mapped.getOffset() + mapped.getLength(); +} + size_t MMapReadBufferFromFileDescriptor::readBigAt(char * to, size_t n, size_t offset, const std::function &) { if (offset >= mapped.getLength()) diff --git a/src/IO/MMapReadBufferFromFileDescriptor.h b/src/IO/MMapReadBufferFromFileDescriptor.h index 2a039e04971..97d8bbe224d 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/src/IO/MMapReadBufferFromFileDescriptor.h @@ -36,6 +36,8 @@ public: std::string getFileName() const override; + size_t getFileOffsetOfBufferEnd() const override; + int getFD() const; size_t getFileSize() override; diff --git a/src/IO/MMapReadBufferFromFileWithCache.cpp b/src/IO/MMapReadBufferFromFileWithCache.cpp index d53f3bc325d..f3c4d6f4e01 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.cpp +++ b/src/IO/MMapReadBufferFromFileWithCache.cpp @@ -76,4 +76,9 @@ off_t MMapReadBufferFromFileWithCache::seek(off_t offset, int whence) return new_pos; } +size_t MMapReadBufferFromFileWithCache::getFileOffsetOfBufferEnd() const +{ + return mapped->getOffset() + mapped->getLength(); +} + } diff --git a/src/IO/MMapReadBufferFromFileWithCache.h b/src/IO/MMapReadBufferFromFileWithCache.h index cb87b03df8d..ce5da29831e 100644 --- a/src/IO/MMapReadBufferFromFileWithCache.h +++ b/src/IO/MMapReadBufferFromFileWithCache.h @@ -19,7 +19,7 @@ public: off_t getPosition() override; std::string getFileName() const override; off_t seek(off_t offset, int whence) override; - + size_t getFileOffsetOfBufferEnd() const override; bool isRegularLocalFile(size_t * /* out_view_offset */) override { return true; } private: diff --git a/src/IO/ReadBufferFromEmptyFile.h b/src/IO/ReadBufferFromEmptyFile.h index f21f2f507dc..e2765765c47 100644 --- a/src/IO/ReadBufferFromEmptyFile.h +++ b/src/IO/ReadBufferFromEmptyFile.h @@ -14,12 +14,18 @@ namespace DB /// - ThreadPoolReader class ReadBufferFromEmptyFile : public ReadBufferFromFileBase { +public: + explicit ReadBufferFromEmptyFile(const String & file_name_) : file_name(file_name_) {} + private: + String file_name; + bool nextImpl() override { return false; } - std::string getFileName() const override { return ""; } + std::string getFileName() const override { return file_name; } off_t seek(off_t /*off*/, int /*whence*/) override { return 0; } off_t getPosition() override { return 0; } size_t getFileSize() override { return 0; } + size_t getFileOffsetOfBufferEnd() const override { return 0; } }; } diff --git a/src/IO/ReadBufferFromEncryptedFile.cpp b/src/IO/ReadBufferFromEncryptedFile.cpp index f9cf1597153..6861ae06dd8 100644 --- a/src/IO/ReadBufferFromEncryptedFile.cpp +++ b/src/IO/ReadBufferFromEncryptedFile.cpp @@ -101,6 +101,18 @@ bool ReadBufferFromEncryptedFile::nextImpl() return true; } +size_t ReadBufferFromEncryptedFile::getFileSize() +{ + size_t size = in->getFileSize(); + return size > FileEncryption::Header::kSize ? size - FileEncryption::Header::kSize : size; +} + +size_t ReadBufferFromEncryptedFile::getFileOffsetOfBufferEnd() const +{ + size_t file_offset = in->getFileOffsetOfBufferEnd(); + return file_offset > FileEncryption::Header::kSize ? file_offset - FileEncryption::Header::kSize : file_offset; +} + } #endif diff --git a/src/IO/ReadBufferFromEncryptedFile.h b/src/IO/ReadBufferFromEncryptedFile.h index 3626daccb3e..2f5093153ea 100644 --- a/src/IO/ReadBufferFromEncryptedFile.h +++ b/src/IO/ReadBufferFromEncryptedFile.h @@ -27,10 +27,10 @@ public: std::string getFileName() const override { return in->getFileName(); } void setReadUntilPosition(size_t position) override { in->setReadUntilPosition(position + FileEncryption::Header::kSize); } - void setReadUntilEnd() override { in->setReadUntilEnd(); } - size_t getFileSize() override { return in->getFileSize(); } + size_t getFileSize() override; + size_t getFileOffsetOfBufferEnd() const override; private: bool nextImpl() override; diff --git a/src/IO/ReadBufferFromFileBase.h b/src/IO/ReadBufferFromFileBase.h index 296edf9c689..b9288ce6636 100644 --- a/src/IO/ReadBufferFromFileBase.h +++ b/src/IO/ReadBufferFromFileBase.h @@ -60,6 +60,12 @@ public: /// file offset and what getPosition() returns. virtual bool isRegularLocalFile(size_t * /* out_view_offset */ = nullptr) { return false; } + /// NOTE: This method should be thread-safe against seek(), since it can be + /// used in CachedOnDiskReadBufferFromFile from multiple threads (because + /// it first releases the buffer, and then do logging, and so other thread + /// can already call seek() which will lead to data-race). + virtual size_t getFileOffsetOfBufferEnd() const = 0; + protected: std::optional file_size; ProfileCallback profile_callback; diff --git a/src/IO/ReadBufferFromFileDecorator.cpp b/src/IO/ReadBufferFromFileDecorator.cpp deleted file mode 100644 index 9ac0fb4e475..00000000000 --- a/src/IO/ReadBufferFromFileDecorator.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include - - -namespace DB -{ - -ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr impl_) - : ReadBufferFromFileDecorator(std::move(impl_), "") -{ -} - - -ReadBufferFromFileDecorator::ReadBufferFromFileDecorator(std::unique_ptr impl_, const String & file_name_) - : impl(std::move(impl_)), file_name(file_name_) -{ - swap(*impl); -} - - -std::string ReadBufferFromFileDecorator::getFileName() const -{ - if (!file_name.empty()) - return file_name; - - return getFileNameFromReadBuffer(*impl); -} - - -off_t ReadBufferFromFileDecorator::getPosition() -{ - swap(*impl); - auto position = impl->getPosition(); - swap(*impl); - return position; -} - - -off_t ReadBufferFromFileDecorator::seek(off_t off, int whence) -{ - swap(*impl); - auto result = impl->seek(off, whence); - swap(*impl); - return result; -} - - -bool ReadBufferFromFileDecorator::nextImpl() -{ - swap(*impl); - auto result = impl->next(); - swap(*impl); - return result; -} - -size_t ReadBufferFromFileDecorator::getFileSize() -{ - return getFileSizeFromReadBuffer(*impl); -} - -} diff --git a/src/IO/ReadBufferFromFileDecorator.h b/src/IO/ReadBufferFromFileDecorator.h deleted file mode 100644 index 6e62c7f741b..00000000000 --- a/src/IO/ReadBufferFromFileDecorator.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/// Delegates all reads to underlying buffer. Doesn't have own memory. -class ReadBufferFromFileDecorator : public ReadBufferFromFileBase -{ -public: - explicit ReadBufferFromFileDecorator(std::unique_ptr impl_); - ReadBufferFromFileDecorator(std::unique_ptr impl_, const String & file_name_); - - std::string getFileName() const override; - - off_t getPosition() override; - - off_t seek(off_t off, int whence) override; - - bool nextImpl() override; - - bool isWithFileSize() const { return dynamic_cast(impl.get()) != nullptr; } - - const ReadBuffer & getWrappedReadBuffer() const { return *impl; } - - ReadBuffer & getWrappedReadBuffer() { return *impl; } - - size_t getFileSize() override; - -protected: - std::unique_ptr impl; - String file_name; -}; - -} diff --git a/src/IO/ReadBufferFromMemory.h b/src/IO/ReadBufferFromMemory.h index ad96e4bfa28..6d3f1a2c6e5 100644 --- a/src/IO/ReadBufferFromMemory.h +++ b/src/IO/ReadBufferFromMemory.h @@ -20,7 +20,6 @@ public: : SeekableReadBuffer(const_cast(str.data()), str.size(), 0) {} off_t seek(off_t off, int whence) override; - off_t getPosition() override; }; diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index c002d30e633..1fb66a5aa9f 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -44,12 +44,6 @@ public: virtual String getInfoForLog() { return ""; } - /// NOTE: This method should be thread-safe against seek(), since it can be - /// used in CachedOnDiskReadBufferFromFile from multiple threads (because - /// it first releases the buffer, and then do logging, and so other thread - /// can already call seek() which will lead to data-race). - virtual size_t getFileOffsetOfBufferEnd() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFileOffsetOfBufferEnd() not implemented"); } - /// If true, setReadUntilPosition() guarantees that eof will be reported at the given position. virtual bool supportsRightBoundedReads() const { return false; } diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index 3660d962c08..435789652dc 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include namespace DB @@ -17,23 +16,15 @@ template static size_t getFileSize(T & in) { if (auto * with_file_size = dynamic_cast(&in)) - { return with_file_size->getFileSize(); - } throw Exception(ErrorCodes::UNKNOWN_FILE_SIZE, "Cannot find out file size"); } size_t getFileSizeFromReadBuffer(ReadBuffer & in) { - if (auto * delegate = dynamic_cast(&in)) - { - return getFileSize(delegate->getWrappedReadBuffer()); - } - else if (auto * compressed = dynamic_cast(&in)) - { + if (auto * compressed = dynamic_cast(&in)) return getFileSize(compressed->getWrappedReadBuffer()); - } return getFileSize(in); } @@ -52,11 +43,7 @@ std::optional tryGetFileSizeFromReadBuffer(ReadBuffer & in) bool isBufferWithFileSize(const ReadBuffer & in) { - if (const auto * delegate = dynamic_cast(&in)) - { - return delegate->isWithFileSize(); - } - else if (const auto * compressed = dynamic_cast(&in)) + if (const auto * compressed = dynamic_cast(&in)) { return isBufferWithFileSize(compressed->getWrappedReadBuffer()); } @@ -66,11 +53,7 @@ bool isBufferWithFileSize(const ReadBuffer & in) size_t getDataOffsetMaybeCompressed(const ReadBuffer & in) { - if (const auto * delegate = dynamic_cast(&in)) - { - return getDataOffsetMaybeCompressed(delegate->getWrappedReadBuffer()); - } - else if (const auto * compressed = dynamic_cast(&in)) + if (const auto * compressed = dynamic_cast(&in)) { return getDataOffsetMaybeCompressed(compressed->getWrappedReadBuffer()); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index f58380ed0b7..5bb01050591 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -178,6 +178,11 @@ void WriteBufferFromS3::preFinalize() void WriteBufferFromS3::finalizeImpl() { + OpenTelemetry::SpanHolder span("WriteBufferFromS3::finalizeImpl"); + span.addAttribute("clickhouse.s3_bucket", bucket); + span.addAttribute("clickhouse.s3_key", key); + span.addAttribute("clickhouse.total_size", total_size); + LOG_TRACE(limitedLog, "finalizeImpl WriteBufferFromS3. {}.", getShortLogDetails()); if (!is_prefinalized) @@ -188,6 +193,8 @@ void WriteBufferFromS3::finalizeImpl() task_tracker->waitAll(); + span.addAttributeIfNotZero("clickhouse.multipart_upload_parts", multipart_tags.size()); + if (!multipart_upload_id.empty()) { completeMultipartUpload(); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index e439620984e..8e5d1fd757f 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -424,7 +424,9 @@ void FileSegment::write(const char * from, size_t size, size_t offset) FileSegment::State FileSegment::wait(size_t offset) { - OpenTelemetry::SpanHolder span{fmt::format("FileSegment::wait({})", key().toString())}; + OpenTelemetry::SpanHolder span("FileSegment::wait"); + span.addAttribute("clickhouse.key", key().toString()); + span.addAttribute("clickhouse.offset", offset); auto lock = lockFileSegment(); diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index b3a48e3e611..5865e669e47 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -322,7 +322,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - const StorageID & main_table, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, @@ -414,7 +413,6 @@ void executeQueryWithParallelReplicas( std::move(coordinator), stream_factory.header, stream_factory.processed_stage, - main_table, new_context, getThrottler(new_context), std::move(scalars), diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index b5ee4a11df6..a19ece0bbdc 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -70,7 +70,6 @@ void executeQuery( void executeQueryWithParallelReplicas( QueryPlan & query_plan, - const StorageID & main_table, SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7e89c794712..38a033bee1d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -290,6 +290,7 @@ struct ContextSharedPart : boost::noncopyable mutable QueryCachePtr query_cache TSA_GUARDED_BY(mutex); /// Cache of query results. mutable MarkCachePtr index_mark_cache TSA_GUARDED_BY(mutex); /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache TSA_GUARDED_BY(mutex); /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + AsynchronousMetrics * asynchronous_metrics TSA_GUARDED_BY(mutex) = nullptr; /// Points to asynchronous metrics ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; @@ -1644,6 +1645,11 @@ void Context::addQueryAccessInfo(const QualifiedProjectionName & qualified_proje "{}.{}", qualified_projection_name.storage_id.getFullTableName(), backQuoteIfNeed(qualified_projection_name.projection_name))); } +Context::QueryFactoriesInfo Context::getQueryFactoriesInfo() const +{ + return query_factories_info; +} + void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const { if (isGlobalContext()) @@ -2860,6 +2866,18 @@ void Context::clearCaches() const /// Intentionally not clearing the query cache which is transactionally inconsistent by design. } +void Context::setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_) +{ + std::lock_guard lock(shared->mutex); + shared->asynchronous_metrics = asynchronous_metrics_; +} + +AsynchronousMetrics * Context::getAsynchronousMetrics() const +{ + SharedLockGuard lock(shared->mutex); + return shared->asynchronous_metrics; +} + ThreadPool & Context::getPrefetchThreadpool() const { callOnce(shared->prefetch_threadpool_initialized, [&] { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 65566876a80..8d40ccb301b 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -70,6 +70,7 @@ class IUserDefinedSQLObjectsStorage; class InterserverCredentials; using InterserverCredentialsPtr = std::shared_ptr; class InterserverIOHandler; +class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; @@ -373,25 +374,6 @@ protected: QueryFactoriesInfo(QueryFactoriesInfo && rhs) = delete; - QueryFactoriesInfo & operator=(QueryFactoriesInfo rhs) - { - swap(rhs); - return *this; - } - - void swap(QueryFactoriesInfo & rhs) - { - std::swap(aggregate_functions, rhs.aggregate_functions); - std::swap(aggregate_function_combinators, rhs.aggregate_function_combinators); - std::swap(database_engines, rhs.database_engines); - std::swap(data_type_families, rhs.data_type_families); - std::swap(dictionaries, rhs.dictionaries); - std::swap(formats, rhs.formats); - std::swap(functions, rhs.functions); - std::swap(storages, rhs.storages); - std::swap(table_functions, rhs.table_functions); - } - std::unordered_set aggregate_functions; std::unordered_set aggregate_function_combinators; std::unordered_set database_engines; @@ -727,7 +709,7 @@ public: TableFunction }; - const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } + QueryFactoriesInfo getQueryFactoriesInfo() const; void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; /// For table functions s3/file/url/hdfs/input we can use structure from @@ -1014,6 +996,9 @@ public: /// ----------------------------------------------------------------------------------------------------- + void setAsynchronousMetrics(AsynchronousMetrics * asynchronous_metrics_); + AsynchronousMetrics * getAsynchronousMetrics() const; + ThreadPool & getPrefetchThreadpool() const; /// Note: prefetchThreadpool is different from threadpoolReader diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 36e864ace26..d002cc6d980 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include #include #include @@ -463,6 +465,14 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->ttl); } + if (!column.settings.empty()) + { + auto settings = std::make_shared(); + settings->is_standalone = false; + settings->changes = column.settings; + column_declaration->settings = std::move(settings); + } + columns_list->children.push_back(column_declaration_ptr); } @@ -670,6 +680,12 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( if (col_decl.ttl) column.ttl = col_decl.ttl; + if (col_decl.settings) + { + column.settings = col_decl.settings->as().changes; + MergeTreeColumnSettings::validate(column.settings); + } + res.add(std::move(column)); } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1712c9608bf..1411e7e017b 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -561,6 +561,14 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_RELOAD_USERS); system_context->getAccessControl().reload(AccessControl::ReloadMode::ALL); break; + case Type::RELOAD_ASYNCHRONOUS_METRICS: + { + getContext()->checkAccess(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS); + auto * asynchronous_metrics = system_context->getAsynchronousMetrics(); + if (asynchronous_metrics) + asynchronous_metrics->update(std::chrono::system_clock::now(), /*force_update*/ true); + break; + } case Type::STOP_MERGES: startStopAction(ActionLocks::PartsMerge, false); break; @@ -1225,6 +1233,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_RELOAD_USERS); break; } + case Type::RELOAD_ASYNCHRONOUS_METRICS: + { + required_access.emplace_back(AccessType::SYSTEM_RELOAD_ASYNCHRONOUS_METRICS); + break; + } case Type::STOP_MERGES: case Type::START_MERGES: { diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index 4e71bd56851..cc386825024 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -1,17 +1,13 @@ #pragma once #include -#include #include -#include #include #include -#include #include -#include +#include #include #include -#include #include #include #include @@ -37,8 +33,6 @@ public: ASTIdentifier * identifier = nullptr; DataTypePtr arg_data_type = {}; - Range range = Range::createWholeUniverse(); - void reject() { monotonicity.is_monotonic = false; } bool isRejected() const { return !monotonicity.is_monotonic; } @@ -103,30 +97,13 @@ public: if (data.isRejected()) return; - /// Monotonicity check only works for functions that contain at most two arguments and one of them must be a constant. - if (!ast_function.arguments) + /// TODO: monotonicity for functions of several arguments + if (!ast_function.arguments || ast_function.arguments->children.size() != 1) { data.reject(); return; } - auto arguments_size = ast_function.arguments->children.size(); - - if (arguments_size == 0 || arguments_size > 2) - { - data.reject(); - return; - } - else if (arguments_size == 2) - { - /// If the function has two arguments, then one of them must be a constant. - if (!ast_function.arguments->children[0]->as() && !ast_function.arguments->children[1]->as()) - { - data.reject(); - return; - } - } - if (!data.canOptimize(ast_function)) { data.reject(); @@ -147,33 +124,14 @@ public: return; } - auto function_arguments = getFunctionArguments(ast_function, data); - - auto function_base = function->build(function_arguments); + ColumnsWithTypeAndName args; + args.emplace_back(data.arg_data_type, "tmp"); + auto function_base = function->build(args); if (function_base && function_base->hasInformationAboutMonotonicity()) { bool is_positive = data.monotonicity.is_positive; - data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, data.range.left, data.range.right); - - auto & key_range = data.range; - - /// If we apply function to open interval, we can get empty intervals in result. - /// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001'). - /// To avoid this we make range left and right included. - /// Any function that treats NULL specially is not monotonic. - /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. - if (!key_range.left.isNull()) - { - key_range.left = applyFunction(function_base, data.arg_data_type, key_range.left); - key_range.left_included = true; - } - - if (!key_range.right.isNull()) - { - key_range.right = applyFunction(function_base, data.arg_data_type, key_range.right); - key_range.right_included = true; - } + data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field()); if (!is_positive) data.monotonicity.is_positive = !data.monotonicity.is_positive; @@ -185,53 +143,13 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { - /// Multi-argument functions with all but one constant arguments can be monotonic. + /// Currently we check monotonicity only for single-argument functions. + /// Although, multi-argument functions with all but one constant arguments can also be monotonic. if (const auto * func = typeid_cast(parent.get())) - return func->arguments->children.size() <= 2; + return func->arguments->children.size() < 2; return true; } - - static ColumnWithTypeAndName extractLiteralColumnAndTypeFromAstLiteral(const ASTLiteral * literal) - { - ColumnWithTypeAndName result; - - result.type = applyVisitor(FieldToDataType(), literal->value); - result.column = result.type->createColumnConst(0, literal->value); - - return result; - } - - static ColumnsWithTypeAndName getFunctionArguments(const ASTFunction & ast_function, const Data & data) - { - ColumnsWithTypeAndName args; - - auto arguments_size = ast_function.arguments->children.size(); - - chassert(arguments_size == 1 || arguments_size == 2); - - if (arguments_size == 2) - { - if (ast_function.arguments->children[0]->as()) - { - const auto * literal = ast_function.arguments->children[0]->as(); - args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); - args.emplace_back(data.arg_data_type, "tmp"); - } - else - { - const auto * literal = ast_function.arguments->children[1]->as(); - args.emplace_back(data.arg_data_type, "tmp"); - args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); - } - } - else - { - args.emplace_back(data.arg_data_type, "tmp"); - } - - return args; - } }; using MonotonicityCheckVisitor = ConstInDepthNodeVisitor; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 31d4a4e51a4..bdf314f35b9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -69,7 +69,7 @@ ServerAsynchronousMetrics::~ServerAsynchronousMetrics() stop(); } -void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) +void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) { if (auto mark_cache = getContext()->getMarkCache()) { @@ -377,7 +377,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values } #endif - updateHeavyMetricsIfNeeded(current_time, update_time, new_values); + updateHeavyMetricsIfNeeded(current_time, update_time, force_update, first_run, new_values); } void ServerAsynchronousMetrics::logImpl(AsynchronousMetricValues & new_values) @@ -421,19 +421,19 @@ void ServerAsynchronousMetrics::updateDetachedPartsStats() detached_parts_stats = current_values; } -void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values) +void ServerAsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) { - const auto time_after_previous_update = current_time - heavy_metric_previous_update_time; - const bool update_heavy_metric = time_after_previous_update >= heavy_metric_update_period || first_run; + const auto time_since_previous_update = current_time - heavy_metric_previous_update_time; + const bool update_heavy_metrics = (time_since_previous_update >= heavy_metric_update_period) || force_update || first_run; Stopwatch watch; - if (update_heavy_metric) + if (update_heavy_metrics) { heavy_metric_previous_update_time = update_time; if (first_run) heavy_update_interval = heavy_metric_update_period.count(); else - heavy_update_interval = std::chrono::duration_cast(time_after_previous_update).count() / 1e6; + heavy_update_interval = std::chrono::duration_cast(time_since_previous_update).count() / 1e6; /// Test shows that listing 100000 entries consuming around 0.15 sec. updateDetachedPartsStats(); diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index a579d12de2c..b0cf8efbfd7 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -18,7 +18,7 @@ public: ~ServerAsynchronousMetrics() override; private: - void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override; + void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) override; void logImpl(AsynchronousMetricValues & new_values) override; const Duration heavy_metric_update_period; @@ -34,7 +34,7 @@ private: DetachedPartsStats detached_parts_stats{}; void updateDetachedPartsStats(); - void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, AsynchronousMetricValues & new_values); + void updateHeavyMetricsIfNeeded(TimePoint current_time, TimePoint update_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values); }; } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index de3b72f4801..2aa52c33048 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -752,7 +752,8 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, } /// Rewrite sum(column +/- literal) function with sum(column) +/- literal * count(column). - rewriteSumFunctionWithSumAndCount(query, tables_with_columns); + if (settings.optimize_arithmetic_operations_in_aggregate_functions) + rewriteSumFunctionWithSumAndCount(query, tables_with_columns); /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc. optimizeDateFilters(select_query, tables_with_columns, context); diff --git a/src/Interpreters/applyFunction.cpp b/src/Interpreters/applyFunction.cpp deleted file mode 100644 index a53f14f0381..00000000000 --- a/src/Interpreters/applyFunction.cpp +++ /dev/null @@ -1,43 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -static Field applyFunctionForField(const FunctionBasePtr & func, const DataTypePtr & arg_type, const Field & arg_value) -{ - ColumnsWithTypeAndName columns{ - {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, - }; - - auto col = func->execute(columns, func->getResultType(), 1); - return (*col)[0]; -} - -FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) -{ - /// Fallback for fields without block reference. - if (field.isExplicit()) - return applyFunctionForField(func, current_type, field); - - String result_name = "_" + func->getName() + "_" + toString(field.column_idx); - const auto & columns = field.columns; - size_t result_idx = columns->size(); - - for (size_t i = 0; i < result_idx; ++i) - if ((*columns)[i].name == result_name) - result_idx = i; - - if (result_idx == columns->size()) - { - ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; - field.columns->emplace_back(ColumnWithTypeAndName{nullptr, func->getResultType(), result_name}); - (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); - } - - return {field.columns, field.row_idx, result_idx}; -} - -} diff --git a/src/Interpreters/applyFunction.h b/src/Interpreters/applyFunction.h deleted file mode 100644 index 9b8ae43a53c..00000000000 --- a/src/Interpreters/applyFunction.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -struct FieldRef; - -class IFunctionBase; -class IDataType; - -using DataTypePtr = std::shared_ptr; -using FunctionBasePtr = std::shared_ptr; - -FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field); -} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 4b5a6a84e17..533d58aaa8f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -257,7 +257,7 @@ addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo element.query_projections.insert(access_info.projections.begin(), access_info.projections.end()); element.query_views.insert(access_info.views.begin(), access_info.views.end()); - const auto & factories_info = context_ptr->getQueryFactoriesInfo(); + const auto factories_info = context_ptr->getQueryFactoriesInfo(); element.used_aggregate_functions = factories_info.aggregate_functions; element.used_aggregate_function_combinators = factories_info.aggregate_function_combinators; element.used_database_engines = factories_info.database_engines; diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index b0d812eec6b..6c29e0bf9d5 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -57,70 +57,83 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->collation); } + if (settings) + { + res->settings = settings->clone(); + res->children.push_back(res->settings); + } + return res; } -void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const { frame.need_parens = false; /// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query. - settings.ostr << backQuote(name); + format_settings.ostr << backQuote(name); if (type) { - settings.ostr << ' '; + format_settings.ostr << ' '; FormatStateStacked type_frame = frame; type_frame.indent = 0; - type->formatImpl(settings, state, type_frame); + type->formatImpl(format_settings, state, type_frame); } if (null_modifier) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") - << (*null_modifier ? "" : "NOT ") << "NULL" << (settings.hilite ? hilite_none : ""); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") + << (*null_modifier ? "" : "NOT ") << "NULL" << (format_settings.hilite ? hilite_none : ""); } if (default_expression) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : ""); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : ""); if (!ephemeral_default) { - settings.ostr << ' '; - default_expression->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + default_expression->formatImpl(format_settings, state, frame); } } if (comment) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; - comment->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' '; + comment->formatImpl(format_settings, state, frame); } if (codec) { - settings.ostr << ' '; - codec->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + codec->formatImpl(format_settings, state, frame); } if (stat_type) { - settings.ostr << ' '; - stat_type->formatImpl(settings, state, frame); + format_settings.ostr << ' '; + stat_type->formatImpl(format_settings, state, frame); } if (ttl) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' '; - ttl->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' '; + ttl->formatImpl(format_settings, state, frame); } if (collation) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COLLATE" << (settings.hilite ? hilite_none : "") << ' '; - collation->formatImpl(settings, state, frame); + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' '; + collation->formatImpl(format_settings, state, frame); + } + + if (settings) + { + format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '('; + settings->formatImpl(format_settings, state, frame); + format_settings.ostr << ')'; } } diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index a54abae97ea..d775928d05c 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -22,12 +22,13 @@ public: ASTPtr stat_type; ASTPtr ttl; ASTPtr collation; + ASTPtr settings; bool primary_key_specifier = false; String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); } ASTPtr clone() const override; - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override; }; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 027bb1ec56f..a74275f1638 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -67,6 +67,7 @@ public: RELOAD_EMBEDDED_DICTIONARIES, RELOAD_CONFIG, RELOAD_USERS, + RELOAD_ASYNCHRONOUS_METRICS, RESTART_DISK, STOP_MERGES, START_MERGES, diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 8afab927d18..d72fb493368 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -111,6 +111,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_comment("COMMENT"); ParserKeyword s_codec("CODEC"); ParserKeyword s_ttl("TTL"); + ParserKeyword s_settings("SETTINGS"); ParserKeyword s_remove_ttl("REMOVE TTL"); ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); @@ -725,9 +726,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->remove_property = "CODEC"; else if (s_ttl.ignore(pos, expected)) command->remove_property = "TTL"; + else if (s_settings.ignore(pos, expected)) + command->remove_property = "SETTINGS"; else return false; } + else if (s_modify_setting.ignore(pos, expected)) + { + if (!parser_settings.parse(pos, command_settings_changes, expected)) + return false; + } + else if (s_reset_setting.ignore(pos, expected)) + { + if (!parser_reset_setting.parse(pos, command_settings_resets, expected)) + return false; + } else { if (s_first.ignore(pos, expected)) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 910ee048442..c9059324bbe 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -10,6 +10,7 @@ #include #include #include +#include #include namespace DB @@ -120,8 +121,6 @@ using ParserCompoundColumnDeclaration = IParserColumnDeclaration bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - NameParser name_parser; - ParserDataType type_parser; ParserKeyword s_default{"DEFAULT"}; ParserKeyword s_null{"NULL"}; ParserKeyword s_not{"NOT"}; @@ -134,9 +133,15 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_stat{"STATISTIC"}; ParserKeyword s_ttl{"TTL"}; ParserKeyword s_remove{"REMOVE"}; + ParserKeyword s_modify_setting("MODIFY SETTING"); + ParserKeyword s_reset_setting("RESET SETTING"); + ParserKeyword s_settings("SETTINGS"); ParserKeyword s_type{"TYPE"}; ParserKeyword s_collate{"COLLATE"}; ParserKeyword s_primary_key{"PRIMARY KEY"}; + + NameParser name_parser; + ParserDataType type_parser; ParserExpression expr_parser; ParserStringLiteral string_literal_parser; ParserLiteral literal_parser; @@ -144,6 +149,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserCollation collation_parser; ParserStatisticType stat_type_parser; ParserExpression expression_parser; + ParserSetQuery settings_parser(true); /// mandatory column name ASTPtr name; @@ -155,11 +161,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E /// This keyword may occur only in MODIFY COLUMN query. We check it here /// because ParserDataType parses types as an arbitrary identifiers and - /// doesn't check that parsed string is existing data type. In this way - /// REMOVE keyword can be parsed as data type and further parsing will fail. - /// So we just check this keyword and in case of success return column - /// declaration with name only. - if (!require_type && s_remove.checkWithoutMoving(pos, expected)) + /// doesn't check that parsed string is existing data type. In this way, + /// REMOVE, MODIFY SETTING, or RESET SETTING can be parsed as data type + /// and further parsing will fail. So we just check these keyword and in + /// case of success return column declaration with name only. + if (!require_type + && (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected) || s_reset_setting.checkWithoutMoving(pos, expected))) { if (!check_keywords_after_name) return false; @@ -181,6 +188,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr stat_type_expression; ASTPtr ttl_expression; ASTPtr collation_expression; + ASTPtr settings; bool primary_key_specifier = false; auto null_check_without_moving = [&]() -> bool @@ -321,6 +329,28 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E primary_key_specifier = true; } + auto old_pos = pos; + if (s_settings.ignore(pos, expected)) + { + /// When the keyword `SETTINGS` appear here, it can be a column settings declaration or query settings + /// For example: + /// - Column settings: `ALTER TABLE xx MODIFY COLUMN yy SETTINGS (name = value)` + /// - Query settings: ` ALTER TABLE xx MODIFY COLUMN yy SETTINGS mutation_sync = 2` + /// So after parsing keyword `SETTINGS`, we check if it's followed by an `(` then it's the column + /// settings, otherwise it's the query settings and we need to move `pos` back to origin position. + ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket); + if (parser_opening_bracket.ignore(pos, expected)) + { + if (!settings_parser.parse(pos, settings, expected)) + return false; + ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket); + if (!parser_closing_bracket.ignore(pos, expected)) + return false; + } + else + pos = old_pos; + } + node = column_declaration; if (type) @@ -351,6 +381,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(codec_expression)); } + if (settings) + { + column_declaration->settings = settings; + column_declaration->children.push_back(std::move(settings)); + } + if (stat_type_expression) { column_declaration->stat_type = stat_type_expression; @@ -362,6 +398,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->ttl = ttl_expression; column_declaration->children.push_back(std::move(ttl_expression)); } + if (collation_expression) { column_declaration->collation = collation_expression; diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 4a1903393f6..9721aa1f128 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -3,11 +3,6 @@ namespace DB { - String queryToStringNullable(const ASTPtr & query) - { - return query ? queryToString(query) : ""; - } - String queryToString(const ASTPtr & query) { return queryToString(*query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 3acd560b1e2..873de218293 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -6,5 +6,4 @@ namespace DB { String queryToString(const ASTPtr & query); String queryToString(const IAST & query); - String queryToStringNullable(const ASTPtr & query); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 5ee7bd3e0ba..f2def571325 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1391,7 +1391,7 @@ void Planner::buildPlanForQueryNode() } } - if (query_context->canUseTaskBasedParallelReplicas() || !settings.parallel_replicas_custom_key.value.empty()) + if (!settings.parallel_replicas_custom_key.value.empty()) { /// Check support for JOIN for parallel replicas with custom key if (planner_context->getTableExpressionNodeToData().size() > 1) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 43dcca6d59b..4bbba4cfa30 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -357,7 +357,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, - StorageID main_table_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -369,7 +368,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , query_ast(query_ast_) , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) - , main_table(std::move(main_table_)) , context(context_) , throttler(throttler_) , scalars(scalars_) diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index f6fd7b3a630..82ef45d6bbf 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -76,7 +76,6 @@ public: ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, - StorageID main_table_, ContextMutablePtr context_, ThrottlerPtr throttler_, Scalars scalars_, @@ -99,7 +98,6 @@ private: ASTPtr query_ast; ParallelReplicasReadingCoordinatorPtr coordinator; QueryProcessingStage::Enum stage; - StorageID main_table; ContextMutablePtr context; ThrottlerPtr throttler; Scalars scalars; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ddd60fc81d2..766863ed9f9 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include @@ -74,6 +75,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property) return AlterCommand::RemoveProperty::CODEC; else if (property == "TTL") return AlterCommand::RemoveProperty::TTL; + else if (property == "SETTINGS") + return AlterCommand::RemoveProperty::SETTINGS; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property); } @@ -173,6 +176,25 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (ast_col_decl.codec) command.codec = ast_col_decl.codec; + if (ast_col_decl.settings) + command.settings_changes = ast_col_decl.settings->as().changes; + + /// At most only one of ast_col_decl.settings or command_ast->settings_changes is non-null + if (command_ast->settings_changes) + { + command.settings_changes = command_ast->settings_changes->as().changes; + command.append_column_setting = true; + } + + if (command_ast->settings_resets) + { + for (const ASTPtr & identifier_ast : command_ast->settings_resets->children) + { + const auto & identifier = identifier_ast->as(); + command.settings_resets.emplace(identifier.name()); + } + } + if (command_ast->column) command.after_column = getIdentifierName(command_ast->column); @@ -501,6 +523,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { column.ttl.reset(); } + else if (to_remove == RemoveProperty::SETTINGS) + { + column.settings.clear(); + } else { if (codec) @@ -515,6 +541,22 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) if (data_type) column.type = data_type; + if (!settings_changes.empty()) + { + MergeTreeColumnSettings::validate(settings_changes); + if (append_column_setting) + for (const auto & change : settings_changes) + column.settings.setSetting(change.name, change.value); + else + column.settings = settings_changes; + } + + if (!settings_resets.empty()) + { + for (const auto & setting : settings_resets) + column.settings.removeSetting(setting); + } + /// User specified default expression or changed /// datatype. We have to replace default. if (default_expression || data_type) @@ -1357,7 +1399,6 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have COMMENT, cannot remove it", backQuote(column_name)); - } modified_columns.emplace(column_name); diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index f40fdc954ec..d0d5d02b5f7 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -64,7 +64,8 @@ struct AlterCommand /// Other properties COMMENT, CODEC, - TTL + TTL, + SETTINGS }; Type type = UNKNOWN; @@ -137,10 +138,10 @@ struct AlterCommand /// For ADD and MODIFY ASTPtr codec = nullptr; - /// For MODIFY SETTING + /// For MODIFY SETTING or MODIFY COLUMN MODIFY SETTING SettingsChanges settings_changes; - /// For RESET SETTING + /// For RESET SETTING or MODIFY COLUMN RESET SETTING std::set settings_resets; /// For MODIFY_QUERY @@ -155,6 +156,9 @@ struct AlterCommand /// What to remove from column (or TTL) RemoveProperty to_remove = RemoveProperty::NO_PROPERTY; + /// Is this MODIFY COLUMN MODIFY SETTING or MODIFY COLUMN column with settings declaration) + bool append_column_setting = false; + static std::optional parse(const ASTAlterCommand * command); void apply(StorageInMemoryMetadata & metadata, ContextPtr context) const; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 1712b984596..d6a241da032 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Parsers/ASTSetQuery.h" #include #include #include @@ -72,6 +73,7 @@ bool ColumnDescription::operator==(const ColumnDescription & other) const && default_desc == other.default_desc && stat == other.stat && ast_to_str(codec) == ast_to_str(other.codec) + && settings == other.settings && ast_to_str(ttl) == ast_to_str(other.ttl); } @@ -104,6 +106,18 @@ void ColumnDescription::writeText(WriteBuffer & buf) const writeEscapedString(queryToString(codec), buf); } + if (!settings.empty()) + { + writeChar('\t', buf); + DB::writeText("SETTINGS ", buf); + DB::writeText("(", buf); + ASTSetQuery ast; + ast.is_standalone = false; + ast.changes = settings; + writeEscapedString(queryToString(ast), buf); + DB::writeText(")", buf); + } + if (stat) { writeChar('\t', buf); @@ -154,6 +168,9 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->ttl) ttl = col_ast->ttl; + + if (col_ast->settings) + settings = col_ast->settings->as().changes; } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 9a133f81d7a..59179aac17a 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -83,6 +84,7 @@ struct ColumnDescription ColumnDefault default_desc; String comment; ASTPtr codec; + SettingsChanges settings; ASTPtr ttl; std::optional stat; diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index b490c5cac63..553473fcc9e 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -37,7 +37,7 @@ namespace ErrorCodes AsynchronousReadBufferFromHDFS::AsynchronousReadBufferFromHDFS( IAsynchronousReader & reader_, const ReadSettings & settings_, std::shared_ptr impl_) - : BufferWithOwnMemory(settings_.remote_fs_buffer_size) + : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) , reader(reader_) , base_priority(settings_.priority) , impl(std::move(impl_)) diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index d89aa60ab71..9c01bd6e434 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -21,7 +21,7 @@ namespace DB class IAsynchronousReader; -class AsynchronousReadBufferFromHDFS : public BufferWithOwnMemory, public WithFileName, public WithFileSize +class AsynchronousReadBufferFromHDFS : public ReadBufferFromFileBase { public: AsynchronousReadBufferFromHDFS( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index f3057a8254f..87f23b0da2a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -81,7 +81,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); - hyperrectangle.clear(); hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { @@ -105,39 +104,6 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par initialized = true; } -Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const -{ - if (!initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); - - Block block; - - const auto metadata_snapshot = data.getInMemoryMetadataPtr(); - const auto & partition_key = metadata_snapshot->getPartitionKey(); - - const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); - const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); - const auto minmax_idx_size = minmax_column_types.size(); - - for (size_t i = 0; i < minmax_idx_size; ++i) - { - const auto & data_type = minmax_column_types[i]; - const auto & column_name = minmax_column_names[i]; - - const auto column = data_type->createColumn(); - - const auto min_val = hyperrectangle.at(i).left; - const auto max_val = hyperrectangle.at(i).right; - - column->insert(min_val); - column->insert(max_val); - - block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); - } - - return block; -} - IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store( const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & out_checksums) const { @@ -219,7 +185,8 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) if (!initialized) { - *this = other; + hyperrectangle = other.hyperrectangle; + initialized = true; } else { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 29f0f54d419..640a1f1d0a3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -336,7 +336,6 @@ public: } void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); - Block getBlock(const MergeTreeData & data) const; using WrittenFiles = std::vector>; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e5bcb11091f..d5922ae1bc2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1,37 +1,36 @@ -#include -#include +#include +#include +#include #include #include #include #include -#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include #include -#include +#include +#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -837,6 +836,21 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants return node.tryGetConstant(out_value, out_type); } + +static Field applyFunctionForField( + const FunctionBasePtr & func, + const DataTypePtr & arg_type, + const Field & arg_value) +{ + ColumnsWithTypeAndName columns + { + { arg_type->createColumnConst(1, arg_value), arg_type, "x" }, + }; + + auto col = func->execute(columns, func->getResultType(), 1); + return (*col)[0]; +} + /// The case when arguments may have types different than in the primary key. static std::pair applyFunctionForFieldOfUnknownType( const FunctionBasePtr & func, @@ -876,6 +890,33 @@ static std::pair applyBinaryFunctionForFieldOfUnknownType( return {std::move(result), std::move(return_type)}; } + +static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + String result_name = "_" + func->getName() + "_" + toString(field.column_idx); + const auto & columns = field.columns; + size_t result_idx = columns->size(); + + for (size_t i = 0; i < result_idx; ++i) + { + if ((*columns)[i].name == result_name) + result_idx = i; + } + + if (result_idx == columns->size()) + { + ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; + field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); + (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); + } + + return {field.columns, field.row_idx, result_idx}; +} + /** When table's key has expression with these functions from a column, * and when a column in a query is compared with a constant, such as: * CREATE TABLE (x String) ORDER BY toDate(x) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c3e348a549a..61332a4ff38 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,6 +8,21 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -28,20 +43,19 @@ #include #include #include +#include +#include #include #include #include #include #include #include -#include -#include #include -#include #include -#include -#include #include +#include +#include #include #include #include @@ -50,41 +64,26 @@ #include #include #include +#include #include #include #include #include #include #include -#include #include #include -#include #include #include #include #include #include -#include #include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include @@ -198,50 +197,6 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; } -static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast) -{ - if (partition_ast.fields_count.has_value()) - return *partition_ast.fields_count; - - if (partition_value_ast->as()) - return 1; - - const auto * tuple_ast = partition_value_ast->as(); - - if (!tuple_ast) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); - } - - if (tuple_ast->name != "tuple") - { - if (!isFunctionCast(tuple_ast)) - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - - if (tuple_ast->arguments->as()->children.empty()) - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - - auto first_arg = tuple_ast->arguments->as()->children.at(0); - if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") - { - const auto * arguments_ast = tuple_ast->arguments->as(); - return arguments_ast ? arguments_ast->children.size() : 0; - } - else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) - { - return inner_literal_tuple->value.getType() == Field::Types::Tuple ? inner_literal_tuple->value.safeGet().size() : 1; - } - - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - else - { - const auto * arguments_ast = tuple_ast->arguments->as(); - return arguments_ast ? arguments_ast->children.size() : 0; - } -} - static void checkSuspiciousIndices(const ASTFunction * index_function) { std::unordered_set unique_index_expression_hashes; @@ -4899,7 +4854,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D } void MergeTreeData::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr) const + const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const { for (const auto & command : commands) { @@ -4927,15 +4882,7 @@ void MergeTreeData::checkAlterPartitionIsPossible( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else - { - // The below `getPartitionIDFromQuery` call will not work for attach / replace because it assumes the partition expressions - // are the same and deliberately uses this storage. Later on, `MergeTreeData::replaceFrom` is called, and it makes the right - // call to `getPartitionIDFromQuery` using source storage. - // Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries. - // But not for `ATTACH PARTITION` queries. - if (command.type != PartitionCommand::REPLACE_PARTITION) - getPartitionIDFromQuery(command.partition, getContext()); - } + getPartitionIDFromQuery(command.partition, local_context); } } } @@ -5669,8 +5616,69 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); return partition_ast.id->as()->value.safeGet(); } + size_t partition_ast_fields_count = 0; ASTPtr partition_value_ast = partition_ast.value->clone(); - auto partition_ast_fields_count = getPartitionAstFieldsCount(partition_ast, partition_value_ast); + if (!partition_ast.fields_count.has_value()) + { + if (partition_value_ast->as()) + { + partition_ast_fields_count = 1; + } + else if (const auto * tuple_ast = partition_value_ast->as()) + { + if (tuple_ast->name != "tuple") + { + if (isFunctionCast(tuple_ast)) + { + if (tuple_ast->arguments->as()->children.empty()) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + auto first_arg = tuple_ast->arguments->as()->children.at(0); + if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) + { + if (inner_literal_tuple->value.getType() == Field::Types::Tuple) + partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); + else + partition_ast_fields_count = 1; + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + } + else + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + else + { + const auto * arguments_ast = tuple_ast->arguments->as(); + if (arguments_ast) + partition_ast_fields_count = arguments_ast->children.size(); + else + partition_ast_fields_count = 0; + } + } + else + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + } + else + { + partition_ast_fields_count = *partition_ast.fields_count; + } if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -7006,35 +7014,23 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - if (queryToStringNullable(my_snapshot->getSortingKeyAST()) != queryToStringNullable(src_snapshot->getSortingKeyAST())) + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? queryToString(ast) : ""; + }; + + if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering"); + if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + if (format_version != src_data->format_version) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different format_version"); - if (queryToStringNullable(my_snapshot->getPrimaryKeyAST()) != queryToStringNullable(src_snapshot->getPrimaryKeyAST())) + if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); - const auto is_a_subset_of = [](const auto & lhs, const auto & rhs) - { - if (lhs.size() > rhs.size()) - return false; - - const auto rhs_set = NameSet(rhs.begin(), rhs.end()); - for (const auto & lhs_element : lhs) - if (!rhs_set.contains(lhs_element)) - return false; - - return true; - }; - - if (!is_a_subset_of(my_snapshot->getColumnsRequiredForPartitionKey(), src_snapshot->getColumnsRequiredForPartitionKey())) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Destination table partition expression columns must be a subset of source table partition expression columns"); - } - const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions) { if (my_descriptions.size() != src_descriptions.size()) @@ -7075,56 +7071,128 @@ std::pair MergeTreeData::cloneAn const ReadSettings & read_settings, const WriteSettings & write_settings) { - return MergeTreeDataPartCloner::clone( - this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); -} + /// Check that the storage policy contains the disk where the src_part is located. + bool does_storage_policy_allow_same_disk = false; + for (const DiskPtr & disk : getStoragePolicy()->getDisks()) + { + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + { + does_storage_policy_allow_same_disk = true; + break; + } + } + if (!does_storage_policy_allow_same_disk) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Could not clone and load part {} because disk does not belong to storage policy", + quoteString(src_part->getDataPartStorage().getFullPath())); -std::pair MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - const MergeTreeData::DataPartPtr & src_part, - const MergeTreePartition & new_partition, - const String & partition_id, - const IMergeTreeDataPart::MinMaxIndex & min_max_index, - const String & tmp_part_prefix, - const StorageMetadataPtr & my_metadata_snapshot, - const IDataPartStorage::ClonePartParams & clone_params, - ContextPtr local_context, - Int64 min_block, - Int64 max_block -) -{ - MergeTreePartInfo dst_part_info(partition_id, min_block, max_block, src_part->info.level); + String dst_part_name = src_part->getNewName(dst_part_info); + String tmp_dst_part_name = tmp_part_prefix + dst_part_name; + auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); - return MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( - this, - src_part, - my_metadata_snapshot, - dst_part_info, - tmp_part_prefix, - local_context->getReadSettings(), - local_context->getWriteSettings(), - new_partition, - min_max_index, - false, - clone_params); -} + /// Why it is needed if we only hardlink files? + auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); + auto src_part_storage = src_part->getDataPartStoragePtr(); -std::pair MergeTreeData::createPartitionAndMinMaxIndexFromSourcePart( - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context) -{ - const auto & src_data = src_part->storage; + scope_guard src_flushed_tmp_dir_lock; + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - auto metadata_manager = std::make_shared(src_part.get()); - IMergeTreeDataPart::MinMaxIndex min_max_index; + /// If source part is in memory, flush it to disk and clone it already in on-disk format + /// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock + /// Construct src_flushed_tmp_part in order to delete part with its directory at destructor + if (auto src_part_in_memory = asInMemoryPart(src_part)) + { + auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - min_max_index.load(src_data, metadata_manager); + auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); + src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); - MergeTreePartition new_partition; + auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot); - new_partition.create(metadata_snapshot, min_max_index.getBlock(src_data), 0u, local_context); + src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage) + .withPartInfo(src_part->info) + .withPartFormatFromDisk() + .build(); - return {new_partition, min_max_index}; + src_flushed_tmp_part->is_temp = true; + src_part_storage = flushed_part_storage; + } + + String with_copy; + if (params.copy_instead_of_hardlink) + with_copy = " (copying data)"; + + auto dst_part_storage = src_part_storage->freeze( + relative_data_path, + tmp_dst_part_name, + read_settings, + write_settings, + /* save_metadata_callback= */ {}, + params); + + if (params.metadata_version_to_write.has_value()) + { + chassert(!params.keep_metadata_version); + auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (getSettings()->fsync_after_insert) + out_metadata->sync(); + } + + LOG_DEBUG(log, "Clone{} part {} to {}{}", + src_flushed_tmp_part ? " flushed" : "", + src_part_storage->getFullPath(), + std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), + with_copy); + + auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) + .withPartFormatFromDisk() + .build(); + + if (!params.copy_instead_of_hardlink && params.hardlinked_files) + { + params.hardlinked_files->source_part_name = src_part->name; + params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); + + for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) + { + if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } + + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } + } + } + + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; + dst_data_part->version.setCreationTID(tid, nullptr); + dst_data_part->storeVersionMetadata(); + + dst_data_part->is_temp = true; + + dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true); + dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime(); + return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); } String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9c433e11b84..f0dbaf0e307 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -231,7 +231,6 @@ public: } }; - using DataParts = std::set; using MutableDataParts = std::set; using DataPartsVector = std::vector; @@ -849,23 +848,6 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings); - std::pair cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - const MergeTreeData::DataPartPtr & src_part, - const MergeTreePartition & new_partition, - const String & partition_id, - const IMergeTreeDataPart::MinMaxIndex & min_max_index, - const String & tmp_part_prefix, - const StorageMetadataPtr & my_metadata_snapshot, - const IDataPartStorage::ClonePartParams & clone_params, - ContextPtr local_context, - Int64 min_block, - Int64 max_block); - - static std::pair createPartitionAndMinMaxIndexFromSourcePart( - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context); - virtual std::vector getMutationsStatus() const = 0; /// Returns true if table can create new parts with adaptive granularity diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp deleted file mode 100644 index 78cb9aa0624..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp +++ /dev/null @@ -1,320 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner"); - -namespace DistinctPartitionExpression -{ -std::unique_ptr updatePartitionFile( - const MergeTreeData & merge_tree_data, - const MergeTreePartition & partition, - const MergeTreeData::MutableDataPartPtr & dst_part, - IDataPartStorage & storage) -{ - storage.removeFile("partition.dat"); - // Leverage already implemented MergeTreePartition::store to create & store partition.dat. - // Checksum is re-calculated later. - return partition.store(merge_tree_data, storage, dst_part->checksums); -} - -IMergeTreeDataPart::MinMaxIndex::WrittenFiles updateMinMaxFiles( - const MergeTreeData & merge_tree_data, - const MergeTreeData::MutableDataPartPtr & dst_part, - IDataPartStorage & storage, - const StorageMetadataPtr & metadata_snapshot) -{ - for (const auto & column_name : MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->partition_key)) - { - auto file = "minmax_" + escapeForFileName(column_name) + ".idx"; - storage.removeFile(file); - } - - return dst_part->minmax_idx->store(merge_tree_data, storage, dst_part->checksums); -} - -void finalizeNewFiles(const std::vector> & files, bool sync_new_files) -{ - for (const auto & file : files) - { - file->finalize(); - if (sync_new_files) - file->sync(); - } -} - -void updateNewPartFiles( - const MergeTreeData & merge_tree_data, - const MergeTreeData::MutableDataPartPtr & dst_part, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - const StorageMetadataPtr & src_metadata_snapshot, - bool sync_new_files) -{ - auto & storage = dst_part->getDataPartStorage(); - - *dst_part->minmax_idx = new_min_max_index; - - auto partition_file = updatePartitionFile(merge_tree_data, new_partition, dst_part, storage); - - auto min_max_files = updateMinMaxFiles(merge_tree_data, dst_part, storage, src_metadata_snapshot); - - IMergeTreeDataPart::MinMaxIndex::WrittenFiles written_files; - - if (partition_file) - written_files.emplace_back(std::move(partition_file)); - - written_files.insert(written_files.end(), std::make_move_iterator(min_max_files.begin()), std::make_move_iterator(min_max_files.end())); - - finalizeNewFiles(written_files, sync_new_files); - - // MergeTreeDataPartCloner::finalize_part calls IMergeTreeDataPart::loadColumnsChecksumsIndexes, which will re-create - // the checksum file if it doesn't exist. Relying on that is cumbersome, but this refactoring is simply a code extraction - // with small improvements. It can be further improved in the future. - storage.removeFile("checksums.txt"); -} -} - -namespace -{ -bool doesStoragePolicyAllowSameDisk(MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part) -{ - for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks()) - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - return true; - return false; -} - -DataPartStoragePtr flushPartStorageToDiskIfInMemory( - MergeTreeData * merge_tree_data, - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const String & tmp_part_prefix, - const String & tmp_dst_part_name, - scope_guard & src_flushed_tmp_dir_lock, - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part) -{ - if (auto src_part_in_memory = asInMemoryPart(src_part)) - { - auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); - - src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); - - auto flushed_part_storage = src_part_in_memory->flushToDisk(*flushed_part_path, metadata_snapshot); - - src_flushed_tmp_part = MergeTreeDataPartBuilder(*merge_tree_data, src_part->name, flushed_part_storage) - .withPartInfo(src_part->info) - .withPartFormatFromDisk() - .build(); - - src_flushed_tmp_part->is_temp = true; - - return flushed_part_storage; - } - - return src_part->getDataPartStoragePtr(); -} - -std::shared_ptr hardlinkAllFiles( - MergeTreeData * merge_tree_data, - const DB::ReadSettings & read_settings, - const DB::WriteSettings & write_settings, - const DataPartStoragePtr & storage, - const String & path, - const DB::IDataPartStorage::ClonePartParams & params) -{ - return storage->freeze( - merge_tree_data->getRelativeDataPath(), - path, - read_settings, - write_settings, - /*save_metadata_callback=*/{}, - params); -} - -std::pair cloneSourcePart( - MergeTreeData * merge_tree_data, - const MergeTreeData::DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const DB::IDataPartStorage::ClonePartParams & params) -{ - const auto dst_part_name = src_part->getNewName(dst_part_info); - - const auto tmp_dst_part_name = tmp_part_prefix + dst_part_name; - - auto temporary_directory_lock = merge_tree_data->getTemporaryPartDirectoryHolder(tmp_dst_part_name); - - src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); - - scope_guard src_flushed_tmp_dir_lock; - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; - - auto src_part_storage = flushPartStorageToDiskIfInMemory( - merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part); - - auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); - - if (params.metadata_version_to_write.has_value()) - { - chassert(!params.keep_metadata_version); - auto out_metadata = dst_part_storage->writeFile( - IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, merge_tree_data->getContext()->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (merge_tree_data->getSettings()->fsync_after_insert) - out_metadata->sync(); - } - - LOG_DEBUG( - log, - "Clone {} part {} to {}{}", - src_flushed_tmp_part ? "flushed" : "", - src_part_storage->getFullPath(), - std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), - false); - - - auto part = MergeTreeDataPartBuilder(*merge_tree_data, dst_part_name, dst_part_storage).withPartFormatFromDisk().build(); - - return std::make_pair(part, std::move(temporary_directory_lock)); -} - -void handleHardLinkedParameterFiles(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) -{ - const auto & hardlinked_files = params.hardlinked_files; - - hardlinked_files->source_part_name = src_part->name; - hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); - - for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) - { - if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - hardlinked_files->hardlinks_from_source_part.insert(it->name()); - } - } -} - -void handleProjections(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) -{ - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) - { - const auto & projection_storage = projection_part->getDataPartStorage(); - for (auto it = projection_storage.iterate(); it->isValid(); it->next()) - { - auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); - } - } - } -} - -MergeTreeData::MutableDataPartPtr finalizePart( - const MergeTreeData::MutableDataPartPtr & dst_part, const DB::IDataPartStorage::ClonePartParams & params, bool require_part_metadata) -{ - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; - dst_part->version.setCreationTID(tid, nullptr); - dst_part->storeVersionMetadata(); - - dst_part->is_temp = true; - - dst_part->loadColumnsChecksumsIndexes(require_part_metadata, true); - - dst_part->modification_time = dst_part->getDataPartStorage().getLastModified().epochTime(); - - return dst_part; -} - -std::pair cloneAndHandleHardlinksAndProjections( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const IDataPartStorage::ClonePartParams & params) -{ - if (!doesStoragePolicyAllowSameDisk(merge_tree_data, src_part)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Could not clone and load part {} because disk does not belong to storage policy", - quoteString(src_part->getDataPartStorage().getFullPath())); - - auto [destination_part, temporary_directory_lock] = cloneSourcePart( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - if (!params.copy_instead_of_hardlink && params.hardlinked_files) - { - handleHardLinkedParameterFiles(src_part, params); - handleProjections(src_part, params); - } - - return std::make_pair(destination_part, std::move(temporary_directory_lock)); -} -} - -std::pair MergeTreeDataPartCloner::clone( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - bool require_part_metadata, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings) -{ - auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); -} - -std::pair MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - bool sync_new_files, - const IDataPartStorage::ClonePartParams & params) -{ - auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( - merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); - - DistinctPartitionExpression::updateNewPartFiles( - *merge_tree_data, destination_part, new_partition, new_min_max_index, src_part->storage.getInMemoryMetadataPtr(), sync_new_files); - - return std::make_pair(finalizePart(destination_part, params, false), std::move(temporary_directory_lock)); -} - -} diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.h b/src/Storages/MergeTree/MergeTreeDataPartCloner.h deleted file mode 100644 index 53585f20b7f..00000000000 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.h +++ /dev/null @@ -1,43 +0,0 @@ -#pragma once - -namespace DB -{ - -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; -struct MergeTreePartition; -class IMergeTreeDataPart; - -class MergeTreeDataPartCloner -{ -public: - using DataPart = IMergeTreeDataPart; - using MutableDataPartPtr = std::shared_ptr; - using DataPartPtr = std::shared_ptr; - - static std::pair clone( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - bool require_part_metadata, - const IDataPartStorage::ClonePartParams & params, - const ReadSettings & read_settings, - const WriteSettings & write_settings); - - static std::pair cloneWithDistinctPartitionExpression( - MergeTreeData * merge_tree_data, - const DataPartPtr & src_part, - const StorageMetadataPtr & metadata_snapshot, - const MergeTreePartInfo & dst_part_info, - const String & tmp_part_prefix, - const ReadSettings & read_settings, - const WriteSettings & write_settings, - const MergeTreePartition & new_partition, - const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, - bool sync_new_files, - const IDataPartStorage::ClonePartParams & params); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index d86ff3a17ff..9d373504473 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -143,13 +144,22 @@ void MergeTreeDataPartWriterWide::addStreams( auto ast = parseQuery(codec_parser, "(" + Poco::toUpper(settings.marks_compression_codec) + ")", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); CompressionCodecPtr marks_compression_codec = CompressionCodecFactory::instance().get(ast, nullptr); + const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); + + UInt64 max_compress_block_size = 0; + if (column_desc) + if (const auto * value = column_desc->settings.tryGet("max_compress_block_size")) + max_compress_block_size = value->safeGet(); + if (!max_compress_block_size) + max_compress_block_size = settings.max_compress_block_size; + column_streams[stream_name] = std::make_unique>( stream_name, data_part->getDataPartStoragePtr(), stream_name, DATA_FILE_EXTENSION, stream_name, marks_file_extension, compression_codec, - settings.max_compress_block_size, + max_compress_block_size, marks_compression_codec, settings.marks_compress_block_size, settings.query_write_settings); @@ -323,6 +333,13 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( WrittenOffsetColumns & offset_columns) { StreamsWithMarks result; + const auto column_desc = metadata_snapshot->columns.tryGetColumnDescription(GetColumnsOptions(GetColumnsOptions::AllPhysical), column.getNameInStorage()); + UInt64 min_compress_block_size = 0; + if (column_desc) + if (const auto * value = column_desc->settings.tryGet("min_compress_block_size")) + min_compress_block_size = value->safeGet(); + if (!min_compress_block_size) + min_compress_block_size = settings.min_compress_block_size; data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -335,7 +352,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( auto & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. - if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) + if (stream.compressed_hashing.offset() >= min_compress_block_size) stream.compressed_hashing.next(); StreamNameAndMark stream_with_mark; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2a381afa805..3c0b2d2b42e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -314,8 +315,13 @@ Block MergeTreeDataWriter::mergeBlock( IColumn::Permutation *& permutation, const MergeTreeData::MergingParams & merging_params) { + OpenTelemetry::SpanHolder span("MergeTreeDataWriter::mergeBlock"); + size_t block_size = block.rows(); + span.addAttribute("clickhouse.rows", block_size); + span.addAttribute("clickhouse.columns", block.columns()); + auto get_merging_algorithm = [&]() -> std::shared_ptr { switch (merging_params.mode) @@ -351,6 +357,8 @@ Block MergeTreeDataWriter::mergeBlock( if (!merging_algorithm) return block; + span.addAttribute("clickhouse.merging_algorithm", merging_algorithm->getName()); + Chunk chunk(block.getColumns(), block_size); IMergingAlgorithm::Input input; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 76ef3be25b3..ddeaf69136a 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,45 +467,6 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } -void MergeTreePartition::createAndValidateMinMaxPartitionIds( - const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context) -{ - if (!metadata_snapshot->hasPartitionKey()) - return; - - auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block_with_min_max_partition_ids, context); - value.resize(partition_key_names_and_types.size()); - - /// Executing partition_by expression adds new columns to passed block according to partition functions. - /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back - /// with just `modulo`, because it was a temporary substitution. - static constexpr std::string_view modulo_legacy_function_name = "moduloLegacy"; - - size_t i = 0; - for (const auto & element : partition_key_names_and_types) - { - auto & partition_column = block_with_min_max_partition_ids.getByName(element.name); - - if (element.name.starts_with(modulo_legacy_function_name)) - partition_column.name.replace(0, modulo_legacy_function_name.size(), "modulo"); - - Field extracted_min_partition_id_field; - Field extracted_max_partition_id_field; - - partition_column.column->get(0, extracted_min_partition_id_field); - partition_column.column->get(1, extracted_max_partition_id_field); - - if (extracted_min_partition_id_field != extracted_max_partition_id_field) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, - "Can not create the partition. A partition can not contain values that have different partition ids"); - } - - partition_column.column->get(0u, value[i++]); - } -} - NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index fd7ae02cde4..78b141f26ec 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,12 +1,11 @@ #pragma once -#include +#include #include #include #include #include -#include -#include +#include namespace DB { @@ -52,11 +51,6 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); - /// Copy of MergeTreePartition::create, but also validates if min max partition keys are equal. If they are different, - /// it means the partition can't be created because the data doesn't belong to the same partition. - void createAndValidateMinMaxPartitionIds( - const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context); - static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp deleted file mode 100644 index 21bcdb84a96..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp +++ /dev/null @@ -1,91 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace -{ -bool isDestinationPartitionExpressionMonotonicallyIncreasing( - const std::vector & hyperrectangle, const MergeTreeData & destination_storage) -{ - auto destination_table_metadata = destination_storage.getInMemoryMetadataPtr(); - - auto key_description = destination_table_metadata->getPartitionKey(); - auto definition_ast = key_description.definition_ast->clone(); - - auto table_identifier = std::make_shared(destination_storage.getStorageID().getTableName()); - auto table_with_columns - = TableWithColumnNamesAndTypes{DatabaseAndTableWithAlias(table_identifier), destination_table_metadata->getColumns().getOrdinary()}; - - auto expression_list = extractKeyExpressionList(definition_ast); - - MonotonicityCheckVisitor::Data data{{table_with_columns}, destination_storage.getContext(), /*group_by_function_hashes*/ {}}; - - for (auto i = 0u; i < expression_list->children.size(); i++) - { - data.range = hyperrectangle[i]; - - MonotonicityCheckVisitor(data).visit(expression_list->children[i]); - - if (!data.monotonicity.is_monotonic || !data.monotonicity.is_positive) - return false; - } - - return true; -} - -bool isExpressionDirectSubsetOf(const ASTPtr source, const ASTPtr destination) -{ - auto source_expression_list = extractKeyExpressionList(source); - auto destination_expression_list = extractKeyExpressionList(destination); - - std::unordered_set source_columns; - - for (auto i = 0u; i < source_expression_list->children.size(); ++i) - source_columns.insert(source_expression_list->children[i]->getColumnName()); - - for (auto i = 0u; i < destination_expression_list->children.size(); ++i) - if (!source_columns.contains(destination_expression_list->children[i]->getColumnName())) - return false; - - return true; -} -} - -void MergeTreePartitionCompatibilityVerifier::verify( - const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts) -{ - const auto source_metadata = source_storage.getInMemoryMetadataPtr(); - const auto destination_metadata = destination_storage.getInMemoryMetadataPtr(); - - const auto source_partition_key_ast = source_metadata->getPartitionKeyAST(); - const auto destination_partition_key_ast = destination_metadata->getPartitionKeyAST(); - - // If destination partition expression columns are a subset of source partition expression columns, - // there is no need to check for monotonicity. - if (isExpressionDirectSubsetOf(source_partition_key_ast, destination_partition_key_ast)) - return; - - const auto src_global_min_max_indexes = MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(source_parts, destination_storage); - - assert(!src_global_min_max_indexes.hyperrectangle.empty()); - - if (!isDestinationPartitionExpressionMonotonicallyIncreasing(src_global_min_max_indexes.hyperrectangle, destination_storage)) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Destination table partition expression is not monotonically increasing"); - - MergeTreePartition().createAndValidateMinMaxPartitionIds( - destination_storage.getInMemoryMetadataPtr(), - src_global_min_max_indexes.getBlock(destination_storage), - destination_storage.getContext()); -} - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h deleted file mode 100644 index 1682add3ebd..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -/* - * Verifies that source and destination partitions are compatible. - * To be compatible, one of the following criteria must be met: - * 1. Destination partition expression columns are a subset of source partition columns; or - * 2. Destination partition expression is monotonic on the source global min_max idx Range AND the computer partition id for - * the source global min_max idx range is the same. - * - * If not, an exception is thrown. - * */ - -class MergeTreePartitionCompatibilityVerifier -{ -public: - using DataPart = IMergeTreeDataPart; - using DataPartPtr = std::shared_ptr; - using DataPartsVector = std::vector; - - static void - verify(const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp deleted file mode 100644 index 0871efadf0c..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include - -namespace DB -{ - -IMergeTreeDataPart::MinMaxIndex -MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(const DataPartsVector & parts, const MergeTreeData & storage) -{ - IMergeTreeDataPart::MinMaxIndex global_min_max_indexes; - - for (const auto & part : parts) - { - auto metadata_manager = std::make_shared(part.get()); - - auto local_min_max_index = MergeTreeData::DataPart::MinMaxIndex(); - - local_min_max_index.load(storage, metadata_manager); - - global_min_max_indexes.merge(local_min_max_index); - } - - return global_min_max_indexes; -} - -} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h deleted file mode 100644 index 4f271177246..00000000000 --- a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once - -#include - -#include -#include - -namespace DB -{ - -/* - * Calculates global min max indexes for a given set of parts on given storage. - * */ -class MergeTreePartitionGlobalMinMaxIdxCalculator -{ - using DataPart = IMergeTreeDataPart; - using DataPartPtr = std::shared_ptr; - using DataPartsVector = std::vector; - -public: - static IMergeTreeDataPart::MinMaxIndex calculate(const DataPartsVector & parts, const MergeTreeData & storage); -}; - -} diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 654f6cf397f..153930b400d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -213,6 +213,27 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const } } +void MergeTreeColumnSettings::validate(const SettingsChanges & changes) +{ + static const MergeTreeSettings merge_tree_settings; + static const std::set allowed_column_level_settings = + { + "min_compress_block_size", + "max_compress_block_size" + }; + + for (const auto & change : changes) + { + if (!allowed_column_level_settings.contains(change.name)) + throw Exception( + ErrorCodes::UNKNOWN_SETTING, + "Setting {} is unknown or not supported at column level, supported settings: {}", + change.name, + fmt::join(allowed_column_level_settings, ", ")); + merge_tree_settings.checkCanSet(change.name, change.value); + } +} + std::vector MergeTreeSettings::getAllRegisteredNames() const { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 732389927ab..96cab9c0293 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -277,4 +277,11 @@ struct MergeTreeSettings : public BaseSettings, public using MergeTreeSettingsPtr = std::shared_ptr; + +/// Column-level Merge-Tree settings which overwrite MergeTree settings +namespace MergeTreeColumnSettings +{ + void validate(const SettingsChanges & changes); +} + } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fd5354a00a9..fbdde15c2af 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -5,9 +5,9 @@ #include #include +#include #include #include -#include #include "Common/Exception.h" #include #include @@ -20,30 +20,27 @@ #include #include #include -#include #include +#include #include #include #include #include #include -#include #include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -218,16 +215,25 @@ void StorageMergeTree::read( { if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree) { - const auto table_id = getStorageID(); - const auto & modified_query_ast = ClusterProxy::rewriteSelectQuery( - local_context, query_info.query, - table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - + ASTPtr modified_query_ast; Block header; if (local_context->getSettingsRef().allow_experimental_analyzer) - header = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + { + QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); + rewriteJoinToGlobalJoin(modified_query_tree); + modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree); + header = InterpreterSelectQueryAnalyzer::getSampleBlock( + modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); + modified_query_ast = queryNodeToSelectQuery(modified_query_tree); + } else - header = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + { + const auto table_id = getStorageID(); + modified_query_ast = ClusterProxy::rewriteSelectQuery(local_context, query_info.query, + table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); + header + = InterpreterSelectQuery(modified_query_ast, local_context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + } ClusterProxy::SelectStreamFactory select_stream_factory = ClusterProxy::SelectStreamFactory( @@ -238,7 +244,6 @@ void StorageMergeTree::read( ClusterProxy::executeQueryWithParallelReplicas( query_plan, - getStorageID(), select_stream_factory, modified_query_ast, local_context, @@ -2044,74 +2049,42 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = src_data.getPartitionIDFromQuery(partition, local_context); + String partition_id = getPartitionIDFromQuery(partition, local_context); DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); - - bool attach_empty_partition = !replace && src_parts.empty(); - if (attach_empty_partition) - return; - MutableDataPartsVector dst_parts; std::vector dst_parts_locks; static const String TMP_PREFIX = "tmp_replace_from_"; - const auto my_partition_expression = my_metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different && !src_parts.empty()) - MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_parts); - - for (DataPartPtr & src_part : src_parts) + for (const DataPartPtr & src_part : src_parts) { if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", partition_id, src_part->name); - IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; /// This will generate unique name in scope of current server process. - auto index = insert_increment.get(); + Int64 temp_index = insert_increment.get(); + MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( - src_part, my_metadata_snapshot, local_context); - - auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - src_part, - new_partition, - new_partition.getID(*this), - new_min_max_index, - TMP_PREFIX, - my_metadata_snapshot, - clone_params, - local_context, - index, - index); - - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); } + /// ATTACH empty part set + if (!replace && dst_parts.empty()) + return; + MergeTreePartInfo drop_range; if (replace) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 512811e39d7..715cbab9eea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -26,21 +26,22 @@ #include +#include #include #include #include #include #include +#include #include #include +#include #include #include #include #include #include -#include #include -#include #include #include #include @@ -52,11 +53,9 @@ #include #include #include -#include #include #include #include -#include #include #include @@ -2714,48 +2713,16 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - - const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_table->getInMemoryMetadataPtr()->getPartitionKeyAST(); - - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( - part_desc->src_table_part, metadata_snapshot, getContext()); - - auto partition_id = new_partition.getID(*this); - - auto [res_part, temporary_part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - part_desc->src_table_part, - new_partition, - partition_id, - new_min_max_index, - TMP_PREFIX + "clone_", - metadata_snapshot, - clone_params, - getContext(), - part_desc->new_part_info.min_block, - part_desc->new_part_info.max_block); - - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); - } - else - { - auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, - TMP_PREFIX + "clone_", - part_desc->new_part_info, - metadata_snapshot, - clone_params, - getContext()->getReadSettings(), - getContext()->getWriteSettings()); - - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); - } + auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( + part_desc->src_table_part, + TMP_PREFIX + "clone_", + part_desc->new_part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); } else if (!part_desc->replica.empty()) { @@ -5418,7 +5385,9 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( if (local_context->getSettingsRef().allow_experimental_analyzer) { - auto modified_query_tree = buildQueryTreeForShard(query_info, query_info.query_tree); + QueryTreeNodePtr modified_query_tree = query_info.query_tree->clone(); + rewriteJoinToGlobalJoin(modified_query_tree); + modified_query_tree = buildQueryTreeForShard(query_info, modified_query_tree); header = InterpreterSelectQueryAnalyzer::getSampleBlock( modified_query_tree, local_context, SelectQueryOptions(processed_stage).analyze()); @@ -5441,7 +5410,6 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( ClusterProxy::executeQueryWithParallelReplicas( query_plan, - getStorageID(), select_stream_factory, modified_query_ast, local_context, @@ -7885,22 +7853,11 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = src_data.getPartitionIDFromQuery(partition, query_context); + String partition_id = getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); - bool attach_empty_partition = !replace && src_all_parts.empty(); - if (attach_empty_partition) - return; - - const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); - const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); - const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); - - if (is_partition_exp_different && !src_all_parts.empty()) - MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_all_parts); - LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; @@ -7955,18 +7912,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom( "Cannot replace partition '{}' because part '{}" "' has inconsistent granularity with table", partition_id, src_part->name); - IMergeTreeDataPart::MinMaxIndex min_max_index = *src_part->minmax_idx; - MergeTreePartition merge_tree_partition = src_part->partition; - - if (is_partition_exp_different) - { - auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(src_part, metadata_snapshot, query_context); - - merge_tree_partition = new_partition; - min_max_index = new_min_max_index; - partition_id = merge_tree_partition.getID(*this); - } - String hash_hex = src_part->checksums.getTotalChecksumHex(); const bool is_duplicated_part = replaced_parts.contains(hash_hex); replaced_parts.insert(hash_hex); @@ -7985,52 +7930,27 @@ void StorageReplicatedMergeTree::replacePartitionFrom( continue; } + UInt64 index = lock->getNumber(); + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; - - UInt64 index = lock->getNumber(); - IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - - if (is_partition_exp_different) - { - auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( - src_part, - merge_tree_partition, - partition_id, - min_max_index, - TMP_PREFIX, - metadata_snapshot, - clone_params, - query_context, - index, - index); - - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - else - { - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); - - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); - } - + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); src_parts.emplace_back(src_part); + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 00cc5e3ee58..5ea28d9e09c 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -373,11 +373,37 @@ QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeN removeGroupingFunctionSpecializations(query_tree_to_modify); - // std::cerr << "====================== build 1 \n" << query_tree_to_modify->dumpTree() << std::endl; createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext()); - // std::cerr << "====================== build 2 \n" << query_tree_to_modify->dumpTree() << std::endl; return query_tree_to_modify; } +class RewriteJoinToGlobalJoinVisitor : public InDepthQueryTreeVisitor +{ +public: + using Base = InDepthQueryTreeVisitor; + using Base::Base; + + void visitImpl(QueryTreeNodePtr & node) + { + if (auto * join_node = node->as()) + join_node->setLocality(JoinLocality::Global); + } + + static bool needChildVisit(QueryTreeNodePtr & parent, QueryTreeNodePtr & child) + { + auto * join_node = parent->as(); + if (join_node && join_node->getRightTableExpression() == child) + return false; + + return true; + } +}; + +void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify) +{ + RewriteJoinToGlobalJoinVisitor visitor; + visitor.visit(query_tree_to_modify); +} + } diff --git a/src/Storages/buildQueryTreeForShard.h b/src/Storages/buildQueryTreeForShard.h index 05d63faeb9f..eec5a0dc38a 100644 --- a/src/Storages/buildQueryTreeForShard.h +++ b/src/Storages/buildQueryTreeForShard.h @@ -12,4 +12,6 @@ using QueryTreeNodePtr = std::shared_ptr; QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify); +void rewriteJoinToGlobalJoin(QueryTreeNodePtr query_tree_to_modify); + } diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 4643d109c3d..a84f912f371 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -27,8 +27,9 @@ 00917_multiple_joins_denny_crane 02725_agg_projection_resprect_PK 02763_row_policy_storage_merge_alias -02784_parallel_replicas_automatic_decision_join 02818_parameterized_view_with_cte_multiple_usage +# Check after constants refactoring +02901_parallel_replicas_rollup # Flaky. Please don't delete them without fixing them: 01287_max_execution_speed 02003_WithMergeableStateAfterAggregationAndLimit_LIMIT_BY_LIMIT_OFFSET diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml deleted file mode 100644 index b40730e9f7d..00000000000 --- a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - - - true - - replica1 - 9000 - - - replica2 - 9000 - - - - - diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py deleted file mode 100644 index 1d8ac4e9e37..00000000000 --- a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py +++ /dev/null @@ -1,214 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) - -replica1 = cluster.add_instance( - "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) -replica2 = cluster.add_instance( - "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - except Exception as ex: - print(ex) - finally: - cluster.shutdown() - - -def cleanup(nodes): - for node in nodes: - node.query("DROP TABLE IF EXISTS source SYNC") - node.query("DROP TABLE IF EXISTS destination SYNC") - - -def create_table(node, table_name, replicated): - replica = node.name - engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" - if replicated - else "MergeTree()" - ) - partition_expression = ( - "toYYYYMMDD(timestamp)" if table_name == "source" else "toYYYYMM(timestamp)" - ) - node.query_with_retry( - """ - CREATE TABLE {table_name}(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY {partition_expression} - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - table_name=table_name, - engine=engine, - partition_expression=partition_expression, - ) - ) - - -def test_both_replicated(start_cluster): - for node in [replica1, replica2]: - create_table(node, "source", True) - create_table(node, "destination", True) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - - assert_eq_with_retry( - replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination", - replica2.query(f"SELECT * FROM destination"), - ) - - cleanup([replica1, replica2]) - - -def test_only_destination_replicated(start_cluster): - create_table(replica1, "source", False) - create_table(replica1, "destination", True) - create_table(replica2, "destination", True) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA destination") - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - - assert_eq_with_retry( - replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination", - replica2.query(f"SELECT * FROM destination"), - ) - - cleanup([replica1, replica2]) - - -def test_both_replicated_partitioned_to_unpartitioned(start_cluster): - def create_tables(nodes): - for node in nodes: - source_engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" - ) - node.query( - """ - CREATE TABLE source(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp) - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=source_engine, - ) - ) - - destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" - node.query( - """ - CREATE TABLE destination(timestamp DateTime) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY tuple() - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=destination_engine, - ) - ) - - create_tables([replica1, replica2]) - - replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") - replica1.query("INSERT INTO source VALUES ('2010-03-03 02:01:01')") - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" - ) - replica1.query( - f"ALTER TABLE destination ATTACH PARTITION ID '20100303' FROM source" - ) - - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY timestamp", - "2010-03-02 02:01:01\n2010-03-03 02:01:01\n", - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY timestamp", - replica2.query(f"SELECT * FROM destination ORDER BY timestamp"), - ) - - cleanup([replica1, replica2]) - - -def test_both_replicated_different_exp_same_id(start_cluster): - def create_tables(nodes): - for node in nodes: - source_engine = ( - f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" - ) - node.query( - """ - CREATE TABLE source(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY a % 3 - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=source_engine, - ) - ) - - destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" - node.query( - """ - CREATE TABLE destination(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) - ENGINE = {engine} - ORDER BY tuple() PARTITION BY a - SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; - """.format( - engine=destination_engine, - ) - ) - - create_tables([replica1, replica2]) - - replica1.query( - "INSERT INTO source (a, b, c, extra, sign) VALUES (1, 5, 9, 1000, 1)" - ) - replica1.query( - "INSERT INTO source (a, b, c, extra, sign) VALUES (2, 6, 10, 1000, 1)" - ) - replica1.query("SYSTEM SYNC REPLICA source") - replica1.query("SYSTEM SYNC REPLICA destination") - - replica1.query(f"ALTER TABLE destination ATTACH PARTITION 1 FROM source") - replica1.query(f"ALTER TABLE destination ATTACH PARTITION 2 FROM source") - - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY a", - "1\t5\t9\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n2\t6\t10\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n", - ) - assert_eq_with_retry( - replica1, - f"SELECT * FROM destination ORDER BY a", - replica2.query(f"SELECT * FROM destination ORDER BY a"), - ) - - cleanup([replica1, replica2]) diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py b/tests/integration/test_system_reload_async_metrics/__init__.py similarity index 100% rename from tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py rename to tests/integration/test_system_reload_async_metrics/__init__.py diff --git a/tests/integration/test_system_reload_async_metrics/configs/default.xml b/tests/integration/test_system_reload_async_metrics/configs/default.xml new file mode 100644 index 00000000000..eb168c1967d --- /dev/null +++ b/tests/integration/test_system_reload_async_metrics/configs/default.xml @@ -0,0 +1,6 @@ + + + 60000 + 60000 + + diff --git a/tests/integration/test_system_reload_async_metrics/test.py b/tests/integration/test_system_reload_async_metrics/test.py new file mode 100644 index 00000000000..f0572cd2db6 --- /dev/null +++ b/tests/integration/test_system_reload_async_metrics/test.py @@ -0,0 +1,47 @@ +import os +import pytest +import shutil +import time +from helpers.cluster import ClickHouseCluster + +# Tests that SYSTEM RELOAD ASYNCHRONOUS METRICS works. + +# Config default.xml sets a large refresh interval of asynchronous metrics, so that the periodic updates don't interfere with the manual +# update below. +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/default.xml"], + stay_alive=True, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CONFIG_DIR = os.path.join(SCRIPT_DIR, "configs") + + +def test_system_reload_async_metrics(start_cluster): + node.query("SYSTEM DROP QUERY CACHE") + + res1 = node.query( + "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" + ) + + # create table and test that the table creation is reflected in the asynchronous metrics + node.query("CREATE TABLE tab (col UInt64) ENGINE MergeTree ORDER BY tuple()") + + node.query("SYSTEM RELOAD ASYNCHRONOUS METRICS") + + res2 = node.query( + "SELECT value FROM system.asynchronous_metrics WHERE metric = 'NumberOfTables'" + ) + assert int(res1.rstrip()) + 1 == int(res2.rstrip()) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index f9f5c2bd3df..6a7e4748130 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -120,6 +120,7 @@ SYSTEM RELOAD DICTIONARY ['SYSTEM RELOAD DICTIONARIES','RELOAD DICTIONARY','RELO SYSTEM RELOAD MODEL ['SYSTEM RELOAD MODELS','RELOAD MODEL','RELOAD MODELS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD FUNCTION ['SYSTEM RELOAD FUNCTIONS','RELOAD FUNCTION','RELOAD FUNCTIONS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD EMBEDDED DICTIONARIES ['RELOAD EMBEDDED DICTIONARIES'] GLOBAL SYSTEM RELOAD +SYSTEM RELOAD ASYNCHRONOUS METRICS ['RELOAD ASYNCHRONOUS METRICS'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD [] \N SYSTEM SYSTEM RESTART DISK ['SYSTEM RESTART DISK'] GLOBAL SYSTEM SYSTEM MERGES ['SYSTEM STOP MERGES','SYSTEM START MERGES','STOP MERGES','START MERGES'] TABLE SYSTEM diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference deleted file mode 100644 index f1d036b08bf..00000000000 --- a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference +++ /dev/null @@ -1,467 +0,0 @@ --- { echoOn } --- Should be allowed since destination partition expr is monotonically increasing and compatible -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though --- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -20100302 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -20100302 --- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 -2010-03-02 02:01:03 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 --- Should be allowed because dst partition exp is monot inc and data is not split -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; -SELECT * FROM source ORDER BY productName; -mop general -rice food -spaghetti food -SELECT * FROM destination ORDER BY productName; -rice food -spaghetti food -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -59532f3c39a412a413f0f014c7750a9d -59532f3c39a412a413f0f014c7750a9d -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; -SELECT * FROM source ORDER BY productName; -mop general -rice food -spaghetti food -SELECT * FROM destination ORDER BY productName; -rice food -spaghetti food -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -59532f3c39a412a413f0f014c7750a9d -59532f3c39a412a413f0f014c7750a9d --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); -CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); -INSERT INTO TABLE source VALUES (1267495261123); -ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; -SELECT * FROM source ORDER BY timestamp; -1267495261123 -SELECT * FROM destination ORDER BY timestamp; -1267495261123 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -2010 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '14670' from source; -SELECT * FROM source ORDER BY timestamp; -1267495261123 -SELECT * FROM destination ORDER BY timestamp; -1267495261123 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -2010 --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); -CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); -ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -2011-02-02 02:01:03 1 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -14670 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '2010' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -2011-02-02 02:01:03 1 1 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 1 1 -2010-03-02 02:01:01 1 1 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -14670 --- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately --- fall into the same partition. --- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all --- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that --- partition by tuple() is accepted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '201003' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -all --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1-2 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1-2 --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; -SELECT * FROM source ORDER BY (a, b, c); -1 2 3 -1 2 4 -SELECT * FROM destination ORDER BY (a, b, c); -1 2 3 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -1 --- Should be allowed. Special test case, tricky to explain. First column of source partition expression is --- timestamp, while first column of destination partition expression is `A`. One of the previous implementations --- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; -CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); -ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -5 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 5 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -5 --- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically --- increasing in the source partition min max indexes. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -INSERT INTO TABLE source VALUES (6, 12); -ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -3-6 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (6, 12) from source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -3-6 --- Should be allowed. The same scenario as above, but partition expressions inverted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -INSERT INTO TABLE source VALUES (6, 12); -ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -6-12 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION (3, 6) from source; -SELECT * FROM source ORDER BY A; -6 12 -SELECT * FROM destination ORDER BY A; -6 12 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -6-12 --- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE - source(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMMDD(timestamp) - ORDER BY tuple(); -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 -TRUNCATE TABLE destination; -ALTER TABLE destination ATTACH PARTITION '20100302' from source; -SELECT * FROM source ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT * FROM destination ORDER BY timestamp; -2010-03-02 02:01:01 -2010-03-02 02:01:03 -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; -201003 --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } --- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } --- Should not be allowed because dst partition exp depends on a different set of columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } --- Should not be allowed because dst partition exp is not monotonically increasing -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); -CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); -INSERT INTO TABLE source VALUES ('bread'), ('mop'); -INSERT INTO TABLE source VALUES ('broccoli'); -ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } --- Empty/ non-existent partition, same partition expression. Nothing should happen -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Empty/ non-existent partition, different partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -ALTER TABLE destination REPLACE PARTITION '1' FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; --- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -INSERT INTO TABLE destination VALUES (1); -ALTER TABLE destination REPLACE PARTITION '1' FROM source; -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql deleted file mode 100644 index 9547d6ae249..00000000000 --- a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql +++ /dev/null @@ -1,485 +0,0 @@ --- { echoOn } --- Should be allowed since destination partition expr is monotonically increasing and compatible -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though --- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); - -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); - -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION 0 FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because dst partition exp is monot inc and data is not split -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; - -SELECT * FROM source ORDER BY productName; -SELECT * FROM destination ORDER BY productName; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; - -SELECT * FROM source ORDER BY productName; -SELECT * FROM destination ORDER BY productName; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); -CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); - -INSERT INTO TABLE source VALUES (1267495261123); - -ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '14670' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 - -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); -CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); - -ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '2010' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately --- fall into the same partition. --- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that --- partition by tuple() is accepted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '201003' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); - -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); - -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns --- Columns in this case refer to the expression elements, not to the actual table columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); -CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; - -INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); - -ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; - -SELECT * FROM source ORDER BY (a, b, c); -SELECT * FROM destination ORDER BY (a, b, c); -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. Special test case, tricky to explain. First column of source partition expression is --- timestamp, while first column of destination partition expression is `A`. One of the previous implementations --- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; -CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); - -ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically --- increasing in the source partition min max indexes. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); - -INSERT INTO TABLE source VALUES (6, 12); - -ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (6, 12) from source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed. The same scenario as above, but partition expressions inverted. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); -CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); - -INSERT INTO TABLE source VALUES (6, 12); - -ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION (3, 6) from source; - -SELECT * FROM source ORDER BY A; -SELECT * FROM destination ORDER BY A; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; -CREATE TABLE - source(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMMDD(timestamp) - ORDER BY tuple(); - -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; -CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); - -CREATE TABLE - destination(timestamp DateTime) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') - PARTITION BY toYYYYMM(timestamp) - ORDER BY tuple(); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - -TRUNCATE TABLE destination; - -ALTER TABLE destination ATTACH PARTITION '20100302' from source; - -SELECT * FROM source ORDER BY timestamp; -SELECT * FROM destination ORDER BY timestamp; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source SYNC; -DROP TABLE IF EXISTS destination SYNC; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); - -ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } - --- Should not be allowed because data would be split into two different partitions -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); - -CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; - -INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); - -ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } -ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } - --- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } - --- Should not be allowed because dst partition exp depends on a different set of columns -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); -CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); - -INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); -INSERT INTO TABLE source VALUES ('rice', 'food'); - -ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } - --- Should not be allowed because dst partition exp is not monotonically increasing -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); -CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); - -INSERT INTO TABLE source VALUES ('bread'), ('mop'); -INSERT INTO TABLE source VALUES ('broccoli'); - -ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } -ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } - --- Empty/ non-existent partition, same partition expression. Nothing should happen -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Empty/ non-existent partition, different partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; -ALTER TABLE destination ATTACH PARTITION 1 FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); -CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); - -ALTER TABLE destination REPLACE PARTITION '1' FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; - --- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. --- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 -DROP TABLE IF EXISTS source; -DROP TABLE IF EXISTS destination; - -CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; -CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; - -INSERT INTO TABLE destination VALUES (1); - -ALTER TABLE destination REPLACE PARTITION '1' FROM source; - -SELECT * FROM destination; -SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 608fac626fa..ec4928bc325 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -20,9 +20,21 @@ 23 Sx>b:^UG XpedE)Q: 7433019734386307503 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== +0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer = 1 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 -1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; =============== OUTER QUERY (NO PARALLEL) =============== >T%O ,z< 10 NQTpY# W\\Xx4 10 @@ -39,6 +51,16 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 tT%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +t 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` +0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` -0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_7105554115296635472_12427301373021079614` USING (`key`) GROUP BY `key`, `value1`, `value2` -1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index a117378b0bf..7693d0da295 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -21,7 +21,6 @@ SELECT * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 100; -SET allow_experimental_analyzer = 0; SET max_parallel_replicas = 3; SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; @@ -39,6 +38,18 @@ FROM join_inner_table GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10; +-- settings allow_experimental_analyzer=0; + +-- SELECT +-- key, +-- value1, +-- value2, +-- toUInt64(min(time)) AS start_ts +-- FROM join_inner_table +-- PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +-- GROUP BY key, value1, value2 +-- ORDER BY key, value1, value2 +-- LIMIT 10 settings allow_experimental_analyzer=1; SELECT '=============== INNER QUERY (PARALLEL) ==============='; @@ -53,18 +64,31 @@ PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1 GROUP BY key, value1, value2 ORDER BY key, value1, value2 LIMIT 10 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; + +-- Parallel inner query alone +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; SYSTEM FLUSH LOGS; -- There should be 4 queries. The main query as received by the initiator and the 3 equal queries sent to each replica -SELECT is_initial_query, count() as c, query, +SELECT is_initial_query, count() as c, replaceRegexpAll(query, '_data_(\d+)_(\d+)', '_data_') as query FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' - AND initial_query_id = + AND initial_query_id IN ( SELECT query_id FROM system.query_log @@ -160,18 +184,48 @@ FROM ) GROUP BY value1, value2 ORDER BY value1, value2 -SETTINGS allow_experimental_parallel_reading_from_replicas = 1; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; + +-- Parallel full query +SELECT + value1, + value2, + avg(count) AS avg +FROM + ( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 + ) +GROUP BY value1, value2 +ORDER BY value1, value2 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; SYSTEM FLUSH LOGS; -- There should be 7 queries. The main query as received by the initiator, the 3 equal queries to execute the subquery -- in the inner join and the 3 queries executing the whole query (but replacing the subquery with a temp table) -SELECT is_initial_query, count() as c, query, +SELECT is_initial_query, count() as c, replaceRegexpAll(query, '_data_(\d+)_(\d+)', '_data_') as query FROM system.query_log WHERE event_date >= yesterday() AND type = 'QueryFinish' - AND initial_query_id = + AND initial_query_id IN ( SELECT query_id FROM system.query_log diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index 802d55f8ae3..521e3e2edbc 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -1,6 +1,4 @@ -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 0 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_10M_pure 1 estimated parallel replicas -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 0 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_5M_pure 2 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 1 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index baeeb820da5..1a74c3230c6 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -68,7 +68,7 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ - |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' + |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' | sort -n -k2 -b | grep -Pv "\t0 estimated parallel replicas" } query_id_base="02784_automatic_parallel_replicas_join-$CLICKHOUSE_DATABASE" diff --git a/tests/queries/0_stateless/02870_per_column_settings.reference b/tests/queries/0_stateless/02870_per_column_settings.reference new file mode 100644 index 00000000000..144c8c5ee2e --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_settings.reference @@ -0,0 +1,18 @@ +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +1000 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 8192, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String,\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` UInt64,\n `long_string` String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String,\n `v2` UInt64,\n `v3` Float32,\n `v4` Float64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/tab/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192 +--- +(0,0) 0 +(1,1) 1 +(2,2) 2 +(3,3) 3 +(4,4) 4 +(5,5) 5 +(6,6) 6 +(7,7) 7 +(8,8) 8 +(9,9) 9 +--- diff --git a/tests/queries/0_stateless/02870_per_column_settings.sql b/tests/queries/0_stateless/02870_per_column_settings.sql new file mode 100644 index 00000000000..345cf5cc744 --- /dev/null +++ b/tests/queries/0_stateless/02870_per_column_settings.sql @@ -0,0 +1,69 @@ +-- Tags: no-random-merge-tree-settings, no-replicated-database +-- Tag no-replicated-database: Old syntax is not allowed +-- The test use replicated table to test serialize and deserialize column with settings declaration on zookeeper +-- Tests column-level settings for MergeTree* tables + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab +( + id UInt64, + long_string String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840), + v1 String, + v2 UInt64, + v3 Float32, + v4 Float64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/tab/2870', 'r1') +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +SHOW CREATE tab; + +INSERT INTO TABLE tab SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000); +SELECT count() FROM tab; + +ALTER TABLE tab MODIFY COLUMN long_string MODIFY SETTING min_compress_block_size = 8192; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string RESET SETTING min_compress_block_size; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string REMOVE SETTINGS; +SHOW CREATE tab; + +ALTER TABLE tab MODIFY COLUMN long_string String SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840); +SHOW CREATE tab; + +DROP TABLE tab; + +SELECT '--- '; + +SET allow_experimental_object_type = 1; + +CREATE TABLE tab +( + id UInt64, + tup Tuple(UInt64, UInt64) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), + json JSON SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; + +INSERT INTO TABLE tab SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000); +SELECT tup, json.key AS key FROM tab ORDER BY key LIMIT 10; + +DROP TABLE tab; + +SELECT '--- '; + +-- Unsupported column-level settings are rejected +CREATE TABLE tab +( + id UInt64, + long_string String SETTINGS (min_block_size = 81920, max_compress_block_size = 163840), +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 1; -- {serverError UNKNOWN_SETTING} diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference index 29004fde9d4..c7f78cd74c2 100644 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.reference @@ -1,5 +1,7 @@ 1 02901_parallel_replicas_rollup-default Used parallel replicas: true +Distributed query with analyzer +1 0 0 0 6 2019 0 0 2 2019 1 0 2 diff --git a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh index 029b4d07ee2..def813c17b4 100755 --- a/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh +++ b/tests/queries/0_stateless/02901_parallel_replicas_rollup.sh @@ -39,6 +39,11 @@ $CLICKHOUSE_CLIENT \ ORDER BY max((SELECT 1 WHERE 0)); "; were_parallel_replicas_used $query_id + +# It was a bug in analyzer distributed header. +echo "Distributed query with analyzer" +$CLICKHOUSE_CLIENT --query "SELECT 1 FROM remote('127.0.0.{2,3}', currentDatabase(), nested) GROUP BY 1 WITH ROLLUP ORDER BY max((SELECT 1 WHERE 0))" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS nested" diff --git a/tests/queries/0_stateless/02932_kill_query_sleep.sh b/tests/queries/0_stateless/02932_kill_query_sleep.sh index 84e84204aa1..5ae47755722 100755 --- a/tests/queries/0_stateless/02932_kill_query_sleep.sh +++ b/tests/queries/0_stateless/02932_kill_query_sleep.sh @@ -10,7 +10,7 @@ function wait_query_started() local query_id="$1" timeout=60 start=$EPOCHSECONDS - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") == 0 ]]; do + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id' SETTINGS use_query_cache=0") == 0 ]]; do if ((EPOCHSECONDS-start > timeout )); then echo "Timeout while waiting for query $query_id to start" exit 1 @@ -26,7 +26,7 @@ function kill_query() $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id='$query_id'" >/dev/null timeout=60 start=$EPOCHSECONDS - while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id'") != 0 ]]; do + while [[ $($CLICKHOUSE_CLIENT --query="SELECT count() FROM system.processes WHERE query_id='$query_id' SETTINGS use_query_cache=0") != 0 ]]; do if ((EPOCHSECONDS-start > timeout )); then echo "Timeout while waiting for query $query_id to cancel" exit 1 @@ -49,4 +49,4 @@ echo "Cancelling query" kill_query "$sleep_query_id" $CLICKHOUSE_CLIENT --query "SYSTEM FLUSH LOGS;" -$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id' AND current_database = currentDatabase()" | grep -oF "QUERY_WAS_CANCELLED" +$CLICKHOUSE_CLIENT --query "SELECT exception FROM system.query_log WHERE query_id='$sleep_query_id' AND current_database = '$CLICKHOUSE_DATABASE'" | grep -oF "QUERY_WAS_CANCELLED"