From 5e4d3b2e43b04eb0bc0e44c082c86612127726e6 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 4 Jan 2024 00:05:57 +0000 Subject: [PATCH 01/45] Refreshable materialized views improvements --- .../system-tables/view_refreshes.md | 3 +- .../sql-reference/statements/create/view.md | 28 +++- docs/en/sql-reference/statements/system.md | 6 +- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 4 + src/Parsers/ASTRefreshStrategy.cpp | 3 +- src/Parsers/ASTRefreshStrategy.h | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Parsers/ParserRefreshStrategy.cpp | 9 ++ src/Parsers/ParserSystemQuery.cpp | 1 + src/Storages/MaterializedView/RefreshSet.h | 9 +- .../MaterializedView/RefreshSettings.h | 6 +- src/Storages/MaterializedView/RefreshTask.cpp | 144 +++++++++++------- src/Storages/MaterializedView/RefreshTask.h | 28 ++-- .../MaterializedView/RefreshTask_fwd.h | 2 - src/Storages/StorageMaterializedView.cpp | 75 +++++---- src/Storages/StorageMaterializedView.h | 4 +- .../System/StorageSystemViewRefreshes.cpp | 4 +- ...2_refreshable_materialized_views.reference | 33 +++- .../02932_refreshable_materialized_views.sh | 110 ++++++++++--- 20 files changed, 327 insertions(+), 145 deletions(-) diff --git a/docs/en/operations/system-tables/view_refreshes.md b/docs/en/operations/system-tables/view_refreshes.md index 12377507b39..e792e0d095d 100644 --- a/docs/en/operations/system-tables/view_refreshes.md +++ b/docs/en/operations/system-tables/view_refreshes.md @@ -17,7 +17,8 @@ Columns: - `duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — How long the last refresh attempt took. - `next_refresh_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time at which the next refresh is scheduled to start. - `remaining_dependencies` ([Array(String)](../../sql-reference/data-types/array.md)) — If the view has [refresh dependencies](../../sql-reference/statements/create/view.md#refresh-dependencies), this array contains the subset of those dependencies that are not satisfied for the current refresh yet. If `status = 'WaitingForDependencies'`, a refresh is ready to start as soon as these dependencies are fulfilled. -- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Exception'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace. +- `exception` ([String](../../sql-reference/data-types/string.md)) — if `last_refresh_result = 'Error'`, i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace. +- `retry` ([UInt64](../../sql-reference/data-types/int-uint.md)) — If nonzero, the current or next refresh is a retry (see `refresh_retries` refresh setting), and `retry` is the 1-based index of that retry. - `refresh_count` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of successful refreshes since last server restart or table creation. - `progress` ([Float64](../../sql-reference/data-types/float.md)) — Progress of the current refresh, between 0 and 1. - `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Number of rows read by the current refresh so far. diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 1fabb6d8cc7..3e0766794f1 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -159,6 +159,8 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name REFRESH EVERY|AFTER interval [OFFSET interval] RANDOMIZE FOR interval DEPENDS ON [db.]name [, [db.]name [, ...]] +SETTINGS name = value [, name = value [, ...]] +[APPEND] [TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY] AS SELECT ... ``` @@ -167,18 +169,23 @@ where `interval` is a sequence of simple intervals: number SECOND|MINUTE|HOUR|DAY|WEEK|MONTH|YEAR ``` -Periodically runs the corresponding query and stores its result in a table, atomically replacing the table's previous contents. +Periodically runs the corresponding query and stores its result in a table. + * If the query says `APPEND`, each refresh inserts rows into the table without deleting existing rows. The insert is not atomic, just like a regular INSERT SELECT. + * Otherwise each refresh atomically replaces the table's previous contents. Differences from regular non-refreshable materialized views: - * No insert trigger. I.e. when new data is inserted into the table specified in SELECT, it's *not* automatically pushed to the refreshable materialized view. The periodic refresh runs the entire query and replaces the entire table. + * No insert trigger. I.e. when new data is inserted into the table specified in SELECT, it's *not* automatically pushed to the refreshable materialized view. The periodic refresh runs the entire query. * No restrictions on the SELECT query. Table functions (e.g. `url()`), views, UNION, JOIN, are all allowed. +:::note +The settings in the The settings in `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. +::: + :::note Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations: * not compatible with Replicated database or table engines * It is not supported in ClickHouse Cloud * require [Atomic database engine](../../../engines/database-engines/atomic.md), - * no retries for failed refresh - we just skip to the next scheduled refresh time, * no limit on number of concurrent refreshes. ::: @@ -243,15 +250,22 @@ A few more examples: `DEPENDS ON` only works between refreshable materialized views. Listing a regular table in the `DEPENDS ON` list will prevent the view from ever refreshing (dependencies can be removed with `ALTER`, see below). ::: +### Settings + +Available refresh settings: + * `refresh_retries` - How many times to retry if refresh query fails with an exception. If all retries fail, skip to the next scheduled refresh time. 0 means no retries, -1 means infinite retries. Default: 0. + * `refresh_retry_initial_backoff_ms` - Delay before the first retry, if `refresh_retries` is not zero. Each subsequent retry doubles the delay, up to `refresh_retry_max_backoff_ms`. Default: 100 ms. + * `refresh_retry_max_backoff_ms` - Limit on the exponential growth of delay between refresh attempts. Default: 60000 ms (1 minute). + ### Changing Refresh Parameters {#changing-refresh-parameters} To change refresh parameters: ``` -ALTER TABLE [db.]name MODIFY REFRESH EVERY|AFTER ... [RANDOMIZE FOR ...] [DEPENDS ON ...] +ALTER TABLE [db.]name MODIFY REFRESH EVERY|AFTER ... [RANDOMIZE FOR ...] [DEPENDS ON ...] [SETTINGS ...] ``` :::note -This replaces refresh schedule *and* dependencies. If the table had a `DEPENDS ON`, doing a `MODIFY REFRESH` without `DEPENDS ON` will remove the dependencies. +This replaces *all* refresh parameters at once: schedule, dependencies, settings, and APPEND-ness. E.g. if the table had a `DEPENDS ON`, doing a `MODIFY REFRESH` without `DEPENDS ON` will remove the dependencies. ::: ### Other operations @@ -260,6 +274,10 @@ The status of all refreshable materialized views is available in table [`system. To manually stop, start, trigger, or cancel refreshes use [`SYSTEM STOP|START|REFRESH|CANCEL VIEW`](../system.md#refreshable-materialized-views). +:::note +Fun fact: the refresh query is allowed to read from the view that's being refreshed, seeing pre-refresh version of the data. This means you can implement Conway's game of life: https://pastila.nl/?00021a4b/d6156ff819c83d490ad2dcec05676865#O0LGWTO7maUQIA4AcGUtlA== +::: + ## Window View [Experimental] :::info diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index e6d3439d2b9..7b8d2339516 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -389,7 +389,7 @@ SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_ After running this statement the `[db.]replicated_merge_tree_family_table_name` fetches commands from the common replicated log into its own replication queue, and then the query waits till the replica processes all of the fetched commands. The following modifiers are supported: - If a `STRICT` modifier was specified then the query waits for the replication queue to become empty. The `STRICT` version may never succeed if new entries constantly appear in the replication queue. - - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. + - If a `LIGHTWEIGHT` modifier was specified then the query waits only for `GET_PART`, `ATTACH_PART`, `DROP_RANGE`, `REPLACE_RANGE` and `DROP_PART` entries to be processed. Additionally, the LIGHTWEIGHT modifier supports an optional FROM 'srcReplicas' clause, where 'srcReplicas' is a comma-separated list of source replica names. This extension allows for more targeted synchronization by focusing only on replication tasks originating from the specified source replicas. - If a `PULL` modifier was specified then the query pulls new replication queue entries from ZooKeeper, but does not wait for anything to be processed. @@ -515,6 +515,10 @@ Trigger an immediate out-of-schedule refresh of a given view. SYSTEM REFRESH VIEW [db.]name ``` +### REFRESH VIEW + +Wait for the currently running refresh to complete. If the refresh fails, throws an exception. If no refresh is running, completes immediately, throwing an exception if previous refresh failed. + ### STOP VIEW, STOP VIEWS Disable periodic refreshing of the given view or all refreshable views. If a refresh is in progress, cancel it too. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b1b8e2367a4..38fed9f3198 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -604,6 +604,7 @@ M(723, PARQUET_EXCEPTION) \ M(724, TOO_MANY_TABLES) \ M(725, TOO_MANY_DATABASES) \ + M(726, REFRESH_FAILED) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d3526941b33..111817729e2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -664,6 +664,9 @@ BlockIO InterpreterSystemQuery::execute() case Type::REFRESH_VIEW: getRefreshTask()->run(); break; + case Type::WAIT_VIEW: + getRefreshTask()->wait(); + break; case Type::CANCEL_VIEW: getRefreshTask()->cancel(); break; @@ -1411,6 +1414,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() break; } case Type::REFRESH_VIEW: + case Type::WAIT_VIEW: case Type::START_VIEW: case Type::START_VIEWS: case Type::STOP_VIEW: diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp index 2e0c6ee4638..d10c1b4e7f5 100644 --- a/src/Parsers/ASTRefreshStrategy.cpp +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -20,7 +20,6 @@ ASTPtr ASTRefreshStrategy::clone() const res->set(res->settings, settings->clone()); if (dependencies) res->set(res->dependencies, dependencies->clone()); - res->schedule_kind = schedule_kind; return res; } @@ -66,6 +65,8 @@ void ASTRefreshStrategy::formatImpl( f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS " << (f_settings.hilite ? hilite_none : ""); settings->formatImpl(f_settings, state, frame); } + if (append) + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " APPEND" << (f_settings.hilite ? hilite_none : ""); } } diff --git a/src/Parsers/ASTRefreshStrategy.h b/src/Parsers/ASTRefreshStrategy.h index ca248b76b40..bb5ac97c054 100644 --- a/src/Parsers/ASTRefreshStrategy.h +++ b/src/Parsers/ASTRefreshStrategy.h @@ -24,6 +24,7 @@ public: ASTTimeInterval * offset = nullptr; ASTTimeInterval * spread = nullptr; RefreshScheduleKind schedule_kind{RefreshScheduleKind::UNKNOWN}; + bool append = false; String getID(char) const override { return "Refresh strategy definition"; } diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 167e724dcee..59de90b1d8e 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -95,6 +95,7 @@ public: START_CLEANUP, RESET_COVERAGE, REFRESH_VIEW, + WAIT_VIEW, START_VIEW, START_VIEWS, STOP_VIEW, diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index e7912293d85..8d19312996a 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -95,7 +96,15 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserSetQuery{true}.parse(pos, settings, expected)) return false; refresh->set(refresh->settings, settings); + + /// Validate. + RefreshSettings parsed_settings; + parsed_settings.applyChanges(refresh->settings->changes); } + + if (ParserKeyword{"APPEND"}.ignore(pos, expected)) + refresh->append = true; + node = refresh; return true; } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 0545c3e5568..4b89b3817f0 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -421,6 +421,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & break; case Type::REFRESH_VIEW: + case Type::WAIT_VIEW: case Type::START_VIEW: case Type::STOP_VIEW: case Type::CANCEL_VIEW: diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index eff445023a6..c5ec227bd92 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -11,7 +11,7 @@ namespace DB using DatabaseAndTableNameSet = std::unordered_set; -enum class RefreshState : RefreshTaskStateUnderlying +enum class RefreshState { Disabled = 0, Scheduled, @@ -19,11 +19,11 @@ enum class RefreshState : RefreshTaskStateUnderlying Running, }; -enum class LastRefreshResult : RefreshTaskStateUnderlying +enum class LastRefreshResult { Unknown = 0, Cancelled, - Exception, + Error, Finished }; @@ -37,7 +37,8 @@ struct RefreshInfo UInt64 last_attempt_duration_ms = 0; UInt32 next_refresh_time = 0; UInt64 refresh_count = 0; - String exception_message; // if last_refresh_result is Exception + UInt64 retry = 0; + String exception_message; // if last_refresh_result is Error std::vector remaining_dependencies; ProgressValues progress; }; diff --git a/src/Storages/MaterializedView/RefreshSettings.h b/src/Storages/MaterializedView/RefreshSettings.h index 814c7e52b32..23676538788 100644 --- a/src/Storages/MaterializedView/RefreshSettings.h +++ b/src/Storages/MaterializedView/RefreshSettings.h @@ -6,8 +6,10 @@ namespace DB { #define LIST_OF_REFRESH_SETTINGS(M, ALIAS) \ - /// TODO: Add settings - /// M(UInt64, name, 42, "...", 0) + M(Int64, refresh_retries, 0, "How many times to retry refresh query if it fails. If all attempts fail, wait for the next refresh time according to schedule. 0 to disable retries. -1 for infinite retries.", 0) \ + M(UInt64, refresh_retry_initial_backoff_ms, 100, "Delay before the first retry if refresh query fails (if refresh_retries setting is not zero). Each subsequent retry doubles the delay, up to refresh_retry_max_backoff_ms.", 0) \ + M(UInt64, refresh_retry_max_backoff_ms, 60'000, "Limit on the exponential growth of delay between refresh attempts, if they keep failing and refresh_retries is positive.", 0) \ + DECLARE_SETTINGS_TRAITS(RefreshSettingsTraits, LIST_OF_REFRESH_SETTINGS) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index bc8cb0ce69a..83930ed7be9 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -23,41 +23,42 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int QUERY_WAS_CANCELLED; + extern const int REFRESH_FAILED; } RefreshTask::RefreshTask( - const ASTRefreshStrategy & strategy) + StorageMaterializedView * view_, const DB::ASTRefreshStrategy & strategy) : log(getLogger("RefreshTask")) + , view(view_) , refresh_schedule(strategy) -{} + , refresh_append(strategy.append) +{ + if (strategy.settings != nullptr) + refresh_settings.applyChanges(strategy.settings->changes); +} RefreshTaskHolder RefreshTask::create( - const StorageMaterializedView & view, + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) { - auto task = std::make_shared(strategy); + auto task = std::make_shared(view, strategy); - task->refresh_task = context->getSchedulePool().createTask("MaterializedViewRefresherTask", - [self = task->weak_from_this()] - { - if (auto t = self.lock()) - t->refreshTask(); - }); + task->refresh_task = context->getSchedulePool().createTask("RefreshTask", + [self = task.get()] { self->refreshTask(); }); std::vector deps; if (strategy.dependencies) for (auto && dependency : strategy.dependencies->children) deps.emplace_back(dependency->as()); - context->getRefreshSet().emplace(view.getStorageID(), deps, task); + context->getRefreshSet().emplace(view->getStorageID(), deps, task); return task; } -void RefreshTask::initializeAndStart(std::shared_ptr view) +void RefreshTask::initializeAndStart() { - view_to_refresh = view; if (view->getContext()->getSettingsRef().stop_refreshable_materialized_views_on_startup) stop_requested = true; populateDependencies(); @@ -103,7 +104,11 @@ void RefreshTask::alterRefreshParams(const DB::ASTRefreshStrategy & new_strategy if (arriveDependency(id) && !std::exchange(refresh_immediately, true)) refresh_task->schedule(); - /// TODO: Update settings once we have them. + refresh_settings = {}; + if (new_strategy.settings != nullptr) + refresh_settings.applyChanges(new_strategy.settings->changes); + + refresh_append = new_strategy.append; } RefreshInfo RefreshTask::getInfo() const @@ -112,7 +117,7 @@ RefreshInfo RefreshTask::getInfo() const auto res = info; res.view_id = set_handle.getID(); res.remaining_dependencies.assign(remaining_dependencies.begin(), remaining_dependencies.end()); - if (res.last_refresh_result != LastRefreshResult::Exception) + if (res.last_refresh_result != LastRefreshResult::Error) res.exception_message.clear(); res.progress = progress.getValues(); return res; @@ -140,6 +145,8 @@ void RefreshTask::run() std::lock_guard guard(mutex); if (std::exchange(refresh_immediately, true)) return; + next_refresh_prescribed = std::chrono::floor(currentTime()); + next_refresh_actual = currentTime(); refresh_task->schedule(); } @@ -150,6 +157,14 @@ void RefreshTask::cancel() refresh_task->schedule(); } +void RefreshTask::wait() +{ + std::unique_lock lock(mutex); + refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running; }); + if (info.last_refresh_result == LastRefreshResult::Error) + throw Exception(ErrorCodes::REFRESH_FAILED, "Refresh failed: {}", info.exception_message); +} + void RefreshTask::shutdown() { { @@ -167,6 +182,8 @@ void RefreshTask::shutdown() /// (Also, RefreshSet holds a shared_ptr to us.) std::lock_guard guard(mutex); set_handle.reset(); + + view = nullptr; } void RefreshTask::notify(const StorageID & parent_id, std::chrono::sys_seconds parent_next_prescribed_time) @@ -233,6 +250,7 @@ void RefreshTask::refreshTask() chassert(lock.owns_lock()); interrupt_execution.store(false); + refresh_cv.notify_all(); // we'll assign info.state before unlocking the mutex if (stop_requested) { @@ -244,7 +262,7 @@ void RefreshTask::refreshTask() if (!refresh_immediately) { auto now = currentTime(); - if (now >= next_refresh_with_spread) + if (now >= next_refresh_actual) { if (arriveTime()) refresh_immediately = true; @@ -257,7 +275,7 @@ void RefreshTask::refreshTask() else { size_t delay_ms = std::chrono::duration_cast( - next_refresh_with_spread - now).count(); + next_refresh_actual - now).count(); /// If we're in a test that fakes the clock, poll every 100ms. if (fake_clock.load(std::memory_order_relaxed) != INT64_MIN) @@ -271,19 +289,9 @@ void RefreshTask::refreshTask() /// Perform a refresh. + bool append = refresh_append; refresh_immediately = false; - - auto view = lockView(); - if (!view) - { - /// The view was dropped. This RefreshTask should be destroyed soon too. - /// (Maybe this is unreachable.) - info.state = RefreshState::Disabled; - break; - } - info.state = RefreshState::Running; - CurrentMetrics::Increment metric_inc(CurrentMetrics::RefreshingViews); lock.unlock(); @@ -294,7 +302,7 @@ void RefreshTask::refreshTask() try { - executeRefreshUnlocked(view); + executeRefreshUnlocked(append); refreshed = true; } catch (...) @@ -318,18 +326,16 @@ void RefreshTask::refreshTask() if (exception) { - info.last_refresh_result = LastRefreshResult::Exception; + info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - - /// TODO: Do a few retries with exponential backoff. - advanceNextRefreshTime(now); + scheduleRetryOrSkipToNextRefresh(now); } else if (!refreshed) { info.last_refresh_result = LastRefreshResult::Cancelled; /// Make sure we don't just start another refresh immediately. - if (!stop_requested && now >= next_refresh_with_spread) + if (!stop_requested) advanceNextRefreshTime(now); } else @@ -362,17 +368,18 @@ void RefreshTask::refreshTask() } } -void RefreshTask::executeRefreshUnlocked(std::shared_ptr view) +void RefreshTask::executeRefreshUnlocked(bool append) { LOG_DEBUG(log, "Refreshing view {}", view->getStorageID().getFullTableName()); progress.reset(); - /// Create a table. - auto [refresh_context, refresh_query] = view->prepareRefresh(); - - StorageID stale_table = StorageID::createEmpty(); + ContextMutablePtr refresh_context; + std::optional table_to_drop; try { + /// Create a table. + auto refresh_query = view->prepareRefresh(append, refresh_context, table_to_drop); + /// Run the query. { CurrentThread::QueryScope query_scope(refresh_context); // create a thread group for the query @@ -424,37 +431,67 @@ void RefreshTask::executeRefreshUnlocked(std::shared_ptrexchangeTargetTable(refresh_query->table_id, refresh_context); + if (!append) + table_to_drop = view->exchangeTargetTable(refresh_query->table_id, refresh_context); } catch (...) { - try + if (table_to_drop.has_value()) { - InterpreterDropQuery::executeDropQuery( - ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, refresh_query->table_id, /*sync*/ false, /*ignore_sync_setting*/ true); - } - catch (...) - { - tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); - /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. + try + { + InterpreterDropQuery::executeDropQuery( + ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ false, /*ignore_sync_setting*/ true); + } + catch (...) + { + tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); + /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. + } } throw; } /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, stale_table, /*sync*/ true, /*ignore_sync_setting*/ true); + if (table_to_drop.has_value()) + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ true, /*ignore_sync_setting*/ true); } void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point now) { std::chrono::sys_seconds next = refresh_schedule.prescribeNext(next_refresh_prescribed, now); next_refresh_prescribed = next; - next_refresh_with_spread = refresh_schedule.addRandomSpread(next); + next_refresh_actual = refresh_schedule.addRandomSpread(next); - auto secs = std::chrono::floor(next_refresh_with_spread); + num_retries = 0; + info.retry = num_retries; + + auto secs = std::chrono::floor(next_refresh_actual); info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } +void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +{ + if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) + { + advanceNextRefreshTime(now); + return; + } + + num_retries += 1; + info.retry = num_retries; + + UInt64 delay_ms; + UInt64 multiplier = UInt64(1) << std::min(num_retries - 1, Int64(62)); + /// Overflow check: a*b <= c iff a <= c/b iff a <= floor(c/b). + if (refresh_settings.refresh_retry_initial_backoff_ms <= refresh_settings.refresh_retry_max_backoff_ms / multiplier) + delay_ms = refresh_settings.refresh_retry_initial_backoff_ms * multiplier; + else + delay_ms = refresh_settings.refresh_retry_max_backoff_ms; + + next_refresh_actual = now + std::chrono::milliseconds(delay_ms); +} + bool RefreshTask::arriveDependency(const StorageID & parent) { remaining_dependencies.erase(parent); @@ -495,11 +532,6 @@ void RefreshTask::interruptExecution() } } -std::shared_ptr RefreshTask::lockView() -{ - return std::static_pointer_cast(view_to_refresh.lock()); -} - std::chrono::system_clock::time_point RefreshTask::currentTime() const { Int64 fake = fake_clock.load(std::memory_order::relaxed); diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 1f050a97cd9..9281c0469a3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -22,15 +22,15 @@ class RefreshTask : public std::enable_shared_from_this { public: /// Never call it manually, public for shared_ptr construction only - explicit RefreshTask(const ASTRefreshStrategy & strategy); + RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task static RefreshTaskHolder create( - const StorageMaterializedView & view, + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); - void initializeAndStart(std::shared_ptr view); + void initializeAndStart(); /// Call when renaming the materialized view. void rename(StorageID new_id); @@ -52,6 +52,11 @@ public: /// Cancel task execution void cancel(); + /// Waits for the currently running refresh attempt to complete. + /// If the refresh fails, throws an exception. + /// If no refresh is running, completes immediately, throwing an exception if previous refresh failed. + void wait(); + /// Permanently disable task scheduling and remove this table from RefreshSet. void shutdown(); @@ -66,7 +71,7 @@ public: private: LoggerPtr log = nullptr; - std::weak_ptr view_to_refresh; + StorageMaterializedView * view; /// Protects interrupt_execution and running_executor. /// Can be locked while holding `mutex`. @@ -83,7 +88,9 @@ private: mutable std::mutex mutex; RefreshSchedule refresh_schedule; - RefreshSettings refresh_settings; // TODO: populate, use, update on alter + RefreshSettings refresh_settings; + bool refresh_append; + RefreshSet::Handle set_handle; /// StorageIDs of our dependencies that we're waiting for. @@ -111,7 +118,8 @@ private: /// E.g. for REFRESH EVERY 1 DAY, yesterday's refresh of the dependency shouldn't trigger today's /// refresh of the dependent even if it happened today (e.g. it was slow or had random spread > 1 day). std::chrono::sys_seconds next_refresh_prescribed; - std::chrono::system_clock::time_point next_refresh_with_spread; + std::chrono::system_clock::time_point next_refresh_actual; + Int64 num_retries = 0; /// Calls refreshTask() from background thread. BackgroundSchedulePool::TaskHolder refresh_task; @@ -122,6 +130,7 @@ private: /// Just for observability. RefreshInfo info; Progress progress; + std::condition_variable refresh_cv; // notified when info.state changes /// The main loop of the refresh task. It examines the state, sees what needs to be /// done and does it. If there's nothing to do at the moment, returns; it's then scheduled again, @@ -133,11 +142,14 @@ private: /// Perform an actual refresh: create new table, run INSERT SELECT, exchange tables, drop old table. /// Mutex must be unlocked. Called only from refresh_task. - void executeRefreshUnlocked(std::shared_ptr view); + void executeRefreshUnlocked(bool append); /// Assigns next_refresh_* void advanceNextRefreshTime(std::chrono::system_clock::time_point now); + /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). + void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); bool arriveTime(); @@ -145,8 +157,6 @@ private: void interruptExecution(); - std::shared_ptr lockView(); - std::chrono::system_clock::time_point currentTime() const; }; diff --git a/src/Storages/MaterializedView/RefreshTask_fwd.h b/src/Storages/MaterializedView/RefreshTask_fwd.h index 1f366962eb6..9e389316c64 100644 --- a/src/Storages/MaterializedView/RefreshTask_fwd.h +++ b/src/Storages/MaterializedView/RefreshTask_fwd.h @@ -8,8 +8,6 @@ namespace DB class RefreshTask; -using RefreshTaskStateUnderlying = UInt8; using RefreshTaskHolder = std::shared_ptr; -using RefreshTaskObserver = std::weak_ptr; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 316f398b476..500312fd245 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -3,6 +3,8 @@ #include #include +#include +#include #include #include @@ -14,6 +16,7 @@ #include #include #include +#include #include #include @@ -141,6 +144,13 @@ StorageMaterializedView::StorageMaterializedView( if (point_to_itself_by_uuid || point_to_itself_by_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Materialized view {} cannot point to itself", table_id_.getFullTableName()); + if (query.refresh_strategy) + { + fixed_uuid = false; + refresher = RefreshTask::create(this, getContext(), *query.refresh_strategy); + refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; + } + if (!has_inner_table) { target_table_id = query.to_table_id; @@ -190,16 +200,6 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->getDatabase(), manual_create_query->getTable()}, getContext())->getStorageID(); } - - if (query.refresh_strategy) - { - fixed_uuid = false; - refresher = RefreshTask::create( - *this, - getContext(), - *query.refresh_strategy); - refresh_on_start = mode < LoadingStrictnessLevel::ATTACH && !query.is_create_empty; - } } QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( @@ -370,40 +370,57 @@ bool StorageMaterializedView::optimize( return storage_ptr->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); } -std::tuple> StorageMaterializedView::prepareRefresh() const +std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const { auto refresh_context = getInMemoryMetadataPtr()->getSQLSecurityOverriddenContext(getContext()); + out_context = refresh_context; /// Generate a random query id. refresh_context->setCurrentQueryId(""); CurrentThread::QueryScope query_scope(refresh_context); auto inner_table_id = getTargetTableId(); - auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); - auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + StorageID target_table = inner_table_id; - auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); - auto & create_query = create_table_query->as(); - create_query.setTable(new_table_name); - create_query.setDatabase(db->getDatabaseName()); - create_query.create_or_replace = true; - create_query.replace_table = true; - create_query.uuid = UUIDHelpers::Nil; + if (!append) + { + auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); - InterpreterCreateQuery create_interpreter(create_table_query, refresh_context); - create_interpreter.setInternal(true); - create_interpreter.execute(); + auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); + auto & create_query = create_table_query->as(); + create_query.setTable(new_table_name); + create_query.setDatabase(db->getDatabaseName()); + create_query.create_or_replace = true; + create_query.replace_table = true; + create_query.uuid = UUIDHelpers::Nil; - StorageID fresh_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + InterpreterCreateQuery create_interpreter(create_table_query, refresh_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + + target_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + out_temp_table_id = target_table; + } auto insert_query = std::make_shared(); insert_query->select = getInMemoryMetadataPtr()->getSelectQuery().select_query; - insert_query->setTable(fresh_table.table_name); - insert_query->setDatabase(fresh_table.database_name); - insert_query->table_id = fresh_table; + insert_query->setTable(target_table.table_name); + insert_query->setDatabase(target_table.database_name); + insert_query->table_id = target_table; - return {refresh_context, insert_query}; + Block header; + if (refresh_context->getSettingsRef().allow_experimental_analyzer) + header = InterpreterSelectQueryAnalyzer::getSampleBlock(insert_query->select, refresh_context); + else + header = InterpreterSelectWithUnionQuery(insert_query->select, refresh_context, SelectQueryOptions()).getSampleBlock(); + + auto columns = std::make_shared(','); + for (const String & name : header.getNames()) + columns->children.push_back(std::make_shared(name)); + insert_query->columns = std::move(columns); + + return insert_query; } StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context) @@ -569,7 +586,7 @@ void StorageMaterializedView::startup() if (refresher) { - refresher->initializeAndStart(std::static_pointer_cast(shared_from_this())); + refresher->initializeAndStart(); if (refresh_on_start) refresher->run(); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 5ecd2ec3819..12c0dc651e9 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -121,7 +121,9 @@ private: /// Prepare to refresh a refreshable materialized view: create query context, create temporary /// table, form the insert-select query. - std::tuple> prepareRefresh() const; + /// The output arguments may be assigned before throwing an exception, in which case the caller + /// must drop the temp table before rethrowing. + std::shared_ptr prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const; StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); void setTargetTableId(StorageID id); diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 30539ed6b6a..95c89bd0214 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -32,8 +32,9 @@ ColumnsDescription StorageSystemViewRefreshes::getColumnsDescription() "If status = 'WaitingForDependencies', a refresh is ready to start as soon as these dependencies are fulfilled." }, {"exception", std::make_shared(), - "if last_refresh_result = 'Exception', i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace." + "if last_refresh_result = 'Error', i.e. the last refresh attempt failed, this column contains the corresponding error message and stack trace." }, + {"retry", std::make_shared(), "How many failed attempts there were so far, for the current refresh."}, {"refresh_count", std::make_shared(), "Number of successful refreshes since last server restart or table creation."}, {"progress", std::make_shared(), "Progress of the current refresh, between 0 and 1."}, {"elapsed", std::make_shared(), "The amount of nanoseconds the current refresh took."}, @@ -85,6 +86,7 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(Array(deps)); res_columns[i++]->insert(refresh.exception_message); + res_columns[i++]->insert(refresh.retry); res_columns[i++]->insert(refresh.refresh_count); res_columns[i++]->insert(Float64(refresh.progress.read_rows) / refresh.progress.total_rows_to_read); res_columns[i++]->insert(refresh.progress.elapsed_ns / 1e9); diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 2eb41590af1..8d4878f4e3f 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -3,21 +3,25 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n) <2: refreshed> 3 1 1 <3: time difference at least> 1000 <4: next refresh in> 2 +<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a +<7.5: created dependent> 2052-11-11 11:11:11 <8: refreshed> 20 <9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 <9: refreshed> b Scheduled Finished 2054-01-01 00:00:00 -<10: waiting> a Scheduled [] 2054-01-01 00:00:00 -<10: waiting> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 +<9.2: dropping> 0 2 +<9.4: dropped> 0 2 +<10: creating> a Scheduled [] 2054-01-01 00:00:00 +<10: creating> b WaitingForDependencies ['default.a'] 2054-01-01 00:00:00 <11: chain-refreshed a> 4 <12: chain-refreshed b> 40 -<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 -<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 +<13: chain-refreshed> a Scheduled [] Finished 2054-01-01 00:00:01 2056-01-01 00:00:00 1 +<13: chain-refreshed> b Scheduled ['default.a'] Finished 2054-01-24 23:22:21 2056-01-01 00:00:00 1 <14: waiting for next cycle> a Scheduled [] 2058-01-01 00:00:00 <14: waiting for next cycle> b WaitingForDependencies ['default.a'] 2060-01-01 00:00:00 <15: chain-refreshed a> 6 @@ -34,11 +38,24 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <24: rename during refresh> 1 <25: rename during refresh> f Running <27: cancelled> f Scheduled -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 +<28: drop during refresh> 0 0 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 <31: to existing table> 10 <31: to existing table> 20 -<32: empty> i Scheduled Unknown -<32: empty> j Scheduled Finished +<31.5: will retry> Error 1 +<31.6: did retry> 10 +<32: empty> i Scheduled Unknown 0 +<32: empty> j Scheduled Finished 0 +<34: append> 10 +<35: append> 10 +<35: append> 20 +<35: append> 30 +<36: not append> 20 +<36: not append> 30 +<37: append chain> 100 +<38: append chain> 100 +<38: append chain> 100 +<38: append chain> 200 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 89942e25b67..ad6fa3043a1 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -21,9 +21,9 @@ $CLICKHOUSE_CLIENT -nq " refresh after 2 second engine Memory empty - as select number as x from numbers(2) union all select rand64() as x" -$CLICKHOUSE_CLIENT -nq "select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes"; -$CLICKHOUSE_CLIENT -nq "show create a" + as select number as x from numbers(2) union all select rand64() as x; + select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; + show create a;" # Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] do @@ -56,16 +56,15 @@ $CLICKHOUSE_CLIENT -nq " # Create a source table from which views will read. $CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1" + create table src (x Int8) engine Memory as select 1;" # Switch to fake clock, change refresh schedule, change query. $CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " + system test view a set fake time '2050-01-01 00:00:01'; + system wait view a; + system refresh view a; + system wait view a; + select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; alter table a modify refresh every 2 year; alter table a modify query select x*2 as x from src; select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; @@ -87,11 +86,9 @@ $CLICKHOUSE_CLIENT -nq " create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; show create b; system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2052-11-11 11:11:11' ] -do - sleep 0.1 -done + system refresh view b; + system wait view b; + select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" # Next refresh shouldn't start until the dependency refreshes. $CLICKHOUSE_CLIENT -nq " select '<8: refreshed>', * from b; @@ -101,11 +98,20 @@ while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshe do sleep 0.1 done -# Update source table (by dropping and re-creating it - to test that tables are looked up by name -# rather than uuid), kick off refresh of the dependency. + +# Drop the source table, check that refresh fails and doesn't leave a temp table behind. $CLICKHOUSE_CLIENT -nq " - select '<10: waiting>', view, status, remaining_dependencies, next_refresh_time from refreshes; + select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); drop table src; + system refresh view a;" +$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" + +# Create the source table again, check that refresh succeeds (in particular that tables are looked +# up by name rather than uuid). +$CLICKHOUSE_CLIENT -nq " + select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; create table src (x Int16) engine Memory as select 2; system test view a set fake time '2054-01-01 00:00:01';" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] @@ -116,7 +122,7 @@ done $CLICKHOUSE_CLIENT -nq " select '<11: chain-refreshed a>', * from a; select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception from refreshes;" + select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" # Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to # catch up to the same cycle. @@ -172,7 +178,7 @@ $CLICKHOUSE_CLIENT -nq " drop table b; create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; drop table src;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Exception' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Error' ] do sleep 0.1 done @@ -246,7 +252,8 @@ done # Drop. $CLICKHOUSE_CLIENT -nq " drop table f; - select '<28: drop during refresh>', view, status from refreshes;" + select '<28: drop during refresh>', view, status from refreshes; + select '<28: drop during refresh>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase()" # Try OFFSET and RANDOMIZE FOR. $CLICKHOUSE_CLIENT -nq " @@ -283,21 +290,74 @@ done $CLICKHOUSE_CLIENT -nq " select '<31: to existing table>', * from dest; drop table dest; - drop table src; drop table h;" +# Retries. +$CLICKHOUSE_CLIENT -nq " + create materialized view h2 refresh after 1 year settings refresh_retries = 10 (x Int64) engine Memory as select x*10 + throwIf(x % 2 == 0) as x from src;" +$CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; + truncate table src; + insert into src values (1);" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] +do + sleep 0.1 +done +$CLICKHOUSE_CLIENT -nq " + select '<31.6: did retry>', x from h2; + drop table h2" + # EMPTY $CLICKHOUSE_CLIENT -nq " create materialized view i refresh after 1 year engine Memory empty as select number as x from numbers(2); - create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2)" + create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2);" while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] do sleep 0.1 done $CLICKHOUSE_CLIENT -nq " - select '<32: empty>', view, status, last_refresh_result from refreshes order by view; + select '<32: empty>', view, status, last_refresh_result, retry from refreshes order by view; drop table i; - drop table j" + drop table j;" + +# APPEND +$CLICKHOUSE_CLIENT -nq " + create materialized view k refresh every 10 year append (x Int64) engine Memory empty as select x*10 as x from src; + select '<33: append>', * from k; + system refresh view k; + system wait view k; + select '<34: append>', * from k; + truncate table src; + insert into src values (2), (3); + system refresh view k; + system wait view k; + select '<35: append>', * from k order by x;" +# ALTER to non-APPEND +$CLICKHOUSE_CLIENT -nq " + alter table k modify refresh every 10 year; + system refresh view k; + system wait view k; + select '<36: not append>', * from k order by x; + drop table k; + truncate table src;" + +# APPEND + TO + regular materialized view reading from it. +$CLICKHOUSE_CLIENT -nq " + create table mid (x Int64) engine MergeTree order by x; + create materialized view l refresh every 10 year append to mid empty as select x*10 as x from src; + create materialized view m (x Int64) engine Memory as select x*10 as x from mid; + insert into src values (1); + system refresh view l; + system wait view l; + select '<37: append chain>', * from m; + insert into src values (2); + system refresh view l; + system wait view l; + select '<38: append chain>', * from m order by x; + drop table l; + drop table m; + drop table mid;" $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From 361aa4ffadcad1f68c9157bd44a835ac828de1d4 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 26 Jan 2024 23:03:31 +0000 Subject: [PATCH 02/45] Fix build --- src/Parsers/ParserRefreshStrategy.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index 8d19312996a..de7eea93614 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB { @@ -96,10 +95,6 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!ParserSetQuery{true}.parse(pos, settings, expected)) return false; refresh->set(refresh->settings, settings); - - /// Validate. - RefreshSettings parsed_settings; - parsed_settings.applyChanges(refresh->settings->changes); } if (ParserKeyword{"APPEND"}.ignore(pos, expected)) From dcf783c0edad02c45ede10a981465c67d6991c45 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 1 Feb 2024 22:42:30 +0000 Subject: [PATCH 03/45] Check column names when creating table, fix a thing, move code around a little --- src/Interpreters/InterpreterCreateQuery.cpp | 148 ++++++++++++------ src/Interpreters/InterpreterCreateQuery.h | 2 + src/Parsers/ASTCreateQuery.cpp | 2 + src/Parsers/ASTRenameQuery.h | 13 ++ src/Storages/IStorage.h | 2 +- src/Storages/MaterializedView/RefreshTask.cpp | 25 ++- src/Storages/MaterializedView/RefreshTask.h | 24 ++- src/Storages/StorageMaterializedView.cpp | 76 +++++---- src/Storages/StorageMaterializedView.h | 15 +- .../02932_refreshable_materialized_views.sh | 9 +- 10 files changed, 208 insertions(+), 108 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 7272e10b801..ee57c376b31 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -120,6 +120,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_TABLES; extern const int TOO_MANY_DATABASES; + extern const int THERE_IS_NO_COLUMN; } namespace fs = std::filesystem; @@ -852,6 +853,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti } properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList()); + properties.columns_inferred_from_select_query = true; } else if (create.as_table_function) { @@ -939,6 +941,99 @@ void validateVirtualColumns(const IStorage & storage) } } +void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database) +{ + /// This is not strict validation, just catches common errors that would make the view not work. + /// It's possible to circumvent these checks by ALTERing the view or target table after creation. + + String table_engine; + if (create.storage && create.storage->engine) + table_engine = create.storage->engine->name; + + NamesAndTypesList all_output_columns; + bool check_columns = false; + if (create.to_table_id) + { + if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + )) + { + all_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock().getNamesAndTypesList(); + check_columns = true; + table_engine = to_table->getName(); + } + } + else if (!properties.columns_inferred_from_select_query) + { + all_output_columns = properties.columns.getInsertable(); + check_columns = true; + } + + if (create.refresh_strategy && !create.refresh_strategy->append) + { + if (database && database->getEngineName() != "Atomic") + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Refreshable materialized views (except with APPEND) only support Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName()); + } + + if (check_columns) + { + Block input_block; + + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } + + std::unordered_map output_types; + for (const NameAndTypePair & nt : all_output_columns) + output_types[nt.name] = nt.type; + + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + auto it = output_types.find(input_column.name); + if (it != output_types.end()) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); + } + else if (create.refresh_strategy) + { + /// Unrecognized columns produced by SELECT query are allowed for regular materialized + /// views, but not for refreshable ones. This is in part because it was easier to + /// implement, in part because refreshable views have less concern about ALTERing target + /// tables. + /// + /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. + /// Suppose the user removes a column from target table, then a minute later + /// correspondingly updates the view's query to not produce that column. + /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the + /// source table would fail - unacceptable. + /// For refreshable views, during that minute refreshes will fail - acceptable. + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); + } + } + + if (input_columns.empty()) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "None of the columns produced by the SELECT query are present in the target table. Use 'AS' to assign aliases that match column names."); + + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + } +} + namespace { void checkTemporaryTableEngineName(const String & name) @@ -1059,13 +1154,6 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data throw Exception(ErrorCodes::LOGICAL_ERROR, "Table UUID is not specified in DDL log"); } - if (create.refresh_strategy && database->getEngineName() != "Atomic") - throw Exception(ErrorCodes::INCORRECT_QUERY, - "Refreshable materialized view requires Atomic database engine, but database {} has engine {}", create.getDatabase(), database->getEngineName()); - /// TODO: Support Replicated databases, only with Shared/ReplicatedMergeTree. - /// Figure out how to make the refreshed data appear all at once on other - /// replicas; maybe a replicated SYSTEM SYNC REPLICA query before the rename? - if (database->getUUID() != UUIDHelpers::Nil) { if (create.attach && !from_path && create.uuid == UUIDHelpers::Nil) @@ -1257,54 +1345,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); - /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id && mode <= LoadingStrictnessLevel::CREATE) - { - if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( - {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, - getContext() - )) - { - Block input_block; - - if (getContext()->getSettingsRef().allow_experimental_analyzer) - { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); - } - else - { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); - } - - Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName input_columns; - ColumnsWithTypeAndName output_columns; - for (const auto & input_column : input_block) - { - if (const auto * output_column = output_block.findByName(input_column.name)) - { - input_columns.push_back(input_column.cloneEmpty()); - output_columns.push_back(output_column->cloneEmpty()); - } - } - - ActionsDAG::makeConvertingActions( - input_columns, - output_columns, - ActionsDAG::MatchColumnsMode::Position - ); - } - } - DatabasePtr database; bool need_add_to_database = !create.temporary; // In case of an ON CLUSTER query, the database may not be present on the initiator node if (need_add_to_database) database = DatabaseCatalog::instance().tryGetDatabase(database_name); + /// Check type compatible for materialized dest table and select columns + if (create.select && create.is_materialized_view && mode <= LoadingStrictnessLevel::CREATE) + validateMaterializedViewColumnsAndEngine(create, properties, database); + if (database && database->getEngineName() == "Replicated" && create.select) { bool is_storage_replicated = false; diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 70ef29e6b07..c51c0e757df 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -91,6 +91,7 @@ private: IndicesDescription indices; ConstraintsDescription constraints; ProjectionsDescription projections; + bool columns_inferred_from_select_query = false; }; BlockIO createDatabase(ASTCreateQuery & create); @@ -99,6 +100,7 @@ private: /// Calculate list of columns, constraints, indices, etc... of table. Rewrite query in canonical way. TableProperties getTablePropertiesAndNormalizeCreateQuery(ASTCreateQuery & create, LoadingStrictnessLevel mode) const; void validateTableStructure(const ASTCreateQuery & create, const TableProperties & properties) const; + void validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database); void setEngine(ASTCreateQuery & create) const; AccessRightsElements getRequiredAccess() const; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 3e5c6a9d86e..96a29348326 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -254,6 +254,8 @@ ASTPtr ASTCreateQuery::clone() const res->set(res->dictionary, dictionary->clone()); } + if (refresh_strategy) + res->set(res->refresh_strategy, refresh_strategy->clone()); if (as_table_function) res->set(res->as_table_function, as_table_function->clone()); if (comment) diff --git a/src/Parsers/ASTRenameQuery.h b/src/Parsers/ASTRenameQuery.h index d51c382f374..39fc4f787ec 100644 --- a/src/Parsers/ASTRenameQuery.h +++ b/src/Parsers/ASTRenameQuery.h @@ -141,6 +141,19 @@ public: QueryKind getQueryKind() const override { return QueryKind::Rename; } + void addElement(const String & from_db, const String & from_table, const String & to_db, const String & to_table) + { + auto identifier = [&](const String & name) -> ASTPtr + { + if (name.empty()) + return nullptr; + ASTPtr ast = std::make_shared(name); + children.push_back(ast); + return ast; + }; + elements.push_back(Element {.from = Table {.database = identifier(from_db), .table = identifier(from_table)}, .to = Table {.database = identifier(to_db), .table = identifier(to_table)}}); + } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 98afd844046..7d854f35029 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -103,7 +103,7 @@ public: IStorage(const IStorage &) = delete; IStorage & operator=(const IStorage &) = delete; - /// The main name of the table type (for example, StorageMergeTree). + /// The main name of the table type (e.g. Memory, MergeTree, CollapsingMergeTree). virtual std::string getName() const = 0; /// The name of the table. diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 83930ed7be9..e593023caee 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -37,7 +37,7 @@ RefreshTask::RefreshTask( refresh_settings.applyChanges(strategy.settings->changes); } -RefreshTaskHolder RefreshTask::create( +OwnedRefreshTask RefreshTask::create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) @@ -54,7 +54,7 @@ RefreshTaskHolder RefreshTask::create( context->getRefreshSet().emplace(view->getStorageID(), deps, task); - return task; + return OwnedRefreshTask(task); } void RefreshTask::initializeAndStart() @@ -169,6 +169,10 @@ void RefreshTask::shutdown() { { std::lock_guard guard(mutex); + + if (view == nullptr) + return; // already shut down + stop_requested = true; interruptExecution(); } @@ -373,7 +377,7 @@ void RefreshTask::executeRefreshUnlocked(bool append) LOG_DEBUG(log, "Refreshing view {}", view->getStorageID().getFullTableName()); progress.reset(); - ContextMutablePtr refresh_context; + ContextMutablePtr refresh_context = view->createRefreshContext(); std::optional table_to_drop; try { @@ -437,24 +441,13 @@ void RefreshTask::executeRefreshUnlocked(bool append) catch (...) { if (table_to_drop.has_value()) - { - try - { - InterpreterDropQuery::executeDropQuery( - ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ false, /*ignore_sync_setting*/ true); - } - catch (...) - { - tryLogCurrentException(log, "Failed to drop temporary table after a failed refresh"); - /// Let's ignore this and keep going, at risk of accumulating many trash tables if this keeps happening. - } - } + view->dropTempTable(table_to_drop.value(), refresh_context); throw; } /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). if (table_to_drop.has_value()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, view->getContext(), refresh_context, table_to_drop.value(), /*sync*/ true, /*ignore_sync_setting*/ true); + view->dropTempTable(table_to_drop.value(), refresh_context); } void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point now) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 9281c0469a3..411eb6fa7b4 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -17,6 +17,7 @@ class PipelineExecutor; class StorageMaterializedView; class ASTRefreshStrategy; +struct OwnedRefreshTask; class RefreshTask : public std::enable_shared_from_this { @@ -25,12 +26,12 @@ public: RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task - static RefreshTaskHolder create( + static OwnedRefreshTask create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); - void initializeAndStart(); + void initializeAndStart(); // called at most once /// Call when renaming the materialized view. void rename(StorageID new_id); @@ -58,6 +59,8 @@ public: void wait(); /// Permanently disable task scheduling and remove this table from RefreshSet. + /// Ok to call multiple times, but not in parallel. + /// Ok to call even if initializeAndStart() wasn't called or failed. void shutdown(); /// Notify dependent task @@ -160,4 +163,21 @@ private: std::chrono::system_clock::time_point currentTime() const; }; +/// Wrapper around shared_ptr, calls shutdown() in destructor. +struct OwnedRefreshTask +{ + RefreshTaskHolder ptr; + + OwnedRefreshTask() = default; + OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} + OwnedRefreshTask(OwnedRefreshTask &&) = default; + OwnedRefreshTask & operator=(OwnedRefreshTask &&) = default; + + ~OwnedRefreshTask() { if (ptr) ptr->shutdown(); } + + RefreshTask* operator->() const { return ptr.get(); } + RefreshTask& operator*() const { return *ptr; } + operator bool() const { return ptr != nullptr; } +}; + } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 500312fd245..9f793e06c09 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -370,21 +370,28 @@ bool StorageMaterializedView::optimize( return storage_ptr->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, cleanup, local_context); } -std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const +ContextMutablePtr StorageMaterializedView::createRefreshContext() const { auto refresh_context = getInMemoryMetadataPtr()->getSQLSecurityOverriddenContext(getContext()); - out_context = refresh_context; + refresh_context->setQueryKind(ClientInfo::QueryKind::INITIAL_QUERY); /// Generate a random query id. refresh_context->setCurrentQueryId(""); + /// TODO: Set view's definer as the current user in refresh_context, so that the correct user's + /// quotas and permissions apply for this query. + return refresh_context; +} - CurrentThread::QueryScope query_scope(refresh_context); - +std::shared_ptr StorageMaterializedView::prepareRefresh(bool append, ContextMutablePtr refresh_context, std::optional & out_temp_table_id) const +{ auto inner_table_id = getTargetTableId(); - auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); StorageID target_table = inner_table_id; if (!append) { + CurrentThread::QueryScope query_scope(refresh_context); + + auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + String db_name = db->getDatabaseName(); auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); auto create_table_query = db->getCreateTableQuery(inner_table_id.table_name, getContext()); @@ -399,7 +406,7 @@ std::shared_ptr StorageMaterializedView::prepareRefresh(bool app create_interpreter.setInternal(true); create_interpreter.execute(); - target_table = DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); + target_table = DatabaseCatalog::instance().getTable({db_name, new_table_name}, getContext())->getStorageID(); out_temp_table_id = target_table; } @@ -425,6 +432,9 @@ std::shared_ptr StorageMaterializedView::prepareRefresh(bool app StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context) { + /// Known problem: if the target table was ALTERed during refresh, this will effectively revert + /// the ALTER. + auto stale_table_id = getTargetTableId(); auto db = DatabaseCatalog::instance().getDatabase(stale_table_id.database_name); @@ -432,13 +442,34 @@ StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, Co CurrentThread::QueryScope query_scope(refresh_context); - target_db->renameTable( - refresh_context, fresh_table.table_name, *db, stale_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); + auto rename_query = std::make_shared(); + rename_query->exchange = true; + rename_query->addElement(fresh_table.database_name, fresh_table.table_name, stale_table_id.database_name, stale_table_id.table_name); - std::swap(stale_table_id.database_name, fresh_table.database_name); - std::swap(stale_table_id.table_name, fresh_table.table_name); - setTargetTableId(std::move(fresh_table)); - return stale_table_id; + InterpreterRenameQuery(rename_query, refresh_context).execute(); + + return fresh_table; +} + +void StorageMaterializedView::dropTempTable(StorageID table_id, ContextMutablePtr refresh_context) +{ + CurrentThread::QueryScope query_scope(refresh_context); + + try + { + auto drop_query = std::make_shared(); + drop_query->setDatabase(table_id.database_name); + drop_query->setTable(table_id.table_name); + drop_query->kind = ASTDropQuery::Kind::Drop; + drop_query->if_exists = true; + drop_query->sync = false; + + InterpreterDropQuery(drop_query, refresh_context).execute(); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::get("StorageMaterializedView"), "Failed to drop temporary table after refresh"); + } } void StorageMaterializedView::alter( @@ -543,20 +574,7 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); - ASTRenameQuery::Element elem - { - ASTRenameQuery::Table - { - inner_table_id.database_name.empty() ? nullptr : std::make_shared(inner_table_id.database_name), - std::make_shared(inner_table_id.table_name) - }, - ASTRenameQuery::Table - { - new_table_id.database_name.empty() ? nullptr : std::make_shared(new_table_id.database_name), - std::make_shared(new_target_table_name) - } - }; - rename_elements.emplace_back(std::move(elem)); + rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_table_id.table_name); auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); @@ -715,12 +733,6 @@ StorageID StorageMaterializedView::getTargetTableId() const return id; } -void StorageMaterializedView::setTargetTableId(DB::StorageID id) -{ - std::lock_guard guard(target_table_id_mutex); - target_table_id = std::move(id); -} - void StorageMaterializedView::updateTargetTableId(std::optional database_name, std::optional table_name) { std::lock_guard guard(target_table_id_mutex); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 12c0dc651e9..70487c1fad3 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -5,7 +5,7 @@ #include #include -#include +#include namespace DB { @@ -106,7 +106,7 @@ private: /// Will be initialized in constructor StorageID target_table_id = StorageID::createEmpty(); - RefreshTaskHolder refresher; + OwnedRefreshTask refresher; bool refresh_on_start = false; bool has_inner_table = false; @@ -119,14 +119,15 @@ private: void checkStatementCanBeForwarded() const; - /// Prepare to refresh a refreshable materialized view: create query context, create temporary - /// table, form the insert-select query. - /// The output arguments may be assigned before throwing an exception, in which case the caller + ContextMutablePtr createRefreshContext() const; + /// Prepare to refresh a refreshable materialized view: create temporary table and form the + /// insert-select query. + /// out_temp_table_id may be assigned before throwing an exception, in which case the caller /// must drop the temp table before rethrowing. - std::shared_ptr prepareRefresh(bool append, ContextMutablePtr & out_context, std::optional & out_temp_table_id) const; + std::shared_ptr prepareRefresh(bool append, ContextMutablePtr refresh_context, std::optional & out_temp_table_id) const; StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); + void dropTempTable(StorageID table, ContextMutablePtr refresh_context); - void setTargetTableId(StorageID id); void updateTargetTableId(std::optional database_name, std::optional table_name); }; diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index ad6fa3043a1..528a97535cc 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -257,7 +257,7 @@ $CLICKHOUSE_CLIENT -nq " # Try OFFSET and RANDOMIZE FOR. $CLICKHOUSE_CLIENT -nq " - create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42; + create materialized view g refresh every 1 week offset 3 day 4 hour randomize for 4 day 1 hour (x Int64) engine Memory empty as select 42 as x; show create g; system test view g set fake time '2050-02-03 15:30:13';" while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] @@ -359,5 +359,12 @@ $CLICKHOUSE_CLIENT -nq " drop table m; drop table mid;" +# Failing to create inner table. +$CLICKHOUSE_CLIENT -nq " + create materialized view n refresh every 1 second (x Int64) engine MergeTree as select 1 as x from numbers(2);" 2>/dev/null || echo "creating MergeTree without ORDER BY failed, as expected" +$CLICKHOUSE_CLIENT -nq " + create materialized view n refresh every 1 second (x Int64) engine MergeTree order by x as select 1 as x from numbers(2); + drop table n;" + $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From 09f07838af87cdbf3d1c2a829009ce81f7ddd178 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Feb 2024 00:46:35 +0000 Subject: [PATCH 04/45] Add 'ness' (as in 'APPEND-ness') to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f1fcd19ea4a..b1e5c208b1c 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2083,6 +2083,7 @@ namequota namespaces natively nats +ness nestjs netloc ngram From be04b4439c8d7cf9b313cc6e1d750820653d95ef Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 2 Feb 2024 05:49:14 +0000 Subject: [PATCH 05/45] Unbreak --- src/Storages/StorageMaterializedView.cpp | 12 +++++++++++- src/Storages/StorageMaterializedView.h | 1 + .../02932_refreshable_materialized_views.reference | 3 ++- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9f793e06c09..459fc300159 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -448,7 +448,11 @@ StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table, Co InterpreterRenameQuery(rename_query, refresh_context).execute(); - return fresh_table; + std::swap(stale_table_id.database_name, fresh_table.database_name); + std::swap(stale_table_id.table_name, fresh_table.table_name); + + setTargetTableId(std::move(fresh_table)); + return stale_table_id; } void StorageMaterializedView::dropTempTable(StorageID table_id, ContextMutablePtr refresh_context) @@ -733,6 +737,12 @@ StorageID StorageMaterializedView::getTargetTableId() const return id; } +void StorageMaterializedView::setTargetTableId(DB::StorageID id) +{ + std::lock_guard guard(target_table_id_mutex); + target_table_id = std::move(id); +} + void StorageMaterializedView::updateTargetTableId(std::optional database_name, std::optional table_name) { std::lock_guard guard(target_table_id_mutex); diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 70487c1fad3..a09ee07b3f6 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -128,6 +128,7 @@ private: StorageID exchangeTargetTable(StorageID fresh_table, ContextPtr refresh_context); void dropTempTable(StorageID table, ContextMutablePtr refresh_context); + void setTargetTableId(StorageID id); void updateTargetTableId(std::optional database_name, std::optional table_name); }; diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 8d4878f4e3f..3bc0d59a608 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -39,7 +39,7 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <25: rename during refresh> f Running <27: cancelled> f Scheduled <28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 @@ -59,3 +59,4 @@ CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n <38: append chain> 100 <38: append chain> 100 <38: append chain> 200 +creating MergeTree without ORDER BY failed, as expected From 285936fe3fa2fd7bd850b2b10588ad7216938ed4 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 8 Feb 2024 02:55:03 +0000 Subject: [PATCH 06/45] Fix renaming --- src/Storages/StorageMaterializedView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 459fc300159..3691fd5e212 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -578,7 +578,7 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); - rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_table_id.table_name); + rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_target_table_name); auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); From f1ea73624956e80d63ce59b98e0bbf19bfbacb08 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 8 Feb 2024 03:03:12 +0000 Subject: [PATCH 07/45] Fix missing column aliases in other tests --- .../queries/0_stateless/00982_low_cardinality_setting_in_mv.sql | 2 +- tests/queries/0_stateless/01160_table_dependencies.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql b/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql index 7192642bcde..e545dec90b7 100644 --- a/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql +++ b/tests/queries/0_stateless/00982_low_cardinality_setting_in_mv.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS mat_view; CREATE TABLE test1 (a LowCardinality(String)) ENGINE=MergeTree() ORDER BY a; CREATE TABLE test2 (a UInt64) engine=MergeTree() ORDER BY a; -CREATE MATERIALIZED VIEW test_mv TO test2 AS SELECT toUInt64(a = 'test') FROM test1; +CREATE MATERIALIZED VIEW test_mv TO test2 AS SELECT toUInt64(a = 'test') AS a FROM test1; DROP TABLE test_mv; DROP TABLE test1; diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index acb6522e9e2..b72acf62610 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -35,7 +35,7 @@ arraySort(loading_dependencies_table), arraySort(loading_dependent_table) from s $CLICKHOUSE_CLIENT -q "select '====='" $CLICKHOUSE_CLIENT -q "alter table t add column x int default in(1, $CLICKHOUSE_DATABASE.s), drop column y" -$CLICKHOUSE_CLIENT -q "create materialized view mv to s as select n from t where n in (select n from join)" +$CLICKHOUSE_CLIENT -q "create materialized view mv to s as select n as x from t where n in (select n from join)" $CLICKHOUSE_CLIENT -q "select table, arraySort(dependencies_table), arraySort(loading_dependencies_table), arraySort(loading_dependent_table) from system.tables where database=currentDatabase() order by table" From 7575115069b0a0072b0e4af7c849c3eabf55d7ea Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 10 Feb 2024 01:57:28 +0000 Subject: [PATCH 08/45] Fix race conditions in the test, more small fixes to other tests --- tests/queries/0_stateless/02345_implicit_transaction.sql | 2 +- .../0_stateless/02932_refreshable_materialized_views.sh | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02345_implicit_transaction.sql b/tests/queries/0_stateless/02345_implicit_transaction.sql index ee2e0a07c3e..9496de71e13 100644 --- a/tests/queries/0_stateless/02345_implicit_transaction.sql +++ b/tests/queries/0_stateless/02345_implicit_transaction.sql @@ -3,7 +3,7 @@ CREATE TABLE landing (n Int64) engine=MergeTree order by n; CREATE TABLE target (n Int64) engine=MergeTree order by n; CREATE MATERIALIZED VIEW landing_to_target TO target AS - SELECT n + throwIf(n == 3333) + SELECT n + throwIf(n == 3333) AS n FROM landing; INSERT INTO landing SELECT * FROM numbers(10000); -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 528a97535cc..2324041728e 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -298,8 +298,10 @@ $CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" $CLICKHOUSE_CLIENT -nq " select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; - truncate table src; - insert into src values (1);" + create table src2 empty as src; + insert into src2 values (1) + exchange tables src and src2; + drop table src2;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] do sleep 0.1 @@ -336,6 +338,7 @@ $CLICKHOUSE_CLIENT -nq " # ALTER to non-APPEND $CLICKHOUSE_CLIENT -nq " alter table k modify refresh every 10 year; + system wait view k; system refresh view k; system wait view k; select '<36: not append>', * from k order by x; From 279d9c78ee30eafec6a3d2dc711d2c4e0496944a Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 19 Feb 2024 14:52:19 +0000 Subject: [PATCH 09/45] Fix race in wait --- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index e593023caee..fdcf358cc20 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -160,7 +160,7 @@ void RefreshTask::cancel() void RefreshTask::wait() { std::unique_lock lock(mutex); - refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running; }); + refresh_cv.wait(lock, [&] { return info.state != RefreshState::Running && !refresh_immediately; }); if (info.last_refresh_result == LastRefreshResult::Error) throw Exception(ErrorCodes::REFRESH_FAILED, "Refresh failed: {}", info.exception_message); } From 78a3d0d4f71554c7f07e92f3996b56a28d5acbdf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 21 Feb 2024 12:51:19 +0000 Subject: [PATCH 10/45] Fix parallel_mv perf test --- tests/performance/parallel_mv.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/parallel_mv.xml b/tests/performance/parallel_mv.xml index 5b856740a19..0bf5ed1be09 100644 --- a/tests/performance/parallel_mv.xml +++ b/tests/performance/parallel_mv.xml @@ -11,13 +11,13 @@ create table mt_4 (n UInt64, s String) engine = MergeTree order by tuple() create materialized view mv_1 to mt_1 as - select number, toString(number) from main_table where number % 13 != 0 + select number as n, toString(number) as s from main_table where number % 13 != 0 create materialized view mv_2 to mt_2 as - select number, toString(number) from main_table where number % 13 != 1 + select number as n, toString(number) as s from main_table where number % 13 != 1 create materialized view mv_3 to mt_3 as - select number, toString(number) from main_table where number % 13 != 3 + select number as n, toString(number) as s from main_table where number % 13 != 3 create materialized view mv_4 to mt_4 as - select number, toString(number) from main_table where number % 13 != 4 + select number as n, toString(number) as s from main_table where number % 13 != 4 SYSTEM STOP MERGES main_table SYSTEM STOP MERGES mt_1 From ef5070c1e66e41686eef983272b9a434257c34bf Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 9 Mar 2024 02:12:53 +0000 Subject: [PATCH 11/45] Fix conflict --- src/Parsers/ASTSystemQuery.cpp | 1 + .../02932_refreshable_materialized_views.reference | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index a730ea0ba3d..a183eb4c0f6 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -375,6 +375,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s case Type::START_VIEW: case Type::STOP_VIEW: case Type::CANCEL_VIEW: + case Type::WAIT_VIEW: { settings.ostr << ' '; print_database_table(); diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference index 3bc0d59a608..3c6c8b2d778 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.reference @@ -3,13 +3,13 @@ CREATE MATERIALIZED VIEW default.a\nREFRESH AFTER 2 SECOND\n(\n `x` UInt64\n) <2: refreshed> 3 1 1 <3: time difference at least> 1000 <4: next refresh in> 2 -<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 +<4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:03 <4.5: altered> Scheduled Finished 2052-01-01 00:00:00 CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled Finished 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a <7.5: created dependent> 2052-11-11 11:11:11 <8: refreshed> 20 <9: refreshed> a Scheduled Finished 2054-01-01 00:00:00 @@ -39,7 +39,7 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nE <25: rename during refresh> f Running <27: cancelled> f Scheduled <28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x <29: randomize> 1 1 CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 From 5ce3aaf49ef947e4481e8d7934ee84190ef04542 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 23 Mar 2024 01:29:24 +0000 Subject: [PATCH 12/45] Fix conflicts --- src/Parsers/ParserRefreshStrategy.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index de7eea93614..4f3b7c66558 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -97,7 +97,7 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec refresh->set(refresh->settings, settings); } - if (ParserKeyword{"APPEND"}.ignore(pos, expected)) + if (ParserKeyword{Keyword::APPEND}.ignore(pos, expected)) refresh->append = true; node = refresh; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 3691fd5e212..a78bb44eb66 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -575,12 +575,11 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto new_target_table_name = generateInnerTableName(new_table_id); - ASTRenameQuery::Elements rename_elements; assert(inner_table_id.database_name == old_table_id.database_name); + auto rename = std::make_shared(); rename->addElement(inner_table_id.database_name, inner_table_id.table_name, new_table_id.database_name, new_target_table_name); - auto rename = std::make_shared(std::move(rename_elements)); InterpreterRenameQuery(rename, getContext()).execute(); updateTargetTableId(new_table_id.database_name, new_target_table_name); } From 0549d154058169e9f4831923c2c2d2e8fdc295b5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 26 Mar 2024 04:10:33 +0000 Subject: [PATCH 13/45] tidy --- src/Storages/MaterializedView/RefreshTask.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 411eb6fa7b4..e8e3e45ace3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -169,7 +169,7 @@ struct OwnedRefreshTask RefreshTaskHolder ptr; OwnedRefreshTask() = default; - OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} + explicit OwnedRefreshTask(RefreshTaskHolder p) : ptr(std::move(p)) {} OwnedRefreshTask(OwnedRefreshTask &&) = default; OwnedRefreshTask & operator=(OwnedRefreshTask &&) = default; @@ -177,7 +177,7 @@ struct OwnedRefreshTask RefreshTask* operator->() const { return ptr.get(); } RefreshTask& operator*() const { return *ptr; } - operator bool() const { return ptr != nullptr; } + explicit operator bool() const { return ptr != nullptr; } }; } From 90c2b29ea28afa99f4a2dd7919ffc3274cf04764 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 01:44:32 +0000 Subject: [PATCH 14/45] Log retry information, remove outdated comment --- src/Storages/MaterializedView/RefreshTask.cpp | 24 +++++++++---------- src/Storages/MaterializedView/RefreshTask.h | 3 ++- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index fdcf358cc20..71feece13e6 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -1,7 +1,6 @@ #include -#include - +#include #include #include #include @@ -10,6 +9,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -312,13 +312,7 @@ void RefreshTask::refreshTask() catch (...) { if (!interrupt_execution.load()) - { - PreformattedMessage message = getCurrentExceptionMessageAndPattern(true); - auto text = message.text; - message.text = fmt::format("Refresh failed: {}", message.text); - LOG_ERROR(log, message); - exception = text; - } + exception = getCurrentExceptionMessage(true); } lock.lock(); @@ -332,7 +326,10 @@ void RefreshTask::refreshTask() { info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - scheduleRetryOrSkipToNextRefresh(now); + + String retry_info = scheduleRetryOrSkipToNextRefresh(now); + + LOG_ERROR(log, "Refresh failed ({}): {}", retry_info, *exception); } else if (!refreshed) { @@ -445,7 +442,6 @@ void RefreshTask::executeRefreshUnlocked(bool append) throw; } - /// Drop the old table (outside the try-catch so we don't try to drop the other table if this fails). if (table_to_drop.has_value()) view->dropTempTable(table_to_drop.value(), refresh_context); } @@ -463,12 +459,13 @@ void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point n info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } -void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) { + Int64 attempt_number = num_retries + 1; if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) { advanceNextRefreshTime(now); - return; + return fmt::format("attempt {}/{}, next refresh at {:%Y.%m.%d %T}", attempt_number, refresh_settings.refresh_retries + 1, next_refresh_actual); } num_retries += 1; @@ -483,6 +480,7 @@ void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::ti delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); + return fmt::format("attempt {}/{}, retry in {} seconds", attempt_number, refresh_settings.refresh_retries, (delay_ms + 500) / 1000); } bool RefreshTask::arriveDependency(const StorageID & parent) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index e8e3e45ace3..54253548795 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -151,7 +151,8 @@ private: void advanceNextRefreshTime(std::chrono::system_clock::time_point now); /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). - void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + /// Returns human-readable information for logging. + String scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); From f40b45b78cb5838af70df8eb9611370a14a7e7b9 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 02:20:29 +0000 Subject: [PATCH 15/45] Avoid fmt/chrono.h because clang refuses to compile it --- src/Storages/MaterializedView/RefreshTask.cpp | 15 ++++++--------- src/Storages/MaterializedView/RefreshTask.h | 3 +-- 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 71feece13e6..781cefbf449 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -326,10 +325,9 @@ void RefreshTask::refreshTask() { info.last_refresh_result = LastRefreshResult::Error; info.exception_message = *exception; - - String retry_info = scheduleRetryOrSkipToNextRefresh(now); - - LOG_ERROR(log, "Refresh failed ({}): {}", retry_info, *exception); + Int64 attempt_number = num_retries + 1; + scheduleRetryOrSkipToNextRefresh(now); + LOG_ERROR(log, "Refresh failed (attempt {}/{}): {}", attempt_number, refresh_settings.refresh_retries + 1, *exception); } else if (!refreshed) { @@ -459,13 +457,12 @@ void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point n info.next_refresh_time = UInt32(secs.time_since_epoch().count()); } -String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) +void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now) { - Int64 attempt_number = num_retries + 1; if (refresh_settings.refresh_retries >= 0 && num_retries >= refresh_settings.refresh_retries) { advanceNextRefreshTime(now); - return fmt::format("attempt {}/{}, next refresh at {:%Y.%m.%d %T}", attempt_number, refresh_settings.refresh_retries + 1, next_refresh_actual); + return; } num_retries += 1; @@ -480,7 +477,7 @@ String RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock:: delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); - return fmt::format("attempt {}/{}, retry in {} seconds", attempt_number, refresh_settings.refresh_retries, (delay_ms + 500) / 1000); + return; } bool RefreshTask::arriveDependency(const StorageID & parent) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 54253548795..e8e3e45ace3 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -151,8 +151,7 @@ private: void advanceNextRefreshTime(std::chrono::system_clock::time_point now); /// Either advances next_refresh_actual using exponential backoff or does advanceNextRefreshTime(). - /// Returns human-readable information for logging. - String scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); + void scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::time_point now); /// Returns true if all dependencies are fulfilled now. Refills remaining_dependencies in this case. bool arriveDependency(const StorageID & parent); From e1b9844086f7cd31b1d0bd2fa7de80d521325e5f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 05:29:44 +0000 Subject: [PATCH 16/45] Lint --- src/Storages/MaterializedView/RefreshTask.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 781cefbf449..6cc85cc0f82 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -477,7 +477,6 @@ void RefreshTask::scheduleRetryOrSkipToNextRefresh(std::chrono::system_clock::ti delay_ms = refresh_settings.refresh_retry_max_backoff_ms; next_refresh_actual = now + std::chrono::milliseconds(delay_ms); - return; } bool RefreshTask::arriveDependency(const StorageID & parent) From 2e8a7ecc57a103997c5c659960c91d28fab1aa86 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 17:43:02 +0000 Subject: [PATCH 17/45] Remove unused variable --- src/Interpreters/InterpreterCreateQuery.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ee57c376b31..6e5b500cd69 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -946,10 +946,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC /// This is not strict validation, just catches common errors that would make the view not work. /// It's possible to circumvent these checks by ALTERing the view or target table after creation. - String table_engine; - if (create.storage && create.storage->engine) - table_engine = create.storage->engine->name; - NamesAndTypesList all_output_columns; bool check_columns = false; if (create.to_table_id) @@ -961,7 +957,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC { all_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock().getNamesAndTypesList(); check_columns = true; - table_engine = to_table->getName(); } } else if (!properties.columns_inferred_from_select_query) From f4c13a1d9ad82dbc4382729490e3226c81073c1c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 11 Jun 2024 19:13:29 +0000 Subject: [PATCH 18/45] Add compatibility setting allow_materialized_view_with_bad_select --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 44 +++++++++---------- .../test_replicated_database/test.py | 1 + .../02932_refreshable_materialized_views.sh | 8 ++++ 5 files changed, 33 insertions(+), 22 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fbab72446a0..954e21dd988 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -635,6 +635,7 @@ class IColumn; M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ + M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns, or outputs columns with unexpected names (which will be ignored). It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index fbc414d4f2f..9d3832e451d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -115,6 +115,7 @@ static const std::mapgetEngineName()); } + Block input_block; + bool relaxed = getContext()->getSettingsRef().allow_materialized_view_with_bad_select; + if (check_columns) { - Block input_block; - - if (getContext()->getSettingsRef().allow_experimental_analyzer) + try { - input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + if (getContext()->getSettingsRef().allow_experimental_analyzer) + { + input_block = InterpreterSelectQueryAnalyzer::getSampleBlock(create.select->clone(), getContext()); + } + else + { + input_block = InterpreterSelectWithUnionQuery(create.select->clone(), + getContext(), + SelectQueryOptions().analyze()).getSampleBlock(); + } } - else + catch (Exception &) { - input_block = InterpreterSelectWithUnionQuery(create.select->clone(), - getContext(), - SelectQueryOptions().analyze()).getSampleBlock(); + if (!relaxed) + throw; + check_columns = false; } + } + if (check_columns) + { std::unordered_map output_types; for (const NameAndTypePair & nt : all_output_columns) output_types[nt.name] = nt.type; @@ -1001,21 +1014,8 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC input_columns.push_back(input_column.cloneEmpty()); output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); } - else if (create.refresh_strategy) - { - /// Unrecognized columns produced by SELECT query are allowed for regular materialized - /// views, but not for refreshable ones. This is in part because it was easier to - /// implement, in part because refreshable views have less concern about ALTERing target - /// tables. - /// - /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. - /// Suppose the user removes a column from target table, then a minute later - /// correspondingly updates the view's query to not produce that column. - /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the - /// source table would fail - unacceptable. - /// For refreshable views, during that minute refreshes will fail - acceptable. + else if (!relaxed) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); - } } if (input_columns.empty()) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index f23384b5c04..4ecb1aefba7 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -708,6 +708,7 @@ def create_some_tables(db): "distributed_ddl_task_timeout": 0, "allow_experimental_object_type": 1, "allow_suspicious_codecs": 1, + "allow_materialized_view_with_bad_select": 1, } main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 2324041728e..580d1bdc190 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -369,5 +369,13 @@ $CLICKHOUSE_CLIENT -nq " create materialized view n refresh every 1 second (x Int64) engine MergeTree order by x as select 1 as x from numbers(2); drop table n;" +# Reading from table that doesn't exist yet. +$CLICKHOUSE_CLIENT -nq " + create materialized view o refresh every 1 second (x Int64) engine Memory as select x from nonexist; -- { serverError UNKNOWN_TABLE } + create materialized view o (x Int64) engine Memory as select x from nonexist; -- { serverError UNKNOWN_TABLE } + create materialized view o (x Int64) engine Memory as select x from nope.nonexist; -- { serverError UNKNOWN_DATABASE } + create materialized view o refresh every 1 second (x Int64) engine Memory as select x from nope.nonexist settings allow_materialized_view_with_bad_select = 1; + drop table o;" + $CLICKHOUSE_CLIENT -nq " drop table refreshes;" From b3b70b037ff1b7f317ac5fe0b5219e1308971581 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 12 Jun 2024 00:25:31 +0000 Subject: [PATCH 19/45] Revert some of the strictness, because of comment in 01019_materialized_view_select_extra_columns --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 18 +++++++++++++++--- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 954e21dd988..93e60541352 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -635,7 +635,7 @@ class IColumn; M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns, or outputs columns with unexpected names (which will be ignored). It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, allow_experimental_refreshable_materialized_view, false, "Allow refreshable materialized views (CREATE MATERIALIZED VIEW REFRESH ...).", 0) \ M(Bool, stop_refreshable_materialized_views_on_startup, false, "On server startup, prevent scheduling of refreshable materialized views, as if with SYSTEM STOP VIEWS. You can manually start them with SYSTEM START VIEWS or SYSTEM START VIEW afterwards. Also applies to newly created views. Has no effect on non-refreshable materialized views.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f0c450f2ff3..1f6cf31e3d3 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -973,7 +973,6 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC } Block input_block; - bool relaxed = getContext()->getSettingsRef().allow_materialized_view_with_bad_select; if (check_columns) { @@ -992,7 +991,7 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC } catch (Exception &) { - if (!relaxed) + if (!getContext()->getSettingsRef().allow_materialized_view_with_bad_select) throw; check_columns = false; } @@ -1014,8 +1013,21 @@ void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTC input_columns.push_back(input_column.cloneEmpty()); output_columns.push_back(ColumnWithTypeAndName(it->second->createColumn(), it->second, input_column.name)); } - else if (!relaxed) + else if (create.refresh_strategy) + { + /// Unrecognized columns produced by SELECT query are allowed by regular materialized + /// views, but not by refreshable ones. This is in part because it was easier to + /// implement, in part because refreshable views have less concern about ALTERing target + /// tables. + /// + /// The motivating scenario for allowing this in regular MV is ALTERing the table+query. + /// Suppose the user removes a column from target table, then a minute later + /// correspondingly updates the view's query to not produce that column. + /// If MV didn't allow unrecognized columns then during that minute all INSERTs into the + /// source table would fail - unacceptable. + /// For refreshable views, during that minute refreshes will fail - acceptable. throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "SELECT query outputs column with name '{}', which is not found in the target table. Use 'AS' to assign alias that matches a column name.", input_column.name); + } } if (input_columns.empty()) From 9957fe1734260cecd383227abc9ab75283e95d9b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 20 Jun 2024 18:38:12 +0000 Subject: [PATCH 20/45] Comment fixes --- docs/en/sql-reference/statements/create/view.md | 16 ++++++++-------- src/Interpreters/InterpreterCreateQuery.cpp | 3 ++- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 3e0766794f1..950d8207028 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -13,8 +13,8 @@ Creates a new view. Views can be [normal](#normal-view), [materialized](#materia Syntax: ``` sql -CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] -[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] +CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name] +[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... ``` @@ -54,8 +54,8 @@ SELECT * FROM view(column1=value1, column2=value2 ...) ## Materialized View ``` sql -CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] -[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] +CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE] +[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }] AS SELECT ... ``` @@ -90,7 +90,7 @@ Given that `POPULATE` works like `CREATE TABLE ... AS SELECT ...` it has limitat - It is not supported with Replicated database - It is not supported in ClickHouse cloud -Instead a separate `INSERT ... SELECT` can be used. +Instead a separate `INSERT ... SELECT` can be used. ::: A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. @@ -108,7 +108,7 @@ To delete a view, use [DROP VIEW](../../../sql-reference/statements/drop.md#drop `DEFINER` and `SQL SECURITY` allow you to specify which ClickHouse user to use when executing the view's underlying query. `SQL SECURITY` has three legal values: `DEFINER`, `INVOKER`, or `NONE`. You can specify any existing user or `CURRENT_USER` in the `DEFINER` clause. -The following table will explain which rights are required for which user in order to select from view. +The following table will explain which rights are required for which user in order to select from view. Note that regardless of the SQL security option, in every case it is still required to have `GRANT SELECT ON ` in order to read from it. | SQL security option | View | Materialized View | @@ -128,7 +128,7 @@ If `DEFINER`/`SQL SECURITY` aren't specified, the default values are used: If a view is attached without `DEFINER`/`SQL SECURITY` specified, the default value is `SQL SECURITY NONE` for the materialized view and `SQL SECURITY INVOKER` for the normal view. -To change SQL security for an existing view, use +To change SQL security for an existing view, use ```sql ALTER TABLE MODIFY SQL SECURITY { DEFINER | INVOKER | NONE } [DEFINER = { user | CURRENT_USER }] ``` @@ -178,7 +178,7 @@ Differences from regular non-refreshable materialized views: * No restrictions on the SELECT query. Table functions (e.g. `url()`), views, UNION, JOIN, are all allowed. :::note -The settings in the The settings in `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. +The settings in the `REFRESH ... SETTINGS` part of the query are refresh settings (e.g. `refresh_retries`), distinct from regular settings (e.g. `max_threads`). Regular settings can be specified using `SETTINGS` at the end of the query. ::: :::note diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1f6cf31e3d3..186df8f2108 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -944,7 +944,8 @@ void validateVirtualColumns(const IStorage & storage) void InterpreterCreateQuery::validateMaterializedViewColumnsAndEngine(const ASTCreateQuery & create, const TableProperties & properties, const DatabasePtr & database) { /// This is not strict validation, just catches common errors that would make the view not work. - /// It's possible to circumvent these checks by ALTERing the view or target table after creation. + /// It's possible to circumvent these checks by ALTERing the view or target table after creation; + /// we should probably do some of these checks on ALTER as well. NamesAndTypesList all_output_columns; bool check_columns = false; From f72b0335cdc1d76b5aeb6d130900a68a8f4c8e3b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 22 Jun 2024 02:27:06 +0000 Subject: [PATCH 21/45] Test workaround --- .../queries/0_stateless/02932_refreshable_materialized_views.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh index 580d1bdc190..d5b07287936 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views.sh @@ -298,7 +298,7 @@ $CLICKHOUSE_CLIENT -nq " $CLICKHOUSE_CLIENT -nq "system wait view h2;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" $CLICKHOUSE_CLIENT -nq " select '<31.5: will retry>', last_refresh_result, retry > 0 from refreshes; - create table src2 empty as src; + create table src2 (x Int8) engine Memory; insert into src2 values (1) exchange tables src and src2; drop table src2;" From f8a117affc055fc15396374ed8b258b13b5adc3f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 Jul 2024 20:23:39 +0000 Subject: [PATCH 22/45] Split the test --- ...efreshable_materialized_views_1.reference} | 30 --- .../02932_refreshable_materialized_views_1.sh | 179 +++++++++++++++++ ...refreshable_materialized_views_2.reference | 30 +++ ...02932_refreshable_materialized_views_2.sh} | 184 ++---------------- 4 files changed, 222 insertions(+), 201 deletions(-) rename tests/queries/0_stateless/{02932_refreshable_materialized_views.reference => 02932_refreshable_materialized_views_1.reference} (66%) create mode 100755 tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh create mode 100644 tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference rename tests/queries/0_stateless/{02932_refreshable_materialized_views.sh => 02932_refreshable_materialized_views_2.sh} (54%) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference similarity index 66% rename from tests/queries/0_stateless/02932_refreshable_materialized_views.reference rename to tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference index 3c6c8b2d778..bfc6add90a7 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference @@ -30,33 +30,3 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n( <17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 <18: removed dependency> b Scheduled [] 2062-03-03 03:03:03 2064-01-01 00:00:00 5 CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a -<19: exception> 1 -<20: unexception> 1 -<21: rename> 1 -<22: rename> d Finished -<23: simple refresh> 1 -<24: rename during refresh> 1 -<25: rename during refresh> f Running -<27: cancelled> f Scheduled -<28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x -<29: randomize> 1 1 -CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src -<30: to existing table> 10 -<31: to existing table> 10 -<31: to existing table> 20 -<31.5: will retry> Error 1 -<31.6: did retry> 10 -<32: empty> i Scheduled Unknown 0 -<32: empty> j Scheduled Finished 0 -<34: append> 10 -<35: append> 10 -<35: append> 20 -<35: append> 30 -<36: not append> 20 -<36: not append> 30 -<37: append chain> 100 -<38: append chain> 100 -<38: append chain> 100 -<38: append chain> 200 -creating MergeTree without ORDER BY failed, as expected diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh new file mode 100755 index 00000000000..03d752b995d --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.sh @@ -0,0 +1,179 @@ +#!/usr/bin/env bash +# Tags: atomic-database + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# reset --log_comment +CLICKHOUSE_LOG_COMMENT= +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh +# scheduling is done in UTC. +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" + +$CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" + + +# Basic refreshing. +$CLICKHOUSE_CLIENT -nq " + create materialized view a + refresh after 2 second + engine Memory + empty + as select number as x from numbers(2) union all select rand64() as x; + select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; + show create a;" +# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] +do + sleep 0.5 +done +start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" +# Check table contents. +$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" +# Wait for table contents to change. +res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" +while : +do + res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res2" == "$res1" ] || break + sleep 0.5 +done +# Wait for another change. +while : +do + res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" + [ "$res3" == "$res2" ] || break + sleep 0.5 +done +# Check that the two changes were at least 1 second apart, in particular that we're not refreshing +# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer +# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. +$CLICKHOUSE_CLIENT -nq " + select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); + select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" + +# Create a source table from which views will read. +$CLICKHOUSE_CLIENT -nq " + create table src (x Int8) engine Memory as select 1;" + +# Switch to fake clock, change refresh schedule, change query. +$CLICKHOUSE_CLIENT -nq " + system test view a set fake time '2050-01-01 00:00:01'; + system wait view a; + system refresh view a; + system wait view a; + select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; + alter table a modify refresh every 2 year; + alter table a modify query select x*2 as x from src; + select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; + show create a;" +# Advance time to trigger the refresh. +$CLICKHOUSE_CLIENT -nq " + select '<5: no refresh>', count() from a; + system test view a set fake time '2052-02-03 04:05:06';" +while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<6: refreshed>', * from a; + select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" + +# Create a dependent view, refresh it once. +$CLICKHOUSE_CLIENT -nq " + create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; + show create b; + system test view b set fake time '2052-11-11 11:11:11'; + system refresh view b; + system wait view b; + select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" +# Next refresh shouldn't start until the dependency refreshes. +$CLICKHOUSE_CLIENT -nq " + select '<8: refreshed>', * from b; + select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; + system test view b set fake time '2054-01-24 23:22:21';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] +do + sleep 0.5 +done + +# Drop the source table, check that refresh fails and doesn't leave a temp table behind. +$CLICKHOUSE_CLIENT -nq " + select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); + drop table src; + system refresh view a;" +$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" +$CLICKHOUSE_CLIENT -nq " + select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" + +# Create the source table again, check that refresh succeeds (in particular that tables are looked +# up by name rather than uuid). +$CLICKHOUSE_CLIENT -nq " + select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; + create table src (x Int16) engine Memory as select 2; + system test view a set fake time '2054-01-01 00:00:01';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] +do + sleep 0.5 +done +# Both tables should've refreshed. +$CLICKHOUSE_CLIENT -nq " + select '<11: chain-refreshed a>', * from a; + select '<12: chain-refreshed b>', * from b; + select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" + +# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to +# catch up to the same cycle. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2059-01-01 00:00:00'; + system refresh view b;" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2061-01-01 00:00:00'; + system test view a set fake time '2057-01-01 00:00:00';" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] +do + sleep 0.5 +done +sleep 1 +$CLICKHOUSE_CLIENT -nq " + select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; + truncate src; + insert into src values (3); + system test view a set fake time '2060-02-02 02:02:02';" +while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<15: chain-refreshed a>', * from a; + select '<16: chain-refreshed b>', * from b; + select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" + +# Get to WaitingForDependencies state and remove the depencency. +$CLICKHOUSE_CLIENT -nq " + system test view b set fake time '2062-03-03 03:03:03'" +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + alter table b modify refresh every 2 year" +while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] +do + sleep 0.5 +done +$CLICKHOUSE_CLIENT -nq " + select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; + show create b;" + +$CLICKHOUSE_CLIENT -nq " + drop table src; + drop table a; + drop table b; + drop table refreshes;" diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference new file mode 100644 index 00000000000..eb4a0498260 --- /dev/null +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference @@ -0,0 +1,30 @@ +<19: exception> 1 +<20: unexception> 1 +<21: rename> 1 +<22: rename> d Finished +<23: simple refresh> 1 +<24: rename during refresh> 1 +<25: rename during refresh> f Running +<27: cancelled> f Scheduled +<28: drop during refresh> 0 0 +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x +<29: randomize> 1 1 +CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src +<30: to existing table> 10 +<31: to existing table> 10 +<31: to existing table> 20 +<31.5: will retry> Error 1 +<31.6: did retry> 10 +<32: empty> i Scheduled Unknown 0 +<32: empty> j Scheduled Finished 0 +<34: append> 10 +<35: append> 10 +<35: append> 20 +<35: append> 30 +<36: not append> 20 +<36: not append> 30 +<37: append chain> 100 +<38: append chain> 100 +<38: append chain> 100 +<38: append chain> 200 +creating MergeTree without ORDER BY failed, as expected diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh similarity index 54% rename from tests/queries/0_stateless/02932_refreshable_materialized_views.sh rename to tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index d5b07287936..b62ea9fb638 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -15,172 +15,14 @@ CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_ma $CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" -# Basic refreshing. -$CLICKHOUSE_CLIENT -nq " - create materialized view a - refresh after 2 second - engine Memory - empty - as select number as x from numbers(2) union all select rand64() as x; - select '<1: created view>', view, remaining_dependencies, exception, last_refresh_result in ('Unknown', 'Finished') from refreshes; - show create a;" -# Wait for any refresh. (xargs trims the string and turns \t and \n into spaces) -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" == 'Unknown' ] -do - sleep 0.1 -done -start_time="`$CLICKHOUSE_CLIENT -nq "select reinterpret(now64(), 'Int64')"`" -# Check table contents. -$CLICKHOUSE_CLIENT -nq "select '<2: refreshed>', count(), sum(x=0), sum(x=1) from a" -# Wait for table contents to change. -res1="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values'`" -while : -do - res2="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res2" == "$res1" ] || break - sleep 0.1 -done -# Wait for another change. -while : -do - res3="`$CLICKHOUSE_CLIENT -nq 'select * from a order by x format Values -- $LINENO'`" - [ "$res3" == "$res2" ] || break - sleep 0.1 -done -# Check that the two changes were at least 1 second apart, in particular that we're not refreshing -# like crazy. This is potentially flaky, but we need at least one test that uses non-mocked timer -# to make sure the clock+timer code works at all. If it turns out flaky, increase refresh period above. -$CLICKHOUSE_CLIENT -nq " - select '<3: time difference at least>', min2(reinterpret(now64(), 'Int64') - $start_time, 1000); - select '<4: next refresh in>', next_refresh_time-last_refresh_time from refreshes;" - -# Create a source table from which views will read. -$CLICKHOUSE_CLIENT -nq " - create table src (x Int8) engine Memory as select 1;" - -# Switch to fake clock, change refresh schedule, change query. -$CLICKHOUSE_CLIENT -nq " - system test view a set fake time '2050-01-01 00:00:01'; - system wait view a; - system refresh view a; - system wait view a; - select '<4.1: fake clock>', status, last_refresh_time, next_refresh_time from refreshes; - alter table a modify refresh every 2 year; - alter table a modify query select x*2 as x from src; - select '<4.5: altered>', status, last_refresh_result, next_refresh_time from refreshes; - show create a;" -# Advance time to trigger the refresh. -$CLICKHOUSE_CLIENT -nq " - select '<5: no refresh>', count() from a; - system test view a set fake time '2052-02-03 04:05:06';" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_time from refreshes -- $LINENO" | xargs`" != '2052-02-03 04:05:06' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<6: refreshed>', * from a; - select '<7: refreshed>', status, last_refresh_result, next_refresh_time from refreshes;" - -# Create a dependent view, refresh it once. -$CLICKHOUSE_CLIENT -nq " - create materialized view b refresh every 2 year depends on a (y Int32) engine MergeTree order by y empty as select x*10 as y from a; - show create b; - system test view b set fake time '2052-11-11 11:11:11'; - system refresh view b; - system wait view b; - select '<7.5: created dependent>', last_refresh_time from refreshes where view = 'b';" -# Next refresh shouldn't start until the dependency refreshes. -$CLICKHOUSE_CLIENT -nq " - select '<8: refreshed>', * from b; - select '<9: refreshed>', view, status, last_refresh_result, next_refresh_time from refreshes; - system test view b set fake time '2054-01-24 23:22:21';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies 2054-01-01 00:00:00' ] -do - sleep 0.1 -done - -# Drop the source table, check that refresh fails and doesn't leave a temp table behind. -$CLICKHOUSE_CLIENT -nq " - select '<9.2: dropping>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase(); - drop table src; - system refresh view a;" -$CLICKHOUSE_CLIENT -nq "system wait view a;" 2>/dev/null && echo "SYSTEM WAIT VIEW failed to fail at $LINENO" -$CLICKHOUSE_CLIENT -nq " - select '<9.4: dropped>', countIf(name like '%tmp%'), countIf(name like '%.inner%') from system.tables where database = currentDatabase();" - -# Create the source table again, check that refresh succeeds (in particular that tables are looked -# up by name rather than uuid). -$CLICKHOUSE_CLIENT -nq " - select '<10: creating>', view, status, remaining_dependencies, next_refresh_time from refreshes; - create table src (x Int16) engine Memory as select 2; - system test view a set fake time '2054-01-01 00:00:01';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled' ] -do - sleep 0.1 -done -# Both tables should've refreshed. -$CLICKHOUSE_CLIENT -nq " - select '<11: chain-refreshed a>', * from a; - select '<12: chain-refreshed b>', * from b; - select '<13: chain-refreshed>', view, status, remaining_dependencies, last_refresh_result, last_refresh_time, next_refresh_time, exception == '' from refreshes;" - -# Make the dependent table run ahead by one refresh cycle, make sure it waits for the dependency to -# catch up to the same cycle. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2059-01-01 00:00:00'; - system refresh view b;" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2060-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2061-01-01 00:00:00'; - system test view a set fake time '2057-01-01 00:00:00';" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, next_refresh_time from refreshes -- $LINENO" | xargs`" != 'Scheduled 2058-01-01 00:00:00 WaitingForDependencies 2060-01-01 00:00:00' ] -do - sleep 0.1 -done -sleep 1 -$CLICKHOUSE_CLIENT -nq " - select '<14: waiting for next cycle>', view, status, remaining_dependencies, next_refresh_time from refreshes; - truncate src; - insert into src values (3); - system test view a set fake time '2060-02-02 02:02:02';" -while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != '2062-01-01 00:00:00' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<15: chain-refreshed a>', * from a; - select '<16: chain-refreshed b>', * from b; - select '<17: chain-refreshed>', view, status, next_refresh_time from refreshes;" - -# Get to WaitingForDependencies state and remove the depencency. -$CLICKHOUSE_CLIENT -nq " - system test view b set fake time '2062-03-03 03:03:03'" -while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'b' -- $LINENO" | xargs`" != 'WaitingForDependencies' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - alter table b modify refresh every 2 year" -while [ "`$CLICKHOUSE_CLIENT -nq "select status, last_refresh_time from refreshes where view = 'b' -- $LINENO" | xargs`" != 'Scheduled 2062-03-03 03:03:03' ] -do - sleep 0.1 -done -$CLICKHOUSE_CLIENT -nq " - select '<18: removed dependency>', view, status, remaining_dependencies, last_refresh_time,next_refresh_time, refresh_count from refreshes where view = 'b'; - show create b;" - # Select from a table that doesn't exist, get an exception. $CLICKHOUSE_CLIENT -nq " - drop table a; - drop table b; + create table src (x Int8) engine Memory as select 1; create materialized view c refresh every 1 second (x Int64) engine Memory empty as select * from src; drop table src;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Error' ] do - sleep 0.1 + sleep 0.5 done # Check exception, create src, expect successful refresh. $CLICKHOUSE_CLIENT -nq " @@ -189,7 +31,7 @@ $CLICKHOUSE_CLIENT -nq " system refresh view c;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done # Rename table. $CLICKHOUSE_CLIENT -nq " @@ -207,7 +49,7 @@ $CLICKHOUSE_CLIENT -nq " create materialized view e refresh every 1 second (x Int64) engine MergeTree order by x empty as select x + sleepEachRow(1) as x from src settings max_block_size = 1;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done # Stop refreshes. $CLICKHOUSE_CLIENT -nq " @@ -215,7 +57,7 @@ $CLICKHOUSE_CLIENT -nq " system stop view e;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Disabled' ] do - sleep 0.1 + sleep 0.5 done # Make refreshes slow, wait for a slow refresh to start. (We stopped refreshes first to make sure # we wait for a slow refresh, not a previous fast one.) @@ -224,7 +66,7 @@ $CLICKHOUSE_CLIENT -nq " system start view e;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] do - sleep 0.1 + sleep 0.5 done # Rename. $CLICKHOUSE_CLIENT -nq " @@ -238,7 +80,7 @@ $CLICKHOUSE_CLIENT -nq " system cancel view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Cancelled' ] do - sleep 0.1 + sleep 0.5 done # Check that another refresh doesn't immediately start after the cancelled one. sleep 1 @@ -247,7 +89,7 @@ $CLICKHOUSE_CLIENT -nq " system refresh view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes -- $LINENO" | xargs`" != 'Running' ] do - sleep 0.1 + sleep 0.5 done # Drop. $CLICKHOUSE_CLIENT -nq " @@ -262,7 +104,7 @@ $CLICKHOUSE_CLIENT -nq " system test view g set fake time '2050-02-03 15:30:13';" while [ "`$CLICKHOUSE_CLIENT -nq "select next_refresh_time > '2049-01-01' from refreshes -- $LINENO" | xargs`" != '1' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " with '2050-02-10 04:00:00'::DateTime as expected @@ -278,14 +120,14 @@ $CLICKHOUSE_CLIENT -nq " show create h;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes -- $LINENO" | xargs`" != 'Finished' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<30: to existing table>', * from dest; insert into src values (2);" while [ "`$CLICKHOUSE_CLIENT -nq "select count() from dest -- $LINENO" | xargs`" != '2' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<31: to existing table>', * from dest; @@ -304,7 +146,7 @@ $CLICKHOUSE_CLIENT -nq " drop table src2;" while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result, retry from refreshes -- $LINENO" | xargs`" != 'Finished 0' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<31.6: did retry>', x from h2; @@ -316,7 +158,7 @@ $CLICKHOUSE_CLIENT -nq " create materialized view j refresh after 1 year engine Memory as select number as x from numbers(2);" while [ "`$CLICKHOUSE_CLIENT -nq "select sum(last_success_time is null) from refreshes -- $LINENO" | xargs`" == '2' ] do - sleep 0.1 + sleep 0.5 done $CLICKHOUSE_CLIENT -nq " select '<32: empty>', view, status, last_refresh_result, retry from refreshes order by view; From fcb01277e143011154c659b9461891e6a28017c7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 1 Jul 2024 20:25:16 +0000 Subject: [PATCH 23/45] July --- src/Core/SettingsChangesHistory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b0c4f256e45..c20a1dee3c2 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -87,6 +87,7 @@ namespace SettingsChangesHistory static const std::map settings_changes_history = { {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, + {"allow_materialized_view_with_bad_select", true, false, "Stricter validation in CREATE MATERIALIZED VIEW"}, }}, {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, @@ -117,7 +118,6 @@ static const std::map Date: Tue, 2 Jul 2024 03:10:26 +0000 Subject: [PATCH 24/45] Fix unrelated setting --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2806ccf98dc..c17e6642ac2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list Date: Thu, 4 Jul 2024 18:22:58 +0000 Subject: [PATCH 25/45] Conflict --- .../02933_replicated_database_forbid_create_as_select.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh index 15f169d880f..34df0544e13 100755 --- a/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh +++ b/tests/queries/0_stateless/02933_replicated_database_forbid_create_as_select.sh @@ -12,14 +12,14 @@ ${CLICKHOUSE_CLIENT} --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = # Non-replicated engines are allowed ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test (id UInt64) ENGINE = MergeTree() ORDER BY id AS SELECT 1" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv (id UInt64) ENGINE = MergeTree() ORDER BY id POPULATE AS SELECT 1" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv (id UInt64) ENGINE = MergeTree() ORDER BY id POPULATE AS SELECT 1 AS id" # Replicated storafes are forbidden ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" |& grep -cm1 "SUPPORT_IS_DISABLED" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1 AS id" |& grep -cm1 "SUPPORT_IS_DISABLED" # But it is allowed with the special setting ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.test2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id AS SELECT 1" --database_replicated_allow_heavy_create=1 -${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1" --database_replicated_allow_heavy_create=1 +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --query "CREATE MATERIALIZED VIEW ${CLICKHOUSE_DATABASE}_db.test_mv2 (id UInt64) ENGINE = ReplicatedMergeTree ORDER BY id POPULATE AS SELECT 1 AS id" --database_replicated_allow_heavy_create=1 ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" From b337474e05a39b9d30aa8202be40a4ab71718884 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 30 Jul 2024 20:44:19 +0000 Subject: [PATCH 26/45] Fix flaky test --- .../02932_refreshable_materialized_views_2.reference | 2 +- .../0_stateless/02932_refreshable_materialized_views_2.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference index eb4a0498260..cdaad32de0a 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference @@ -5,7 +5,7 @@ <23: simple refresh> 1 <24: rename during refresh> 1 <25: rename during refresh> f Running -<27: cancelled> f Scheduled +<27: cancelled> f Scheduled Cancelled <28: drop during refresh> 0 0 CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x <29: randomize> 1 1 diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index cf0e61fb6a5..2a803114842 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -78,13 +78,13 @@ $CLICKHOUSE_CLIENT -nq " # Cancel. $CLICKHOUSE_CLIENT -nq " system cancel view f;" -while [ "`$CLICKHOUSE_CLIENT -nq "select last_refresh_result from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Cancelled' ] +while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Scheduled' ] do sleep 0.5 done # Check that another refresh doesn't immediately start after the cancelled one. $CLICKHOUSE_CLIENT -nq " - select '<27: cancelled>', view, status from refreshes where view = 'f'; + select '<27: cancelled>', view, status, last_refresh_result from refreshes where view = 'f'; system refresh view f;" while [ "`$CLICKHOUSE_CLIENT -nq "select status from refreshes where view = 'f' -- $LINENO" | xargs`" != 'Running' ] do From 67556a594bb12b944ce4c94cacf9164e92965dfe Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 5 Aug 2024 23:07:53 +0000 Subject: [PATCH 27/45] Conflict --- src/Core/SettingsChangesHistory.cpp | 263 +--------------------------- 1 file changed, 1 insertion(+), 262 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8a40ad94645..dce51b30382 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,268 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"allow_materialized_view_with_bad_select", true, false, "Stricter validation in CREATE MATERIALIZED VIEW"}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"restore_replace_external_table_functions_to_null", false, false, "New setting."}, - {"restore_replace_external_engines_to_null", false, false, "New setting."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"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"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"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"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -340,6 +78,7 @@ static std::initializer_list Date: Tue, 6 Aug 2024 19:38:54 +0200 Subject: [PATCH 28/45] Add keeper error description to the message --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2d826c6c2df..b85c7145a91 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -800,7 +800,8 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto res = future.get(); if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(res.error, "Failed to create new nodes {} at {}", res.path_created, zookeeper_path); + throw Coordination::Exception(res.error, "Failed to create new nodes {} at {} with error {}", + res.path_created, zookeeper_path, Coordination::errorMessage(res.error)); } } From 0882b810a753db4ed7643a2fed3f94fea785ae4d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Aug 2024 01:04:37 +0000 Subject: [PATCH 29/45] try to make bigger steps for execution of merges --- src/Storages/MergeTree/MergeTask.cpp | 54 ++++++++++++++-------- src/Storages/MergeTree/MergeTask.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + 3 files changed, 37 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5f178f08ec3..3203120c6eb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "base/types.h" #include #include #include @@ -512,11 +513,20 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { - Block block; - if (!ctx->is_cancelled() && (global_ctx->merging_executor->pull(block))) - { - global_ctx->rows_written += block.rows(); + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + do + { + Block block; + + if (ctx->is_cancelled() || !global_ctx->merging_executor->pull(block)) + { + finalize(); + return false; + } + + global_ctx->rows_written += block.rows(); const_cast(*global_ctx->to).write(block); UInt64 result_rows = 0; @@ -536,11 +546,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() global_ctx->space_reservation->update(static_cast((1. - progress) * ctx->initial_reservation)); } + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void MergeTask::ExecuteAndFinalizeHorizontalPart::finalize() const +{ global_ctx->merging_executor.reset(); global_ctx->merged_pipeline.reset(); @@ -550,14 +563,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() if (ctx->need_remove_expired_values && global_ctx->ttl_merges_blocker->isCancelled()) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts with expired TTL"); - const auto data_settings = global_ctx->data->getSettings(); const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; - ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); - - return false; + ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *global_ctx->data->getSettings()); } - bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const { /// No need to execute this part if it is horizontal merge. @@ -734,17 +743,24 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { - Block block; - if (!global_ctx->merges_blocker->isCancelled() && !global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) - && ctx->executor->pull(block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block block; + + if (global_ctx->merges_blocker->isCancelled() + || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed) + || !ctx->executor->pull(block)) + return false; + ctx->column_elems_written += block.rows(); ctx->column_to->write(block); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } - return false; + /// Need execute again + return true; } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 979c85482e5..24917a4cb0e 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -254,6 +254,7 @@ private: bool prepare(); bool executeImpl(); + void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 5ba1988cc5d..e56bb15f7b0 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,6 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From e3fded6c94b7fe81af750a6976d704dbeea7a4a5 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Fri, 9 Aug 2024 15:13:55 +0000 Subject: [PATCH 30/45] Log peration error for zk multi request --- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 064ac2261ec..1a9ed4f1ee7 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1570,7 +1570,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, size_t failed_op_index_, const Coordination::Requests & requests_, const Coordination::Responses & responses_) - : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index_), + : KeeperException(exception_code, "Transaction failed ({}): Op #{}, path", exception_code, failed_op_index_), requests(requests_), responses(responses_), failed_op_index(failed_op_index_) { addMessage(getPathForFirstFailedOp()); From c2185606398526716bf4505b4e359be0beb605cc Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 12 Aug 2024 21:14:42 +0000 Subject: [PATCH 31/45] Don't enable allow_materialized_view_with_bad_select yet, someone has to add support for fixture reuse in test_replicated_database first --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fee60ec7981..cb61f829da3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, false, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index dce51b30382..cae37e7ddde 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,7 @@ static std::initializer_list Date: Mon, 12 Aug 2024 23:06:26 +0000 Subject: [PATCH 32/45] allow_materialized_view_with_bad_select=0 in test --- .../0_stateless/02932_refreshable_materialized_views_2.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh index 2a803114842..50a905576d5 100755 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.sh @@ -10,7 +10,7 @@ CLICKHOUSE_LOG_COMMENT= # Set session timezone to UTC to make all DateTime formatting and parsing use UTC, because refresh # scheduling is done in UTC. CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT" | sed 's/--session_timezone[= ][^ ]*//g'`" -CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --session_timezone Etc/UTC"`" +CLICKHOUSE_CLIENT="`echo "$CLICKHOUSE_CLIENT --allow_experimental_refreshable_materialized_view=1 --allow_materialized_view_with_bad_select=0 --session_timezone Etc/UTC"`" $CLICKHOUSE_CLIENT -nq "create view refreshes as select * from system.view_refreshes where database = '$CLICKHOUSE_DATABASE' order by view" From 761a28502ef79787244ad1da7b15b3fedfb24221 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 13 Aug 2024 05:17:18 +0000 Subject: [PATCH 33/45] Unchange integration test --- tests/integration/test_replicated_database/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index a9b178302f9..60a6e099b22 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -712,7 +712,6 @@ def create_some_tables(db): "distributed_ddl_task_timeout": 0, "allow_experimental_object_type": 1, "allow_suspicious_codecs": 1, - "allow_materialized_view_with_bad_select": 1, } main_node.query(f"CREATE TABLE {db}.t1 (n int) ENGINE=Memory", settings=settings) dummy_node.query( From e302e2ef323045bfa7158c0890594ec6ee75c8f1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 01:15:16 +0000 Subject: [PATCH 34/45] Conflicts --- src/Interpreters/InterpreterSystemQuery.cpp | 3 ++- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/MaterializedView/RefreshTask.h | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index aa69854fee8..21c8b44b374 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -667,7 +667,8 @@ BlockIO InterpreterSystemQuery::execute() task->run(); break; case Type::WAIT_VIEW: - getRefreshTask()->wait(); + for (const auto & task : getRefreshTasks()) + task->wait(); break; case Type::CANCEL_VIEW: for (const auto & task : getRefreshTasks()) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 0b51ae85116..ed5a6652288 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -37,7 +37,7 @@ RefreshTask::RefreshTask( refresh_settings.applyChanges(strategy.settings->changes); } -RefreshTaskHolder RefreshTask::create( +OwnedRefreshTask RefreshTask::create( StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index a17932d34e8..ad9d949e18e 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -26,7 +26,8 @@ public: RefreshTask(StorageMaterializedView * view_, const ASTRefreshStrategy & strategy); /// The only proper way to construct task - static RefreshTaskHolder create( + static OwnedRefreshTask create( + StorageMaterializedView * view, ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); From 3ca00440af8129b64c45cba47f212b8f897b9a54 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 14 Aug 2024 20:51:23 +0000 Subject: [PATCH 35/45] Fix off-by-one inline function info in stack traces --- src/Common/StackTrace.cpp | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 76277cbc993..80329b4aec1 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -248,8 +248,31 @@ void StackTrace::forEachFrame( auto dwarf_it = dwarfs.try_emplace(object->name, object->elf).first; DB::Dwarf::LocationInfo location; - if (dwarf_it->second.findAddress( - uintptr_t(current_frame.physical_addr), location, mode, inline_frames)) + uintptr_t adjusted_addr = uintptr_t(current_frame.physical_addr); + if (i > 0) + { + /// For non-innermost stack frames, the address points to the *next* instruction + /// after the `call` instruction. But we want the line number and inline function + /// information for the `call` instruction. So subtract 1 from the address. + /// Caveats: + /// * The `call` instruction can be longer than 1 byte, so addr-1 is in the middle + /// of the instruction. That's ok for debug info lookup: address ranges in debug + /// info cover the whole instruction. + /// * If the stack trace unwound out of a signal handler, the stack frame just + /// outside the signal didn't do a function call. It was interrupted by signal. + /// There's no `call` instruction, and decrementing the address is incorrect. + /// We may get incorrect line number and inlined functions in this case. + /// Unfortunate. + /// Note that libunwind, when producing this stack trace, knows whether this + /// frame is interrupted by signal or not. We could propagate this information + /// from libunwind to here and avoid subtracting 1 in this case, but currently + /// we don't do this. + /// But we don't do the decrement for findSymbol() below (because `call` is + /// ~never the last instruction of a function), so the function name should be + /// correct for both pre-signal frames and regular frames. + adjusted_addr -= 1; + } + if (dwarf_it->second.findAddress(adjusted_addr, location, mode, inline_frames)) { current_frame.file = location.file.toString(); current_frame.line = location.line; From 9f44cd85925b3d691afd71a9ecb9ccc9ec3bf57a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 14 Aug 2024 22:19:17 +0000 Subject: [PATCH 36/45] fix typo --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6c9a29c5bb9..149f9a3e80b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 100, "Target time to execetion of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From 7e006f658127c4efe4c53c0c79880a6b09955755 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 15 Aug 2024 14:09:48 +0200 Subject: [PATCH 37/45] Update delta lake test --- .../DataLakes/DeltaLakeMetadata.cpp | 5 +-- tests/integration/test_storage_delta/test.py | 32 +++++++++---------- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..30001a59a59 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -262,10 +262,11 @@ struct DeltaLakeMetadataImpl partition_name, file_schema.toNamesAndTypesDescription()); } + LOG_TEST(log, "Partition {} value is {} (data type: {}, file: {})", + partition_name, value, name_and_type->type->getName(), filename); + auto field = getFieldValue(value, name_and_type->type); current_partition_columns.emplace_back(*name_and_type, field); - - LOG_TEST(log, "Partition {} value is {} (for {})", partition_name, value, filename); } } } diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index a595d01e6b3..75a4b6cc221 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -572,7 +572,7 @@ def test_partition_columns(started_cluster): "test" + str(i), datetime.strptime(f"2000-01-0{i}", "%Y-%m-%d"), i, - False, + False if i % 2 == 0 else True, ) ] df = spark.createDataFrame(data=data, schema=schema) @@ -622,15 +622,15 @@ def test_partition_columns(started_cluster): ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{result_file}/', 'minio', 'minio123')""" ) assert ( - """1 test1 2000-01-01 1 false + """1 test1 2000-01-01 1 true 2 test2 2000-01-02 2 false -3 test3 2000-01-03 3 false +3 test3 2000-01-03 3 true 4 test4 2000-01-04 4 false -5 test5 2000-01-05 5 false +5 test5 2000-01-05 5 true 6 test6 2000-01-06 6 false -7 test7 2000-01-07 7 false +7 test7 2000-01-07 7 true 8 test8 2000-01-08 8 false -9 test9 2000-01-09 9 false""" +9 test9 2000-01-09 9 true""" == instance.query(f"SELECT * FROM {TABLE_NAME} ORDER BY b").strip() ) @@ -670,7 +670,7 @@ test9 2000-01-09 9""" "test" + str(i), datetime.strptime(f"2000-01-{i}", "%Y-%m-%d"), i, - False, + False if i % 2 == 0 else True, ) ] df = spark.createDataFrame(data=data, schema=schema) @@ -696,23 +696,23 @@ test9 2000-01-09 9""" assert result == num_rows * 2 assert ( - """1 test1 2000-01-01 1 false + """1 test1 2000-01-01 1 true 2 test2 2000-01-02 2 false -3 test3 2000-01-03 3 false +3 test3 2000-01-03 3 true 4 test4 2000-01-04 4 false -5 test5 2000-01-05 5 false +5 test5 2000-01-05 5 true 6 test6 2000-01-06 6 false -7 test7 2000-01-07 7 false +7 test7 2000-01-07 7 true 8 test8 2000-01-08 8 false -9 test9 2000-01-09 9 false +9 test9 2000-01-09 9 true 10 test10 2000-01-10 10 false -11 test11 2000-01-11 11 false +11 test11 2000-01-11 11 true 12 test12 2000-01-12 12 false -13 test13 2000-01-13 13 false +13 test13 2000-01-13 13 true 14 test14 2000-01-14 14 false -15 test15 2000-01-15 15 false +15 test15 2000-01-15 15 true 16 test16 2000-01-16 16 false -17 test17 2000-01-17 17 false +17 test17 2000-01-17 17 true 18 test18 2000-01-18 18 false""" == instance.query( f""" From 9fec833d95a8d2d2727ab7d16d4de89049db82a2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Aug 2024 13:42:57 +0000 Subject: [PATCH 38/45] fix test 03221_mutation_analyzer_skip_part --- .../03221_mutate_profile_events.sh | 53 +++++++++++++++++++ .../03221_mutate_profile_events.sql | 33 ------------ 2 files changed, 53 insertions(+), 33 deletions(-) create mode 100755 tests/queries/0_stateless/03221_mutate_profile_events.sh delete mode 100644 tests/queries/0_stateless/03221_mutate_profile_events.sql diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.sh b/tests/queries/0_stateless/03221_mutate_profile_events.sh new file mode 100755 index 00000000000..3758db905e0 --- /dev/null +++ b/tests/queries/0_stateless/03221_mutate_profile_events.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-random-settings, no-random-merge-tree-settings + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query " + DROP TABLE IF EXISTS t_mutate_profile_events; + + CREATE TABLE t_mutate_profile_events (key UInt64, id UInt64, v1 UInt64, v2 UInt64) + ENGINE = MergeTree ORDER BY id PARTITION BY key + SETTINGS min_bytes_for_wide_part = 0; + + INSERT INTO t_mutate_profile_events SELECT 1, number, number, number FROM numbers(10000); + INSERT INTO t_mutate_profile_events SELECT 2, number, number, number FROM numbers(10000); + + SET mutations_sync = 2; + + ALTER TABLE t_mutate_profile_events UPDATE v1 = 1000 WHERE key = 1; + ALTER TABLE t_mutate_profile_events DELETE WHERE key = 2 AND v2 % 10 = 0; +" + +# Mutation query may return before the entry is added to part log. +# So, we may have to retry the flush of logs until all entries are actually flushed. +for _ in {1..10}; do + ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" + res=$(${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.part_log WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart'") + + if [[ $res -eq 4 ]]; then + break + fi + + sleep 2.0 +done + +${CLICKHOUSE_CLIENT} --query " + SELECT + splitByChar('_', part_name)[-1] AS version, + sum(ProfileEvents['MutationTotalParts']), + sum(ProfileEvents['MutationUntouchedParts']), + sum(ProfileEvents['MutatedRows']), + sum(ProfileEvents['MutatedUncompressedBytes']), + sum(ProfileEvents['MutationAllPartColumns']), + sum(ProfileEvents['MutationSomePartColumns']), + sum(ProfileEvents['MutationTotalMilliseconds']) > 0, + sum(ProfileEvents['MutationExecuteMilliseconds']) > 0, + FROM system.part_log + WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart' + GROUP BY version ORDER BY version; + + DROP TABLE IF EXISTS t_mutate_profile_events; +" diff --git a/tests/queries/0_stateless/03221_mutate_profile_events.sql b/tests/queries/0_stateless/03221_mutate_profile_events.sql deleted file mode 100644 index e9f7f9670bd..00000000000 --- a/tests/queries/0_stateless/03221_mutate_profile_events.sql +++ /dev/null @@ -1,33 +0,0 @@ --- Tags: no-random-settings, no-random-merge-tree-settings - -DROP TABLE IF EXISTS t_mutate_profile_events; - -CREATE TABLE t_mutate_profile_events (key UInt64, id UInt64, v1 UInt64, v2 UInt64) -ENGINE = MergeTree ORDER BY id PARTITION BY key -SETTINGS min_bytes_for_wide_part = 0; - -INSERT INTO t_mutate_profile_events SELECT 1, number, number, number FROM numbers(10000); -INSERT INTO t_mutate_profile_events SELECT 2, number, number, number FROM numbers(10000); - -SET mutations_sync = 2; - -ALTER TABLE t_mutate_profile_events UPDATE v1 = 1000 WHERE key = 1; -ALTER TABLE t_mutate_profile_events DELETE WHERE key = 2 AND v2 % 10 = 0; - -SYSTEM FLUSH LOGS; - -SELECT - splitByChar('_', part_name)[-1] AS version, - sum(ProfileEvents['MutationTotalParts']), - sum(ProfileEvents['MutationUntouchedParts']), - sum(ProfileEvents['MutatedRows']), - sum(ProfileEvents['MutatedUncompressedBytes']), - sum(ProfileEvents['MutationAllPartColumns']), - sum(ProfileEvents['MutationSomePartColumns']), - sum(ProfileEvents['MutationTotalMilliseconds']) > 0, - sum(ProfileEvents['MutationExecuteMilliseconds']) > 0, -FROM system.part_log -WHERE database = currentDatabase() AND table = 't_mutate_profile_events' AND event_type = 'MutatePart' -GROUP BY version ORDER BY version; - -DROP TABLE IF EXISTS t_mutate_profile_events From 4548957f5a73600d227dcc26926c7678dd284c5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Aug 2024 15:07:07 +0000 Subject: [PATCH 39/45] Add default status to not-prepared processors. --- src/Processors/Executors/ExecutingGraph.cpp | 4 ++-- src/Processors/Executors/ExecutingGraph.h | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 2 +- src/Processors/IProcessor.cpp | 7 +++++-- src/Processors/IProcessor.h | 2 +- 5 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 6d5b60d8159..02ae8af5f52 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -279,7 +279,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue try { auto & processor = *node.processor; - IProcessor::Status last_status = node.last_processor_status; + const auto last_status = node.last_processor_status; IProcessor::Status status = processor.prepare(node.updated_input_ports, node.updated_output_ports); node.last_processor_status = status; @@ -319,7 +319,7 @@ bool ExecutingGraph::updateNode(uint64_t pid, Queue & queue, Queue & async_queue node.updated_input_ports.clear(); node.updated_output_ports.clear(); - switch (node.last_processor_status) + switch (*node.last_processor_status) { case IProcessor::Status::NeedData: case IProcessor::Status::PortFull: diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index 71dcd360a2c..6f91c120d47 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -92,7 +92,7 @@ public: std::exception_ptr exception; /// Last state for profiling. - IProcessor::Status last_processor_status = IProcessor::Status::NeedData; + std::optional last_processor_status; /// Ports which have changed their state since last processor->prepare() call. /// They changed when neighbour processors interact with connected ports. diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 82cad471a29..d4630f21688 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -432,7 +432,7 @@ String PipelineExecutor::dumpPipeline() const } } - std::vector statuses; + std::vector> statuses; std::vector proc_list; statuses.reserve(graph->nodes.size()); proc_list.reserve(graph->nodes.size()); diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index edb4d662d8b..fc595a7b565 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -55,9 +55,12 @@ void IProcessor::dump() const } -std::string IProcessor::statusToName(Status status) +std::string IProcessor::statusToName(std::optional status) { - switch (status) + if (status == std::nullopt) + return "NotStarted"; + + switch (*status) { case Status::NeedData: return "NeedData"; diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index f1ce044d92f..02b8a3daa28 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -162,7 +162,7 @@ public: ExpandPipeline, }; - static std::string statusToName(Status status); + static std::string statusToName(std::optional status); /** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations. * From b8055eb346c1eabe006d33e905134c58eaac66e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Aug 2024 17:03:14 +0000 Subject: [PATCH 40/45] Fixing test. --- .../0_stateless/02210_processors_profile_log.reference | 4 ++-- tests/queries/0_stateless/02210_processors_profile_log.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log.reference b/tests/queries/0_stateless/02210_processors_profile_log.reference index 035bd9897ad..12ba17103da 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log.reference @@ -6,6 +6,6 @@ ExpressionTransform ExpressionTransform 1 1 1 1 1 LazyOutputFormat 1 1 1 0 0 LimitsCheckingTransform 1 1 1 1 1 -NullSource 1 0 0 0 0 -NullSource 1 0 0 0 0 +NullSource 0 0 0 0 0 +NullSource 0 0 0 0 0 SourceFromSingleChunk 1 0 0 1 1 diff --git a/tests/queries/0_stateless/02210_processors_profile_log.sql b/tests/queries/0_stateless/02210_processors_profile_log.sql index 75e5bcbb585..a850f4312b3 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.sql +++ b/tests/queries/0_stateless/02210_processors_profile_log.sql @@ -20,8 +20,8 @@ SELECT -- SourceFromSingleChunk, that feed data to ExpressionTransform, -- will feed first block and then wait in PortFull. name = 'SourceFromSingleChunk', output_wait_elapsed_us >= 0.9e6 ? 1 : output_wait_elapsed_us, - -- NullSource/LazyOutputFormatLazyOutputFormat are the outputs - -- so they cannot starts to execute before sleep(1) will be executed. + -- LazyOutputFormatLazyOutputFormat is the output + -- so it cannot starts to execute before sleep(1) will be executed. input_wait_elapsed_us>=1e6 ? 1 : input_wait_elapsed_us) elapsed, input_rows, From 75f951dae57c4de27cdba28792622a35597cf31c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Aug 2024 17:59:31 +0000 Subject: [PATCH 41/45] limit task time for mutations --- src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 88 +++++++++++++--------- 3 files changed, 57 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ee15bc7f711..26cb821f33b 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -9,7 +9,6 @@ #include #include #include -#include "base/types.h" #include #include #include @@ -521,7 +520,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::execute() bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { @@ -751,7 +750,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const bool MergeTask::VerticalMergeStage::executeVerticalMergeForOneColumn() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); - UInt64 step_time_ms = global_ctx->data->getSettings()->merge_preferred_step_execution_time_ms.totalMilliseconds(); + UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); do { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 149f9a3e80b..de1f0f60cfc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -84,7 +84,7 @@ struct Settings; M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, merge_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge. Can be exceeded if one step takes longer time", 0) \ + M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 14c274d7f64..40648439887 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1257,6 +1257,8 @@ public: private: void prepare(); bool mutateOriginalPartAndPrepareProjections(); + void writeTempProjectionPart(size_t projection_idx, Chunk chunk); + void finalizeTempProjections(); bool iterateThroughAllProjections(); void constructTaskForProjectionPartsMerge(); void finalize(); @@ -1307,10 +1309,22 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { - Block cur_block; - Block projection_header; - if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + UInt64 step_time_ms = ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); + + do { + Block cur_block; + Block projection_header; + + MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry); + + if (!ctx->mutating_executor->pull(cur_block)) + { + finalizeTempProjections(); + return false; + } + if (ctx->minmax_idx) ctx->minmax_idx->update(cur_block, MergeTreeData::getMinMaxColumnsNames(ctx->metadata_snapshot->getPartitionKey())); @@ -1322,46 +1336,56 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; + Chunk squashed_chunk; - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - - Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); - if (squashed_chunk) { - auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - tmp_part.finalize(); - tmp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + ProfileEventTimeIncrement projection_watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + Block block_to_squash = ctx->projections_to_build[i]->calculate(cur_block, ctx->context); + + projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); + squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); } + + if (squashed_chunk) + writeTempProjectionPart(i, std::move(squashed_chunk)); } (*ctx->mutate_entry)->rows_written += cur_block.rows(); (*ctx->mutate_entry)->bytes_written_uncompressed += cur_block.bytes(); + } while (watch.elapsedMilliseconds() < step_time_ms); - /// Need execute again - return true; - } + /// Need execute again + return true; +} +void PartMergerWriter::writeTempProjectionPart(size_t projection_idx, Chunk chunk) +{ + const auto & projection = *ctx->projections_to_build[projection_idx]; + const auto & projection_plan = projection_squashes[projection_idx]; + + auto result = projection_plan.getHeader().cloneWithColumns(chunk.detachColumns()); + + auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( + *ctx->data, + ctx->log, + result, + projection, + ctx->new_data_part.get(), + ++block_num); + + tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); + projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); +} + +void PartMergerWriter::finalizeTempProjections() +{ // Write the last block for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - const auto & projection = *ctx->projections_to_build[i]; - auto & projection_squash_plan = projection_squashes[i]; - auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + auto squashed_chunk = Squashing::squash(projection_squashes[i].flush()); if (squashed_chunk) - { - auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); - auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); - temp_part.finalize(); - temp_part.part->getDataPartStorage().commitTransaction(); - projection_parts[projection.name].emplace_back(std::move(temp_part.part)); - } + writeTempProjectionPart(i, std::move(squashed_chunk)); } projection_parts_iterator = std::make_move_iterator(projection_parts.begin()); @@ -1369,12 +1393,8 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() /// Maybe there are no projections ? if (projection_parts_iterator != std::make_move_iterator(projection_parts.end())) constructTaskForProjectionPartsMerge(); - - /// Let's move on to the next stage - return false; } - void PartMergerWriter::constructTaskForProjectionPartsMerge() { auto && [name, parts] = *projection_parts_iterator; From fb7afa779c7ea7d56c67da10a634133fe8416f97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 16 Aug 2024 03:59:17 +0200 Subject: [PATCH 42/45] Support expressions with tuples like `expr().name` --- src/Parsers/ExpressionListParsers.cpp | 17 ++++++++++++++--- .../03224_tuple_element_identifier.reference | 4 ++++ .../03224_tuple_element_identifier.sql | 13 +++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.reference create mode 100644 tests/queries/0_stateless/03224_tuple_element_identifier.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index d38dc6d5f37..ad6b8e13ea6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2811,8 +2811,8 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po if (op.type == OperatorType::TupleElement) { ASTPtr tmp; - if (asterisk_parser.parse(pos, tmp, expected) || - columns_matcher_parser.parse(pos, tmp, expected)) + if (asterisk_parser.parse(pos, tmp, expected) + || columns_matcher_parser.parse(pos, tmp, expected)) { if (auto * asterisk = tmp->as()) { @@ -2833,6 +2833,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.back()->pushOperand(std::move(tmp)); return Action::OPERATOR; } + + /// If it is an identifier, + /// replace it with literal, because an expression `expr().elem` + /// should be transformed to `tupleElement(expr(), 'elem')` for query analysis, + /// otherwise the identifier `elem` will not be found. + if (ParserIdentifier().parse(pos, tmp, expected)) + { + layers.back()->pushOperator(op); + layers.back()->pushOperand(std::make_shared(tmp->as()->name())); + return Action::OPERATOR; + } } /// isNull & isNotNull are postfix unary operators @@ -2863,7 +2874,7 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po layers.push_back(std::make_unique()); if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween) - layers.back()->between_counter++; + ++layers.back()->between_counter; return Action::OPERAND; } diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.reference b/tests/queries/0_stateless/03224_tuple_element_identifier.reference new file mode 100644 index 00000000000..0fc9e7410c1 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.reference @@ -0,0 +1,4 @@ +([('wtf')]) [('wtf')] wtf +([('wtf')]) [('wtf')] wtf +Hello +('Hello') Hello Hello Hello diff --git a/tests/queries/0_stateless/03224_tuple_element_identifier.sql b/tests/queries/0_stateless/03224_tuple_element_identifier.sql new file mode 100644 index 00000000000..2a7fb9a97a3 --- /dev/null +++ b/tests/queries/0_stateless/03224_tuple_element_identifier.sql @@ -0,0 +1,13 @@ +SET enable_analyzer = 1; + +SELECT JSONExtract('{"hello":[{"world":"wtf"}]}', 'Tuple(hello Array(Tuple(world String)))') AS x, + x.hello, x.hello[1].world; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].` wow `; + +SELECT JSONExtract('{"hello":[{" wow ":"wtf"}]}', 'Tuple(hello Array(Tuple(` wow ` String)))') AS x, + x.hello, x.hello[1].`wow`; -- { serverError NOT_FOUND_COLUMN_IN_BLOCK } + +SELECT ('Hello' AS world,).world; +SELECT ('Hello' AS world,) AS t, t.world, (t).world, identity(t).world; From 1b52466cbbbbccad74ca9c92c650758c1dfdb8d2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 16 Aug 2024 05:36:03 +0000 Subject: [PATCH 43/45] () --- src/Common/StackTrace.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 80329b4aec1..bd01b639913 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -267,7 +267,7 @@ void StackTrace::forEachFrame( /// frame is interrupted by signal or not. We could propagate this information /// from libunwind to here and avoid subtracting 1 in this case, but currently /// we don't do this. - /// But we don't do the decrement for findSymbol() below (because `call` is + /// But we don't do the decrement for findSymbol below (because `call` is /// ~never the last instruction of a function), so the function name should be /// correct for both pre-signal frames and regular frames. adjusted_addr -= 1; From b1c18e2dd42051fb045431278eceae9dd22563bd Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 13:29:25 +0000 Subject: [PATCH 44/45] Fix check for allowed date symbols in date time best effort schema infernece --- src/IO/parseDateTimeBestEffort.cpp | 4 ++-- tests/queries/0_stateless/03222_date_time_inference.reference | 1 + tests/queries/0_stateless/03222_date_time_inference.sql | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 8d798bf725c..03429b46b2a 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -350,7 +350,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month && !day_of_month) day_of_month = hour_or_day_of_month_or_month; } - else if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + else if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (day_of_month) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: day of month is duplicated"); @@ -399,7 +399,7 @@ ReturnType parseDateTimeBestEffortImpl( if (month > 12) std::swap(month, day_of_month); - if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) + if ((!in.eof() && isSymbolIn(*in.position(), allowed_date_delimiters)) && (checkChar('/', in) || checkChar('.', in) || checkChar('-', in))) { if (year) return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: year component is duplicated"); diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 3288308a1d0..838b103f106 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -251,3 +251,4 @@ Mar 2000 00:00:00.000 String 2000 00:00:00.000 String Mar 2000-01-01 00:00:00 String Mar 2000-01-01 00:00:00.000 String +1.7.10 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index ebd472294be..b0c4df530cf 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -265,5 +265,5 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 00:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 2000-01-01 00:00:00.000"}'); - +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "1.7.10"}'); From 4975264c9d39d560c32799dcf14bd3f9d40fcfa4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 14:08:07 +0000 Subject: [PATCH 45/45] More strict parsing --- src/IO/parseDateTimeBestEffort.cpp | 23 ++++--- .../03222_date_time_inference.reference | 67 ++++++++++++------- .../0_stateless/03222_date_time_inference.sql | 19 ++++++ 3 files changed, 75 insertions(+), 34 deletions(-) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 03429b46b2a..52bcdc6bbb4 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -180,8 +180,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 10 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<10>(res, digits); @@ -189,8 +189,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 9 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow timestamps"); /// This is unix timestamp. readDecimalNumber<9>(res, digits); @@ -198,8 +198,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 14 && !year && !has_time) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDDhhmmss readDecimalNumber<4>(year, digits); @@ -212,8 +212,8 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 8 && !year) { - if (strict && month) - return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: month component is duplicated"); + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); /// This is YYYYMMDD readDecimalNumber<4>(year, digits); @@ -222,6 +222,9 @@ ReturnType parseDateTimeBestEffortImpl( } else if (num_digits == 6) { + if (strict) + return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Strict best effort parsing doesn't allow date times without separators"); + /// This is YYYYMM or hhmmss if (!year && !month) { @@ -593,8 +596,8 @@ ReturnType parseDateTimeBestEffortImpl( else return on_error(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot read DateTime: unexpected word"); -// while (!in.eof() && isAlphaASCII(*in.position())) -// ++in.position(); + while (!in.eof() && isAlphaASCII(*in.position())) + ++in.position(); /// For RFC 2822 if (has_day_of_week) diff --git a/tests/queries/0_stateless/03222_date_time_inference.reference b/tests/queries/0_stateless/03222_date_time_inference.reference index 838b103f106..221ab1fe5f5 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.reference +++ b/tests/queries/0_stateless/03222_date_time_inference.reference @@ -45,34 +45,34 @@ DateTime/DateTime64 best effort 2000-01-01 00:00:00 DateTime 2000-01-01 01:00:00 DateTime 2000-01-01 01:00:00.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-01 21:02:03 DateTime -2017-01-01 21:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) -2017-01-02 01:02:03 DateTime -2017-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +02/01/17 010203 MSK String +02/01/17 010203.000 MSK String +02/01/17 010203 MSK+0100 String +02/01/17 010203.000 MSK+0100 String +02/01/17 010203 UTC+0300 String +02/01/17 010203.000 UTC+0300 String +02/01/17 010203Z String +02/01/17 010203.000Z String +02/01/1970 010203Z String +02/01/1970 010203.000Z String +02/01/70 010203Z String +02/01/70 010203.000Z String 2018-02-11 03:40:50 DateTime 2018-02-11 03:40:50.000000000 DateTime64(9) 2000-04-17 01:02:03 DateTime 2000-04-17 01:02:03.000000000 DateTime64(9) -1970-01-02 01:00:00 DateTime -1970-01-02 01:00:00.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) -1970-01-02 01:02:03 DateTime -1970-01-02 01:02:03.000000000 DateTime64(9) +19700102 01:00:00 String +19700102 01:00:00.000 String +19700102010203Z String +19700102010203Z.000 String +1970/01/02 010203Z String +1970/01/02 010203.000Z String 2015-12-31 20:00:00 DateTime 2015-12-31 20:00:00 DateTime 2016-01-01 00:00:00 DateTime 2016-01-01 00:00:00 DateTime -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +201701 02 010203 UTC+0300 String +201701 02 010203.000 UTC+0300 String 2017-01-02 03:04:05 DateTime 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-01-02 03:04:05 DateTime @@ -117,8 +117,8 @@ DateTime/DateTime64 best effort 2017-01-02 03:04:05.000000000 DateTime64(9) 2017-04-01 11:22:33 DateTime 2017-04-01 11:22:33.000000000 DateTime64(9) -2017-04-01 22:02:03 DateTime -2017-04-01 22:02:03.000000000 DateTime64(9) +2017 Apr 02 010203 UTC+0300 String +2017 Apr 02 010203.000 UTC+0300 String 2017-04-01 22:02:03 DateTime 2017-04-01 22:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime @@ -143,8 +143,8 @@ DateTime/DateTime64 best effort 2017-04-01 21:02:03.000000000 DateTime64(9) 2017-04-02 01:02:03 DateTime 2017-04-02 01:02:03.000000000 DateTime64(9) -2017-01-01 22:02:03 DateTime -2017-01-01 22:02:03.000000000 DateTime64(9) +2017 Jan 02 010203 UTC+0300 String +2017 Jan 02 010203.000 UTC+0300 String 2017-04-25 01:02:03 DateTime 2017-04-25 01:02:03.000000000 DateTime64(9) 2017-04-25 01:02:03 DateTime @@ -231,6 +231,25 @@ Mar01012020010101 String Mar 01012020010101 String Mar01012020010101.000 String Mar 0101202001010101.000 String +Sun String +Sun1 String +Sun 1 String +Sun01 String +Sun 01 String +Sun2020 String +Sun 2020 String +Sun012020 String +Sun 012020 String +Sun01012020 String +Sun 01012020 String +Sun0101202001 String +Sun 0101202001 String +Sun010120200101 String +Sun 010120200101 String +Sun01012020010101 String +Sun 01012020010101 String +Sun01012020010101.000 String +Sun 0101202001010101.000 String 2000 01 01 01:00:00 String 2000 01 01 01:00:00.000 String 2000a01a01 01:00:00 String diff --git a/tests/queries/0_stateless/03222_date_time_inference.sql b/tests/queries/0_stateless/03222_date_time_inference.sql index b0c4df530cf..b16f72c72f4 100644 --- a/tests/queries/0_stateless/03222_date_time_inference.sql +++ b/tests/queries/0_stateless/03222_date_time_inference.sql @@ -245,6 +245,25 @@ select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 01012020010101"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar01012020010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Mar 0101202001010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 1"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 2020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 010120200101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 01012020010101"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun01012020010101.000"}'); +select x, toTypeName(x) from format(JSONEachRow, '{"x" : "Sun 0101202001010101.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000 01 01 01:00:00.000"}'); select x, toTypeName(x) from format(JSONEachRow, '{"x" : "2000a01a01 01:00:00"}');