From c762898adb2c509289f78d96c1716b8f1b0c7e5e Mon Sep 17 00:00:00 2001 From: koloshmet Date: Sun, 12 Feb 2023 21:17:55 +0200 Subject: [PATCH 01/25] refreshable materialized views --- src/Access/Common/AccessType.h | 1 + src/CMakeLists.txt | 1 + src/Common/CurrentMetrics.cpp | 1 + src/Common/IntervalKind.h | 2 + src/Interpreters/ActionLocksManager.cpp | 1 + src/Interpreters/Context.cpp | 4 + src/Interpreters/Context.h | 4 + src/Interpreters/InterpreterSystemQuery.cpp | 54 ++++ src/Interpreters/InterpreterSystemQuery.h | 3 + src/Parsers/ASTCreateQuery.cpp | 7 +- src/Parsers/ASTCreateQuery.h | 2 + src/Parsers/ASTRefreshStrategy.cpp | 74 +++++ src/Parsers/ASTRefreshStrategy.h | 36 +++ src/Parsers/ASTSystemQuery.h | 8 + src/Parsers/ASTTimeInterval.cpp | 40 +++ src/Parsers/ASTTimeInterval.h | 37 +++ src/Parsers/ParserCreateQuery.cpp | 14 +- src/Parsers/ParserRefreshStrategy.cpp | 77 +++++ src/Parsers/ParserRefreshStrategy.h | 16 + src/Parsers/ParserSystemQuery.cpp | 14 + src/Parsers/ParserTimeInterval.cpp | 71 +++++ src/Parsers/ParserTimeInterval.h | 24 ++ .../Executors/ManualPipelineExecutor.cpp | 55 ++++ .../Executors/ManualPipelineExecutor.h | 27 ++ src/QueryPipeline/QueryPipeline.h | 1 + .../MaterializedView/RefreshAllCombiner.cpp | 58 ++++ .../MaterializedView/RefreshAllCombiner.h | 33 ++ .../MaterializedView/RefreshDependencies.cpp | 60 ++++ .../MaterializedView/RefreshDependencies.h | 56 ++++ src/Storages/MaterializedView/RefreshSet.cpp | 128 ++++++++ src/Storages/MaterializedView/RefreshSet.h | 142 +++++++++ src/Storages/MaterializedView/RefreshTask.cpp | 292 ++++++++++++++++++ src/Storages/MaterializedView/RefreshTask.h | 150 +++++++++ .../MaterializedView/RefreshTask_fwd.h | 15 + .../MaterializedView/RefreshTimers.cpp | 243 +++++++++++++++ src/Storages/MaterializedView/RefreshTimers.h | 69 +++++ .../MaterializedView/tests/gtest_timers.cpp | 27 ++ src/Storages/StorageMaterializedView.cpp | 122 +++++++- src/Storages/StorageMaterializedView.h | 15 + .../System/StorageSystemViewRefreshes.cpp | 67 ++++ .../System/StorageSystemViewRefreshes.h | 27 ++ src/Storages/System/attachSystemTables.cpp | 2 + 42 files changed, 2065 insertions(+), 15 deletions(-) create mode 100644 src/Parsers/ASTRefreshStrategy.cpp create mode 100644 src/Parsers/ASTRefreshStrategy.h create mode 100644 src/Parsers/ASTTimeInterval.cpp create mode 100644 src/Parsers/ASTTimeInterval.h create mode 100644 src/Parsers/ParserRefreshStrategy.cpp create mode 100644 src/Parsers/ParserRefreshStrategy.h create mode 100644 src/Parsers/ParserTimeInterval.cpp create mode 100644 src/Parsers/ParserTimeInterval.h create mode 100644 src/Processors/Executors/ManualPipelineExecutor.cpp create mode 100644 src/Processors/Executors/ManualPipelineExecutor.h create mode 100644 src/Storages/MaterializedView/RefreshAllCombiner.cpp create mode 100644 src/Storages/MaterializedView/RefreshAllCombiner.h create mode 100644 src/Storages/MaterializedView/RefreshDependencies.cpp create mode 100644 src/Storages/MaterializedView/RefreshDependencies.h create mode 100644 src/Storages/MaterializedView/RefreshSet.cpp create mode 100644 src/Storages/MaterializedView/RefreshSet.h create mode 100644 src/Storages/MaterializedView/RefreshTask.cpp create mode 100644 src/Storages/MaterializedView/RefreshTask.h create mode 100644 src/Storages/MaterializedView/RefreshTask_fwd.h create mode 100644 src/Storages/MaterializedView/RefreshTimers.cpp create mode 100644 src/Storages/MaterializedView/RefreshTimers.h create mode 100644 src/Storages/MaterializedView/tests/gtest_timers.cpp create mode 100644 src/Storages/System/StorageSystemViewRefreshes.cpp create mode 100644 src/Storages/System/StorageSystemViewRefreshes.h diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 45d427a7c55..1f0e678461f 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -177,6 +177,7 @@ enum class AccessType M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \ M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \ M(SYSTEM_CLEANUP, "SYSTEM STOP CLEANUP, SYSTEM START CLEANUP", TABLE, SYSTEM) \ + M(SYSTEM_VIEWS, "SYSTEM REFRESH VIEW, START VIEWS, STOP VIEWS, CANCEL VIEW, PAUSE VIEW, RESUME VIEW", VIEW, SYSTEM) \ M(SYSTEM_DISTRIBUTED_SENDS, "SYSTEM STOP DISTRIBUTED SENDS, SYSTEM START DISTRIBUTED SENDS, STOP DISTRIBUTED SENDS, START DISTRIBUTED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP REPLICATED SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 6063c701708..86cb9acd056 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -226,6 +226,7 @@ add_object_library(clickhouse_storages_statistics Storages/Statistics) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) add_object_library(clickhouse_storages_s3queue Storages/S3Queue) +add_object_library(clickhouse_storages_materializedview Storages/MaterializedView) add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 38b14e4b0b4..fccdeaa3c57 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -253,6 +253,7 @@ M(MergeTreeAllRangesAnnouncementsSent, "The current number of announcement being sent in flight from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ M(CreatedTimersInQueryProfiler, "Number of Created thread local timers in QueryProfiler") \ M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \ + M(Refresh, "Number of active refreshes") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 6893286f196..0f45d0ac169 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -71,6 +71,8 @@ struct IntervalKind /// Returns false if the conversion did not succeed. /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); + + auto operator<=>(const IntervalKind & other) const { return kind <=> other.kind; } }; /// NOLINTNEXTLINE diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index fb5ef4b98ae..43b49b024aa 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -18,6 +18,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove = 7; extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; + extern const StorageActionBlockType ViewRefresh = 8; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 25146ebc10d..fda22e4075e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -95,6 +95,7 @@ #include #include #include +#include #include #include #include @@ -289,6 +290,7 @@ struct ContextSharedPart : boost::noncopyable MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; + RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. @@ -825,6 +827,8 @@ MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } +RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } +const RefreshSet & Context::getRefreshSet() const { return shared->refresh_set; } String Context::resolveDatabase(const String & database_name) const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 39d2212ce80..b09eeb8ca2d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -74,6 +74,7 @@ class BackgroundSchedulePool; class MergeList; class MovesList; class ReplicatedFetchList; +class RefreshSet; class Cluster; class Compiler; class MarkCache; @@ -922,6 +923,9 @@ public: ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; + RefreshSet & getRefreshSet(); + const RefreshSet & getRefreshSet() const; + /// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call. /// If no ZooKeeper configured, throws an exception. std::shared_ptr getZooKeeper() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index fc040e2af04..2f504e97857 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -54,6 +54,7 @@ #include #include #include +#include #include #include #include @@ -108,6 +109,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove; extern const StorageActionBlockType PullReplicationLog; extern const StorageActionBlockType Cleanup; + extern const StorageActionBlockType ViewRefresh; } @@ -165,6 +167,8 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type) return AccessType::SYSTEM_PULLING_REPLICATION_LOG; else if (action_type == ActionLocks::Cleanup) return AccessType::SYSTEM_CLEANUP; + else if (action_type == ActionLocks::ViewRefresh) + return AccessType::SYSTEM_VIEWS; else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type: {}", std::to_string(action_type)); } @@ -605,6 +609,30 @@ BlockIO InterpreterSystemQuery::execute() case Type::START_CLEANUP: startStopAction(ActionLocks::Cleanup, true); break; + case Type::START_VIEWS: + startStopAction(ActionLocks::ViewRefresh, true); + break; + case Type::STOP_VIEWS: + startStopAction(ActionLocks::ViewRefresh, false); + break; + case Type::START_VIEW: + startStopAction(ActionLocks::ViewRefresh, true); + break; + case Type::STOP_VIEW: + startStopAction(ActionLocks::ViewRefresh, false); + break; + case Type::REFRESH_VIEW: + getRefreshTask()->run(); + break; + case Type::CANCEL_VIEW: + getRefreshTask()->cancel(); + break; + case Type::PAUSE_VIEW: + getRefreshTask()->pause(); + break; + case Type::RESUME_VIEW: + getRefreshTask()->resume(); + break; case Type::DROP_REPLICA: dropReplica(query); break; @@ -1092,6 +1120,17 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RESTART DISK is not supported"); } +RefreshTaskHolder InterpreterSystemQuery::getRefreshTask() +{ + auto ctx = getContext(); + ctx->checkAccess(AccessType::SYSTEM_VIEWS); + auto task = ctx->getRefreshSet().getTask(table_id); + if (!task) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Refreshable view {} doesn't exist", table_id.getNameForLogs()); + return task; +} + AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() const { @@ -1241,6 +1280,21 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_REPLICATION_QUEUES, query.getDatabase(), query.getTable()); break; } + case Type::REFRESH_VIEW: + case Type::START_VIEW: + case Type::START_VIEWS: + case Type::STOP_VIEW: + case Type::STOP_VIEWS: + case Type::CANCEL_VIEW: + case Type::PAUSE_VIEW: + case Type::RESUME_VIEW: + { + if (!query.table) + required_access.emplace_back(AccessType::SYSTEM_VIEWS); + else + required_access.emplace_back(AccessType::SYSTEM_VIEWS, query.getDatabase(), query.getTable()); + break; + } case Type::DROP_REPLICA: case Type::DROP_DATABASE_REPLICA: { diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 462449623d0..89de7402b4d 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -72,6 +73,8 @@ private: void flushDistributed(ASTSystemQuery & query); [[noreturn]] void restartDisk(String & name); + RefreshTaskHolder getRefreshTask(); + AccessRightsElements getRequiredAccessForDDLOnCluster() const; void startStopAction(StorageActionBlockType action_type, bool start); }; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 1562586bd93..9d5f0bcddbd 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -340,6 +339,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat formatOnCluster(settings); } + if (refresh_strategy) + { + settings.ostr << settings.nl_or_ws; + refresh_strategy->formatImpl(settings, state, frame); + } + if (to_table_id) { assert((is_materialized_view || is_window_view) && to_inner_uuid == UUIDHelpers::Nil); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index 28f5e05802b..49a0140625c 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -116,6 +117,7 @@ public: ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.) + ASTRefreshStrategy * refresh_strategy = nullptr; // For CREATE MATERIALIZED VIEW ... REFRESH ... std::optional live_view_periodic_refresh; /// For CREATE LIVE VIEW ... WITH [PERIODIC] REFRESH ... bool is_watermark_strictly_ascending{false}; /// STRICTLY ASCENDING WATERMARK STRATEGY FOR WINDOW VIEW diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp new file mode 100644 index 00000000000..f168bcc09c1 --- /dev/null +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -0,0 +1,74 @@ +#include + +#include + +namespace DB +{ + +ASTPtr ASTRefreshStrategy::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (interval) + res->set(res->interval, interval->clone()); + if (period) + res->set(res->period, period->clone()); + if (periodic_offset) + res->set(res->periodic_offset, periodic_offset->clone()); + if (spread) + res->set(res->spread, spread->clone()); + if (settings) + res->set(res->settings, settings->clone()); + if (dependencies) + res->set(res->dependencies, dependencies->clone()); + res->interval = interval; + res->spread = spread; + res->schedule_kind = schedule_kind; + return res; +} + +void ASTRefreshStrategy::formatImpl( + const IAST::FormatSettings & f_settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const +{ + frame.need_parens = false; + + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << "REFRESH "; + using enum ScheduleKind; + switch (schedule_kind) + { + case AFTER: + f_settings.ostr << "AFTER "; + interval->formatImpl(f_settings, state, frame); + break; + case EVERY: + f_settings.ostr << "EVERY "; + period->formatImpl(f_settings, state, frame); + if (periodic_offset) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " OFFSET "; + periodic_offset->formatImpl(f_settings, state, frame); + } + break; + default: + break; + } + + if (spread) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " RANDOMIZE FOR "; + spread->formatImpl(f_settings, state, frame); + } + if (dependencies) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " DEPENDS ON "; + dependencies->formatImpl(f_settings, state, frame); + } + if (settings) + { + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS "; + settings->formatImpl(f_settings, state, frame); + } +} + +} diff --git a/src/Parsers/ASTRefreshStrategy.h b/src/Parsers/ASTRefreshStrategy.h new file mode 100644 index 00000000000..0df6a6e0e10 --- /dev/null +++ b/src/Parsers/ASTRefreshStrategy.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Strategy for MATERIALIZED VIEW ... REFRESH .. +class ASTRefreshStrategy : public IAST +{ +public: + enum class ScheduleKind : UInt8 + { + UNKNOWN = 0, + AFTER, + EVERY + }; + + ASTSetQuery * settings = nullptr; + ASTExpressionList * dependencies = nullptr; + ASTTimeInterval * interval = nullptr; + ASTTimePeriod * period = nullptr; + ASTTimeInterval * periodic_offset = nullptr; + ASTTimePeriod * spread = nullptr; + ScheduleKind schedule_kind{ScheduleKind::UNKNOWN}; + + String getID(char) const override { return "Refresh strategy definition"; } + + ASTPtr clone() const override; + + void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 8e6100fe7b4..ec8e47f9513 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -90,6 +90,14 @@ public: STOP_CLEANUP, START_CLEANUP, RESET_COVERAGE, + REFRESH_VIEW, + START_VIEW, + START_VIEWS, + STOP_VIEW, + STOP_VIEWS, + CANCEL_VIEW, + PAUSE_VIEW, + RESUME_VIEW, END }; diff --git a/src/Parsers/ASTTimeInterval.cpp b/src/Parsers/ASTTimeInterval.cpp new file mode 100644 index 00000000000..4edda531202 --- /dev/null +++ b/src/Parsers/ASTTimeInterval.cpp @@ -0,0 +1,40 @@ +#include + +#include + +#include + +namespace DB +{ + +ASTPtr ASTTimePeriod::clone() const +{ + return std::make_shared(*this); +} + +void ASTTimePeriod::formatImpl(const FormatSettings & f_settings, FormatState &, FormatStateStacked frame) const +{ + frame.need_parens = false; + f_settings.ostr << (f_settings.hilite ? hilite_none : "") << value << ' '; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword(); +} + +ASTPtr ASTTimeInterval::clone() const +{ + return std::make_shared(*this); +} + +void ASTTimeInterval::formatImpl(const FormatSettings & f_settings, FormatState &, FormatStateStacked frame) const +{ + frame.need_parens = false; + + for (bool is_first = true; auto [kind, value] : kinds | std::views::reverse) + { + if (!std::exchange(is_first, false)) + f_settings.ostr << ' '; + f_settings.ostr << (f_settings.hilite ? hilite_none : "") << value << ' '; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword(); + } +} + +} diff --git a/src/Parsers/ASTTimeInterval.h b/src/Parsers/ASTTimeInterval.h new file mode 100644 index 00000000000..374d1e1ec55 --- /dev/null +++ b/src/Parsers/ASTTimeInterval.h @@ -0,0 +1,37 @@ + #pragma once + +#include + +#include + +#include + +namespace DB +{ + +/// Simple periodic time interval like 10 SECOND +class ASTTimePeriod : public IAST +{ +public: + UInt64 value{0}; + IntervalKind kind{IntervalKind::Second}; + + String getID(char) const override { return "TimePeriod"; } + + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +/// Compound time interval like 1 YEAR 3 DAY 15 MINUTE +class ASTTimeInterval : public IAST +{ +public: + std::map kinds; + + String getID(char) const override { return "TimeInterval"; } + + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +} diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index f79850467e4..3921a0e37e7 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -1390,6 +1391,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ASTPtr as_database; ASTPtr as_table; ASTPtr select; + ASTPtr refresh_strategy; String cluster_str; bool attach = false; @@ -1436,6 +1438,15 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } + if (ParserKeyword{"REFRESH"}.ignore(pos, expected)) + { + // REFRESH only with materialized views + if (!is_materialized_view) + return false; + if (!ParserRefreshStrategy{}.parse(pos, refresh_strategy, expected)) + return false; + } + if (is_materialized_view && ParserKeyword{"TO INNER UUID"}.ignore(pos, expected)) { ParserStringLiteral literal_p; @@ -1527,6 +1538,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->set(query->columns_list, columns_list); query->set(query->storage, storage); + if (refresh_strategy) + query->set(query->refresh_strategy, refresh_strategy); if (comment) query->set(query->comment, comment); @@ -1535,7 +1548,6 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->set(query->select, select); return true; - } bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp new file mode 100644 index 00000000000..a448556bd4f --- /dev/null +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -0,0 +1,77 @@ +#include + +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto refresh = std::make_shared(); + + if (ParserKeyword{"AFTER"}.ignore(pos, expected)) + { + refresh->schedule_kind = ASTRefreshStrategy::ScheduleKind::AFTER; + ASTPtr interval; + if (!ParserTimeInterval{}.parse(pos, interval, expected)) + return false; + + refresh->set(refresh->interval, interval); + } + else if (ParserKeyword{"EVERY"}.ignore(pos, expected)) + { + refresh->schedule_kind = ASTRefreshStrategy::ScheduleKind::EVERY; + ASTPtr period; + ASTPtr periodic_offset; + if (!ParserTimePeriod{}.parse(pos, period, expected)) + return false; + if (!ParserTimeInterval{}.parse(pos, periodic_offset, expected)) + return false; + + refresh->set(refresh->period, period); + refresh->set(refresh->periodic_offset, periodic_offset); + } + if (refresh->schedule_kind == ASTRefreshStrategy::ScheduleKind::UNKNOWN) + return false; + + if (ParserKeyword{"RANDOMIZE FOR"}.ignore(pos, expected)) + { + ASTPtr spread; + if (!ParserTimePeriod{}.parse(pos, spread, expected)) + return false; + + refresh->set(refresh->spread, spread); + } + + if (ParserKeyword{"DEPENDS ON"}.ignore(pos, expected)) + { + ASTPtr dependencies; + auto list_parser = ParserList{ + std::make_unique(), + std::make_unique(TokenType::Comma), + /* allow_empty= */ false}; + if (!list_parser.parse(pos, dependencies, expected)) + return false; + refresh->set(refresh->dependencies, dependencies); + } + + // Refresh SETTINGS + if (ParserKeyword{"SETTINGS"}.ignore(pos, expected)) + { + /// Settings are written like SET query, so parse them with ParserSetQuery + ASTPtr settings; + if (!ParserSetQuery{true}.parse(pos, settings, expected)) + return false; + refresh->set(refresh->settings, settings); + } + node = refresh; + return true; +} + +} diff --git a/src/Parsers/ParserRefreshStrategy.h b/src/Parsers/ParserRefreshStrategy.h new file mode 100644 index 00000000000..e9edabd7129 --- /dev/null +++ b/src/Parsers/ParserRefreshStrategy.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ + +/// Parser for ASTRefreshStrategy +class ParserRefreshStrategy : public IParserBase +{ +protected: + const char * getName() const override { return "refresh strategy"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 2e1283187d3..9115d195904 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -388,6 +388,20 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & parseDatabaseAndTableAsAST(pos, expected, res->database, res->table); break; + case Type::REFRESH_VIEW: + case Type::START_VIEW: + case Type::STOP_VIEW: + case Type::CANCEL_VIEW: + case Type::PAUSE_VIEW: + case Type::RESUME_VIEW: + if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table)) + return false; + break; + + case Type::START_VIEWS: + case Type::STOP_VIEWS: + break; + case Type::SUSPEND: { if (!parseQueryWithOnCluster(res, pos, expected)) diff --git a/src/Parsers/ParserTimeInterval.cpp b/src/Parsers/ParserTimeInterval.cpp new file mode 100644 index 00000000000..dac66883083 --- /dev/null +++ b/src/Parsers/ParserTimeInterval.cpp @@ -0,0 +1,71 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +struct ValKind +{ + UInt64 val; + IntervalKind kind; + bool empty; +}; + +std::optional parseValKind(IParser::Pos & pos, Expected & expected) +{ + ASTPtr value; + IntervalKind kind; + if (!ParserNumber{}.parse(pos, value, expected)) + return ValKind{ .empty = true }; + if (!parseIntervalKind(pos, expected, kind)) + return {}; + return ValKind{ value->as().value.safeGet(), kind, false }; +} + +} + +bool ParserTimePeriod::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto parsed = parseValKind(pos, expected); + + if (!parsed || parsed->empty || parsed->val == 0) + return false; + + auto time_period = std::make_shared(); + time_period->value = parsed->val; + time_period->kind = parsed->kind; + + node = time_period; + return true; +} + +bool ParserTimeInterval::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto time_interval = std::make_shared(); + + auto parsed = parseValKind(pos, expected); + while (parsed && !parsed->empty) + { + if (parsed->val == 0) + return false; + auto [it, inserted] = time_interval->kinds.emplace(parsed->kind, parsed->val); + if (!inserted) + return false; + parsed = parseValKind(pos, expected); + } + + if (!parsed || time_interval->kinds.empty()) + return false; + node = time_interval; + return true; +} + +} diff --git a/src/Parsers/ParserTimeInterval.h b/src/Parsers/ParserTimeInterval.h new file mode 100644 index 00000000000..6eae1fa4133 --- /dev/null +++ b/src/Parsers/ParserTimeInterval.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB +{ + +/// Parser for ASTTimePeriod +class ParserTimePeriod : public IParserBase +{ +protected: + const char * getName() const override { return "time period"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +/// Parser for ASTTimeInterval +class ParserTimeInterval : public IParserBase +{ +protected: + const char * getName() const override { return "time interval"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} diff --git a/src/Processors/Executors/ManualPipelineExecutor.cpp b/src/Processors/Executors/ManualPipelineExecutor.cpp new file mode 100644 index 00000000000..f3ac5028b77 --- /dev/null +++ b/src/Processors/Executors/ManualPipelineExecutor.cpp @@ -0,0 +1,55 @@ +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ + +QueryPipeline & validatePipeline(QueryPipeline & query_pipeline) +{ + if (!query_pipeline.completed()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for ManualPipelineExecutor must be completed"); + return query_pipeline; +} + +} + +ManualPipelineExecutor::ManualPipelineExecutor(QueryPipeline & query_pipeline) + : pipeline{&validatePipeline(query_pipeline)} + , executor(pipeline->processors, pipeline->process_list_element) +{ + executor.setReadProgressCallback(pipeline->getReadProgressCallback()); +} + +ManualPipelineExecutor::~ManualPipelineExecutor() +{ + try + { + executor.cancel(); + } + catch (...) + { + tryLogCurrentException("ManualPipelineExecutor"); + } +} + +bool ManualPipelineExecutor::executeStep() +{ + return executor.executeStep(); +} + +bool ManualPipelineExecutor::executeStep(std::atomic_bool & yield_flag) +{ + return executor.executeStep(&yield_flag); +} + +} diff --git a/src/Processors/Executors/ManualPipelineExecutor.h b/src/Processors/Executors/ManualPipelineExecutor.h new file mode 100644 index 00000000000..2fc441609b7 --- /dev/null +++ b/src/Processors/Executors/ManualPipelineExecutor.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class QueryPipeline; + +/// Simple executor for step by step execution of completed QueryPipeline +class ManualPipelineExecutor +{ +public: + explicit ManualPipelineExecutor(QueryPipeline & query_pipeline); + ~ManualPipelineExecutor(); + + bool executeStep(); + bool executeStep(std::atomic_bool & yield_flag); + +private: + QueryPipeline * pipeline; + PipelineExecutor executor; +}; + +} diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h index f14cf61aac2..326d2721d9f 100644 --- a/src/QueryPipeline/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -167,6 +167,7 @@ private: friend class PushingAsyncPipelineExecutor; friend class PullingAsyncPipelineExecutor; friend class CompletedPipelineExecutor; + friend class ManualPipelineExecutor; friend class QueryPipelineBuilder; }; diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.cpp b/src/Storages/MaterializedView/RefreshAllCombiner.cpp new file mode 100644 index 00000000000..5cb06ade9c7 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshAllCombiner.cpp @@ -0,0 +1,58 @@ +#include + +#include + +namespace DB +{ + +RefreshAllCombiner::RefreshAllCombiner() + : time_arrived{false} +{} + +RefreshAllCombiner::RefreshAllCombiner(const std::vector & parents) + : time_arrived{false} +{ + parents_arrived.reserve(parents.size()); + for (auto && parent : parents) + parents_arrived.emplace(parent.uuid, false); +} + +bool RefreshAllCombiner::arriveTime() +{ + std::lock_guard lock(combiner_mutex); + time_arrived = true; + return allArrivedLocked(); +} + +bool RefreshAllCombiner::arriveParent(const StorageID & id) +{ + std::lock_guard lock(combiner_mutex); + parents_arrived[id.uuid] = true; + return allArrivedLocked(); +} + +void RefreshAllCombiner::flush() +{ + std::lock_guard lock(combiner_mutex); + flushLocked(); +} + +bool RefreshAllCombiner::allArrivedLocked() +{ + auto is_value = [](auto && key_value) { return key_value.second; }; + if (time_arrived && std::ranges::all_of(parents_arrived, is_value)) + { + flushLocked(); + return true; + } + return false; +} + +void RefreshAllCombiner::flushLocked() +{ + for (auto & [parent, arrived] : parents_arrived) + arrived = false; + time_arrived = false; +} + +} diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.h b/src/Storages/MaterializedView/RefreshAllCombiner.h new file mode 100644 index 00000000000..f4faf073ae4 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshAllCombiner.h @@ -0,0 +1,33 @@ +#pragma once + +#include + +namespace DB +{ + +/// Concurrent primitive for dependency completeness registration +/// When arrive methods return true, dependant task must be executed (or scheduled) +class RefreshAllCombiner +{ +public: + RefreshAllCombiner(); + + explicit RefreshAllCombiner(const std::vector & parents); + + bool arriveTime(); + + bool arriveParent(const StorageID & id); + + void flush(); + +private: + bool allArrivedLocked(); + + void flushLocked(); + + std::mutex combiner_mutex; + std::unordered_map parents_arrived; + bool time_arrived; +}; + +} diff --git a/src/Storages/MaterializedView/RefreshDependencies.cpp b/src/Storages/MaterializedView/RefreshDependencies.cpp new file mode 100644 index 00000000000..f1a834a6b3a --- /dev/null +++ b/src/Storages/MaterializedView/RefreshDependencies.cpp @@ -0,0 +1,60 @@ +#include + +#include + +namespace DB +{ + +RefreshDependencies::Entry::Entry(RefreshDependencies & deps, ContainerIter it) + : dependencies{&deps} + , entry_it{it} +{} + +RefreshDependencies::Entry::Entry(Entry && other) noexcept + : dependencies(std::exchange(other.dependencies, nullptr)) + , entry_it(std::move(other.entry_it)) +{} + +RefreshDependencies::Entry & RefreshDependencies::Entry::operator=(Entry && other) noexcept +{ + if (this == &other) + return *this; + cleanup(std::exchange(dependencies, std::exchange(other.dependencies, nullptr))); + entry_it = std::move(other.entry_it); + return *this; +} + +RefreshDependencies::Entry::~Entry() +{ + cleanup(dependencies); +} + +void RefreshDependencies::Entry::cleanup(RefreshDependencies * deps) +{ + if (deps) + deps->erase(entry_it); +} + +RefreshDependenciesEntry RefreshDependencies::add(RefreshTaskHolder dependency) +{ + std::lock_guard lock(dependencies_mutex); + return Entry(*this, dependencies.emplace(dependencies.end(), dependency)); +} + +void RefreshDependencies::notifyAll(const StorageID & id) +{ + std::lock_guard lock(dependencies_mutex); + for (auto && dep : dependencies) + { + if (auto task = dep.lock()) + task->notify(id); + } +} + +void RefreshDependencies::erase(ContainerIter it) +{ + std::lock_guard lock(dependencies_mutex); + dependencies.erase(it); +} + +} diff --git a/src/Storages/MaterializedView/RefreshDependencies.h b/src/Storages/MaterializedView/RefreshDependencies.h new file mode 100644 index 00000000000..a4488053adf --- /dev/null +++ b/src/Storages/MaterializedView/RefreshDependencies.h @@ -0,0 +1,56 @@ +#pragma once + +#include + +#include + +#include + + +namespace DB +{ + +class RefreshTask; + +/// Concurrent primitive for managing list of dependant task and notifying them +class RefreshDependencies +{ + using Container = std::list; + using ContainerIter = typename Container::iterator; + +public: + class Entry + { + friend class RefreshDependencies; + + public: + Entry(Entry &&) noexcept; + Entry & operator=(Entry &&) noexcept; + + ~Entry(); + + private: + Entry(RefreshDependencies & deps, ContainerIter it); + + void cleanup(RefreshDependencies * deps); + + RefreshDependencies * dependencies; + ContainerIter entry_it; + }; + + RefreshDependencies() = default; + + Entry add(RefreshTaskHolder dependency); + + void notifyAll(const StorageID & id); + +private: + void erase(ContainerIter it); + + std::mutex dependencies_mutex; + std::list dependencies; +}; + +using RefreshDependenciesEntry = RefreshDependencies::Entry; + +} diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp new file mode 100644 index 00000000000..c38d3408495 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -0,0 +1,128 @@ +#include +#include + +namespace DB +{ + +RefreshSetElement::RefreshSetElement(RefreshTaskHolder task, StorageID id) + : corresponding_task(task) + , view_id(std::move(id)) +{} + +RefreshInfo RefreshSetElement::getInfo() const +{ + return { + .database = view_id.getDatabaseName(), + .view_name = view_id.getTableName(), + .refresh_status = toString(RefreshTask::TaskState{state.load()}), + .last_refresh_status = toString(RefreshTask::LastTaskState{last_state.load()}), + .last_refresh_time = static_cast(last_s.load(std::memory_order_relaxed)), + .next_refresh_time = static_cast(next_s.load(std::memory_order_relaxed)), + .progress = static_cast(written_rows) / total_rows_to_read, + .elapsed_ns = elapsed_ns / 1e9, + .read_rows = read_rows.load(std::memory_order_relaxed), + .read_bytes = read_bytes.load(std::memory_order_relaxed), + .total_rows_to_read = total_rows_to_read.load(std::memory_order_relaxed), + .total_bytes_to_read = total_bytes_to_read.load(std::memory_order_relaxed), + .written_rows = written_rows.load(std::memory_order_relaxed), + .written_bytes = written_bytes.load(std::memory_order_relaxed), + .result_rows = result_rows.load(std::memory_order_relaxed), + .result_bytes = result_bytes.load(std::memory_order_relaxed) + }; +} + +const StorageID & RefreshSetElement::getID() const +{ + return view_id; +} + +RefreshTaskHolder RefreshSetElement::getTask() const +{ + return corresponding_task.lock(); +} + +bool RefreshSetLess::operator()(const RefreshSetElement & l, const RefreshSetElement & r) const +{ + return l.getID().uuid < r.getID().uuid; +} + +bool RefreshSetLess::operator()(const StorageID & l, const RefreshSetElement & r) const +{ + return l.uuid < r.getID().uuid; +} + +bool RefreshSetLess::operator()(const RefreshSetElement & l, const StorageID & r) const +{ + return l.getID().uuid < r.uuid; +} + +bool RefreshSetLess::operator()(const StorageID & l, const StorageID & r) const +{ + return l.uuid < r.uuid; +} + +RefreshSet::Entry::Entry() + : parent_set{nullptr} + , metric_increment{} +{} + +RefreshSet::Entry::Entry(Entry && other) noexcept + : parent_set{std::exchange(other.parent_set, nullptr)} + , iter(std::move(other.iter)) + , metric_increment(std::move(other.metric_increment)) +{} + +RefreshSet::Entry & RefreshSet::Entry::operator=(Entry && other) noexcept +{ + if (this == &other) + return *this; + cleanup(std::exchange(parent_set, std::exchange(other.parent_set, nullptr))); + iter = std::move(other.iter); + metric_increment = std::move(other.metric_increment); + return *this; +} + +RefreshSet::Entry::~Entry() +{ + cleanup(parent_set); +} + +RefreshSet::Entry::Entry(RefreshSet & set, ContainerIter it, const CurrentMetrics::Metric & metric) + : parent_set{&set}, iter(std::move(it)), metric_increment(metric) +{} + +void RefreshSet::Entry::cleanup(RefreshSet * set) +{ + if (set) + set->erase(iter); +} + +RefreshSet::RefreshSet() + : set_metric(CurrentMetrics::Refresh) +{} + +RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const +{ + std::lock_guard lock(elements_mutex); + if (auto element = elements.find(id); element != elements.end()) + return element->getTask(); + return nullptr; +} + +RefreshSet::InfoContainer RefreshSet::getInfo() const +{ + std::lock_guard lock(elements_mutex); + InfoContainer res; + res.reserve(elements.size()); + for (auto && element : elements) + res.emplace_back(element.getInfo()); + return res; +} + +void RefreshSet::erase(ContainerIter it) +{ + std::lock_guard lock(elements_mutex); + elements.erase(it); +} + +} diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h new file mode 100644 index 00000000000..48962d8c96d --- /dev/null +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -0,0 +1,142 @@ +#pragma once + +#include +#include +#include + +#include + +namespace CurrentMetrics +{ + extern const Metric Refresh; +} + +namespace DB +{ + +struct RefreshInfo +{ + String database; + String view_name; + String refresh_status; + String last_refresh_status; + UInt32 last_refresh_time; + UInt32 next_refresh_time; + Float64 progress; + Float64 elapsed_ns; + UInt64 read_rows; + UInt64 read_bytes; + UInt64 total_rows_to_read; + UInt64 total_bytes_to_read; + UInt64 written_rows; + UInt64 written_bytes; + UInt64 result_rows; + UInt64 result_bytes; +}; + +class RefreshSetElement +{ + friend class RefreshTask; +public: + RefreshSetElement(RefreshTaskHolder task, StorageID id); + + RefreshSetElement(const RefreshSetElement &) = delete; + RefreshSetElement & operator=(const RefreshSetElement &) = delete; + + RefreshInfo getInfo() const; + + RefreshTaskHolder getTask() const; + + const StorageID & getID() const; + +private: + RefreshTaskObserver corresponding_task; + StorageID view_id; + + mutable std::atomic read_rows{0}; + mutable std::atomic read_bytes{0}; + mutable std::atomic total_rows_to_read{0}; + mutable std::atomic total_bytes_to_read{0}; + mutable std::atomic written_rows{0}; + mutable std::atomic written_bytes{0}; + mutable std::atomic result_rows{0}; + mutable std::atomic result_bytes{0}; + mutable std::atomic elapsed_ns{0}; + mutable std::atomic last_s{0}; + mutable std::atomic next_s{0}; + mutable std::atomic state{0}; + mutable std::atomic last_state{0}; +}; + +struct RefreshSetLess +{ + using is_transparent = std::true_type; + + bool operator()(const RefreshSetElement & l, const RefreshSetElement & r) const; + bool operator()(const StorageID & l, const RefreshSetElement & r) const; + bool operator()(const RefreshSetElement & l, const StorageID & r) const; + bool operator()(const StorageID & l, const StorageID & r) const; +}; + +/// Set of refreshable views +class RefreshSet +{ +private: + using Container = std::set; + using ContainerIter = typename Container::iterator; + +public: + class Entry + { + friend class RefreshSet; + public: + Entry(); + + Entry(Entry &&) noexcept; + Entry & operator=(Entry &&) noexcept; + + ~Entry(); + + const RefreshSetElement * operator->() const { return std::to_address(iter); } + + private: + RefreshSet * parent_set; + ContainerIter iter; + std::optional metric_increment; + + Entry( + RefreshSet & set, + ContainerIter it, + const CurrentMetrics::Metric & metric); + + void cleanup(RefreshSet * set); + }; + + using InfoContainer = std::vector; + + RefreshSet(); + + template + std::optional emplace(Args &&... args) + { + std::lock_guard guard(elements_mutex); + if (auto [it, is_inserted] = elements.emplace(std::forward(args)...); is_inserted) + return Entry(*this, std::move(it), set_metric); + return {}; + } + + RefreshTaskHolder getTask(const StorageID & id) const; + + InfoContainer getInfo() const; + +private: + mutable std::mutex elements_mutex; + Container elements; + CurrentMetrics::Metric set_metric; + + void erase(ContainerIter it); +}; + +using RefreshSetEntry = RefreshSet::Entry; + +} diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp new file mode 100644 index 00000000000..579d3252865 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -0,0 +1,292 @@ +#include + +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +std::uniform_int_distribution makeSpreadDistribution(const ASTTimePeriod * spread) +{ + if (!spread) + return std::uniform_int_distribution(0, 0); + Int64 limit = spread->kind.toAvgSeconds() * spread->value / 2; + return std::uniform_int_distribution(-limit, limit); +} + +std::variant makeRefreshTimer(const ASTRefreshStrategy & strategy) +{ + using enum ASTRefreshStrategy::ScheduleKind; + switch (strategy.schedule_kind) + { + case EVERY: + return RefreshEveryTimer{*strategy.period, strategy.interval}; + case AFTER: + return RefreshAfterTimer{strategy.interval}; + default: + throw Exception("Unknown refresh strategy kind", ErrorCodes::BAD_ARGUMENTS); + } +} + +} + +RefreshTask::RefreshTask( + const ASTRefreshStrategy & strategy) + : refresh_timer(makeRefreshTimer(strategy)) + , refresh_spread{makeSpreadDistribution(strategy.spread)} + , refresh_immediately{false} + , interrupt_execution{false} + , canceled{false} +{} + +RefreshTaskHolder RefreshTask::create( + const StorageMaterializedView & view, + ContextMutablePtr context, + const DB::ASTRefreshStrategy & strategy) +{ + auto task = std::make_shared(strategy); + + task->refresh_task = context->getSchedulePool().createTask("MaterializedViewRefresherTask", task->makePoolTask()); + task->set_entry = context->getRefreshSet().emplace(task, view.getStorageID()).value(); + if (strategy.dependencies) + { + if (strategy.schedule_kind != ASTRefreshStrategy::ScheduleKind::AFTER) + throw Exception("Dependencies are allowed only for AFTER refresh kind", ErrorCodes::BAD_ARGUMENTS); + + task->deps_entries.reserve(strategy.dependencies->children.size()); + for (auto && dependency : strategy.dependencies->children) + { + StorageID dep_id(dependency->as()); + if (auto dep_task = context->getRefreshSet().getTask(dep_id)) + task->deps_entries.push_back(dep_task->dependencies.add(task)); + } + } + + return task; +} + +void RefreshTask::initialize(std::shared_ptr view) +{ + view_to_refresh = view; +} + +void RefreshTask::start() +{ + storeState(TaskState::Scheduled); + refresh_task->activateAndSchedule(); +} + +void RefreshTask::stop() +{ + refresh_task->deactivate(); + cancel(); + storeState(TaskState::Disabled); +} + +void RefreshTask::run() +{ + refresh_immediately.store(true); + refresh_task->activateAndSchedule(); +} + +void RefreshTask::cancel() +{ + canceled.store(true); + interrupt_execution.store(true); +} + +void RefreshTask::pause() +{ + interrupt_execution.store(true); +} + +void RefreshTask::resume() +{ + interrupt_execution.store(false); + refresh_immediately.store(true); + refresh_task->schedule(); +} + +void RefreshTask::notify(const StorageID & parent_id) +{ + if (combiner.arriveParent(parent_id)) + { + refresh_immediately.store(true); + refresh_task->schedule(); + } +} + +void RefreshTask::doRefresh() +{ + if (refresh_immediately.exchange(false)) + { + refresh(); + } + else + { + auto now = std::chrono::system_clock::now(); + if (now >= next_refresh) + { + if (combiner.arriveTime()) + refresh(); + } + else + scheduleRefresh(now); + } +} + +void RefreshTask::refresh() +{ + auto view = lockView(); + if (!view) + return; + + if (!refresh_executor) + initializeRefresh(view); + + storeState(TaskState::Running); + + switch (executeRefresh()) + { + case ExecutionResult::Paused: + storeState(TaskState::Paused); + return; + case ExecutionResult::Finished: + completeRefresh(view); + storeLastState(LastTaskState::Finished); + break; + case ExecutionResult::Cancelled: + storeLastState(LastTaskState::Canceled); + break; + } + + refresh_executor.reset(); + refresh_block.reset(); + + storeLastRefresh(std::chrono::system_clock::now()); + scheduleRefresh(last_refresh); +} + +RefreshTask::ExecutionResult RefreshTask::executeRefresh() +{ + bool not_finished{true}; + while (!interrupt_execution.load() && not_finished) + not_finished = refresh_executor->executeStep(interrupt_execution); + + if (!not_finished) + return ExecutionResult::Finished; + if (interrupt_execution.load() && !canceled.load()) + return ExecutionResult::Paused; + return ExecutionResult::Cancelled; + +} + +void RefreshTask::initializeRefresh(std::shared_ptr view) +{ + refresh_query = view->prepareRefreshQuery(); + auto refresh_context = Context::createCopy(view->getContext()); + refresh_block = InterpreterInsertQuery(refresh_query, refresh_context).execute(); + refresh_block->pipeline.setProgressCallback([this](const Progress & progress){ progressCallback(progress); }); + + canceled.store(false); + interrupt_execution.store(false); + + refresh_executor.emplace(refresh_block->pipeline); +} + +void RefreshTask::completeRefresh(std::shared_ptr view) +{ + view->updateInnerTableAfterRefresh(refresh_query); + dependencies.notifyAll(view->getStorageID()); +} + +void RefreshTask::scheduleRefresh(std::chrono::system_clock::time_point now) +{ + using namespace std::chrono_literals; + auto scheduled_refresh = calculateRefreshTime(now) + genSpreadSeconds(); + storeNextRefresh(scheduled_refresh); + auto schedule_time = std::chrono::ceil(scheduled_refresh - now); + storeState(TaskState::Scheduled); + refresh_task->scheduleAfter(std::max(schedule_time, 0ms).count()); +} + +namespace +{ + +template +struct CombinedVisitor : Ts... { using Ts::operator()...; }; +template +CombinedVisitor(Ts...) -> CombinedVisitor; + +} + +std::chrono::sys_seconds RefreshTask::calculateRefreshTime(std::chrono::system_clock::time_point now) const +{ + CombinedVisitor refresh_time_visitor{ + [now](const RefreshAfterTimer & timer) { return timer.after(now); }, + [now](const RefreshEveryTimer & timer) { return timer.next(now); }}; + return std::visit(std::move(refresh_time_visitor), refresh_timer); +} + +std::chrono::seconds RefreshTask::genSpreadSeconds() +{ + return std::chrono::seconds{refresh_spread(thread_local_rng)}; +} + +void RefreshTask::progressCallback(const Progress & progress) +{ + set_entry->read_rows.store(progress.read_rows, std::memory_order_relaxed); + set_entry->read_bytes.store(progress.read_bytes, std::memory_order_relaxed); + set_entry->total_rows_to_read.store(progress.total_rows_to_read, std::memory_order_relaxed); + set_entry->total_bytes_to_read.store(progress.total_bytes_to_read, std::memory_order_relaxed); + set_entry->written_rows.store(progress.written_rows, std::memory_order_relaxed); + set_entry->written_bytes.store(progress.written_bytes, std::memory_order_relaxed); + set_entry->result_rows.store(progress.result_rows, std::memory_order_relaxed); + set_entry->result_bytes.store(progress.result_bytes, std::memory_order_relaxed); + set_entry->elapsed_ns.store(progress.elapsed_ns, std::memory_order_relaxed); +} + +std::shared_ptr RefreshTask::lockView() +{ + return std::static_pointer_cast(view_to_refresh.lock()); +} + +void RefreshTask::storeState(TaskState task_state) +{ + state.store(task_state); + set_entry->state.store(static_cast(task_state)); +} + +void RefreshTask::storeLastState(LastTaskState task_state) +{ + last_state = task_state; + set_entry->last_state.store(static_cast(task_state)); +} + +void RefreshTask::storeLastRefresh(std::chrono::system_clock::time_point last) +{ + last_refresh = last; + auto secs = std::chrono::floor(last); + set_entry->last_s.store(secs.time_since_epoch().count()); +} + +void RefreshTask::storeNextRefresh(std::chrono::system_clock::time_point next) +{ + next_refresh = next; + auto secs = std::chrono::floor(next); + set_entry->next_s.store(secs.time_since_epoch().count()); +} + +} diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h new file mode 100644 index 00000000000..5e9bb618372 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -0,0 +1,150 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + +#include + +#include + + +namespace DB +{ + +class StorageMaterializedView; +class ASTRefreshStrategy; + +class RefreshTask : public std::enable_shared_from_this +{ +public: + enum class TaskState : RefreshTaskStateUnderlying + { + Disabled = 0, + Scheduled, + Running, + Paused + }; + + enum class LastTaskState : RefreshTaskStateUnderlying + { + Unknown = 0, + Canceled, + Finished + }; + + /// Never call it manual, public for shared_ptr construction only + RefreshTask(const ASTRefreshStrategy & strategy); + + /// The only proper way to construct task + static RefreshTaskHolder create( + const StorageMaterializedView & view, + ContextMutablePtr context, + const DB::ASTRefreshStrategy & strategy); + + void initialize(std::shared_ptr view); + + /// Enable task scheduling + void start(); + + /// Disable task scheduling + void stop(); + + /// Schedule task immediately + void run(); + + /// Cancel task execution + void cancel(); + + /// Pause task execution (must be either resumed or canceled later) + void pause(); + + /// Resume task execution + void resume(); + + /// Notify dependant task + void notify(const StorageID & parent_id); + +private: + enum class ExecutionResult : UInt8 + { + Finished, + Paused, + Cancelled + }; + + void doRefresh(); + + void scheduleRefresh(std::chrono::system_clock::time_point now); + + void refresh(); + + ExecutionResult executeRefresh(); + + void initializeRefresh(std::shared_ptr view); + + void completeRefresh(std::shared_ptr view); + + std::chrono::sys_seconds calculateRefreshTime(std::chrono::system_clock::time_point now) const; + + std::chrono::seconds genSpreadSeconds(); + + void progressCallback(const Progress & progress); + + auto makePoolTask() + { + return [self = this->weak_from_this()] + { + if (auto task = self.lock()) + task->doRefresh(); + }; + } + + std::shared_ptr lockView(); + + void storeState(TaskState task_state); + + void storeLastState(LastTaskState task_state); + + void storeLastRefresh(std::chrono::system_clock::time_point last); + + void storeNextRefresh(std::chrono::system_clock::time_point next); + + /// Task ownership + BackgroundSchedulePool::TaskHolder refresh_task; + std::weak_ptr view_to_refresh; + RefreshSet::Entry set_entry; + + /// Task execution + std::optional refresh_executor; + std::optional refresh_block; + std::shared_ptr refresh_query; + + /// Concurrent dependency management + RefreshAllCombiner combiner; + RefreshDependencies dependencies; + std::vector deps_entries; + + /// Refresh time settings and data + std::chrono::system_clock::time_point last_refresh; + std::chrono::system_clock::time_point next_refresh; + std::variant refresh_timer; + + /// Refresh time randomization + std::uniform_int_distribution refresh_spread; + + /// Task state + std::atomic state{TaskState::Disabled}; + LastTaskState last_state{LastTaskState::Unknown}; + + /// Outer triggers + std::atomic_bool refresh_immediately; + std::atomic_bool interrupt_execution; + std::atomic_bool canceled; +}; + +} diff --git a/src/Storages/MaterializedView/RefreshTask_fwd.h b/src/Storages/MaterializedView/RefreshTask_fwd.h new file mode 100644 index 00000000000..1f366962eb6 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshTask_fwd.h @@ -0,0 +1,15 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class RefreshTask; + +using RefreshTaskStateUnderlying = UInt8; +using RefreshTaskHolder = std::shared_ptr; +using RefreshTaskObserver = std::weak_ptr; + +} diff --git a/src/Storages/MaterializedView/RefreshTimers.cpp b/src/Storages/MaterializedView/RefreshTimers.cpp new file mode 100644 index 00000000000..973eba46057 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshTimers.cpp @@ -0,0 +1,243 @@ +#include + +#include + +namespace DB +{ + +namespace +{ + constexpr std::chrono::days ZERO_DAYS{0}; + constexpr std::chrono::days ONE_DAY{1}; +} + +RefreshAfterTimer::RefreshAfterTimer(const ASTTimeInterval * time_interval) +{ + if (time_interval) + { + for (auto && [kind, value] : time_interval->kinds) + setWithKind(kind, value); + } +} + +std::chrono::sys_seconds RefreshAfterTimer::after(std::chrono::system_clock::time_point tp) const +{ + auto tp_date = std::chrono::floor(tp); + auto tp_time_offset = std::chrono::floor(tp - tp_date); + std::chrono::year_month_day ymd(tp_date); + ymd += years; + ymd += months; + std::chrono::sys_days date = ymd; + date += weeks; + date += days; + auto result = std::chrono::time_point_cast(date); + result += tp_time_offset; + result += hours; + result += minutes; + result += seconds; + return result; +} + +void RefreshAfterTimer::setWithKind(IntervalKind kind, UInt64 val) +{ + switch (kind) + { + case IntervalKind::Second: + seconds = std::chrono::seconds{val}; + break; + case IntervalKind::Minute: + minutes = std::chrono::minutes{val}; + break; + case IntervalKind::Hour: + hours = std::chrono::hours{val}; + break; + case IntervalKind::Day: + days = std::chrono::days{val}; + break; + case IntervalKind::Week: + weeks = std::chrono::weeks{val}; + break; + case IntervalKind::Month: + months = std::chrono::months{val}; + break; + case IntervalKind::Year: + years = std::chrono::years{val}; + break; + default: + break; + } +} + +RefreshEveryTimer::RefreshEveryTimer(const ASTTimePeriod & time_period, const ASTTimeInterval * time_offset) + : offset(time_offset) + , value{static_cast(time_period.value)} + , kind{time_period.kind} +{ + // TODO: validate invariants +} + +std::chrono::sys_seconds RefreshEveryTimer::next(std::chrono::system_clock::time_point tp) const +{ + if (value == 0) + return std::chrono::floor(tp); + switch (kind) + { + case IntervalKind::Second: + return alignedToSeconds(tp); + case IntervalKind::Minute: + return alignedToMinutes(tp); + case IntervalKind::Hour: + return alignedToHours(tp); + case IntervalKind::Day: + return alignedToDays(tp); + case IntervalKind::Week: + return alignedToWeeks(tp); + case IntervalKind::Month: + return alignedToMonths(tp); + case IntervalKind::Year: + return alignedToYears(tp); + default: + return std::chrono::ceil(tp); + } +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToYears(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto tp_days = std::chrono::floor(tp); + std::chrono::year_month_day tp_ymd(tp_days); + auto normalize_years = [](std::chrono::year year) -> std::chrono::sys_days + { + return year / std::chrono::January / 1d; + }; + + auto prev_years = normalize_years(tp_ymd.year()); + if (auto prev_time = offset.after(prev_years); prev_time > tp) + return prev_time; + + auto next_years = normalize_years(tp_ymd.year() + std::chrono::years{1}); + return offset.after(next_years); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToMonths(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto tp_days = std::chrono::floor(tp); + std::chrono::year_month_day tp_ymd(tp_days); + auto normalize_months = [](const std::chrono::year_month_day & ymd, unsigned month_value) -> std::chrono::sys_days + { + return ymd.year() / std::chrono::month{month_value} / 1d; + }; + + auto prev_month_value = static_cast(tp_ymd.month()) / value * value; + auto prev_months = normalize_months(tp_ymd, prev_month_value); + if (auto prev_time = offset.after(prev_months); prev_time > tp) + return prev_time; + + auto next_month_value = (static_cast(tp_ymd.month()) / value + 1) * value; + auto next_months = normalize_months(tp_ymd, next_month_value); + std::chrono::year_month_day next_ymd(next_months); + if (next_ymd.year() > tp_ymd.year()) + return offset.after(normalize_months(next_ymd, value)); + return offset.after(next_months); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToWeeks(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto cpp_weekday = offset.getDays() + ONE_DAY; + std::chrono::weekday offset_weekday((cpp_weekday - std::chrono::floor(cpp_weekday)).count()); + + auto tp_days = std::chrono::floor(tp); + std::chrono::year_month_weekday tp_ymd(tp_days); + auto normalize_weeks = [offset_weekday](const std::chrono::year_month_weekday & ymd, unsigned week_value) + { + return std::chrono::sys_days(ymd.year() / ymd.month() / std::chrono::weekday{offset_weekday}[week_value]); + }; + + auto prev_week_value = tp_ymd.index() / value * value; + auto prev_days = normalize_weeks(tp_ymd, prev_week_value); + if (auto prev_time = offset.after(prev_days - offset.getDays()); prev_time > tp) + return prev_time; + + auto next_day_value = (tp_ymd.index() / value + 1) * value; + auto next_days = normalize_weeks(tp_ymd, next_day_value); + std::chrono::year_month_weekday next_ymd(next_days); + if (next_ymd.year() > tp_ymd.year() || next_ymd.month() > tp_ymd.month()) + return offset.after(normalize_weeks(next_ymd, value) - offset.getDays()); + return offset.after(next_days); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToDays(std::chrono::system_clock::time_point tp) const +{ + auto tp_days = std::chrono::floor(tp); + std::chrono::year_month_day tp_ymd(tp_days); + auto normalize_days = [](const std::chrono::year_month_day & ymd, unsigned day_value) -> std::chrono::sys_days + { + return ymd.year() / ymd.month() / std::chrono::day{day_value}; + }; + + auto prev_day_value = static_cast(tp_ymd.day()) / value * value; + auto prev_days = normalize_days(tp_ymd, prev_day_value); + if (auto prev_time = offset.after(prev_days); prev_time > tp) + return prev_time; + + auto next_day_value = (static_cast(tp_ymd.day()) / value + 1) * value; + auto next_days = normalize_days(tp_ymd, next_day_value); + std::chrono::year_month_day next_ymd(next_days); + if (next_ymd.year() > tp_ymd.year() || next_ymd.month() > tp_ymd.month()) + return offset.after(normalize_days(next_ymd, value)); + return offset.after(next_days); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToHours(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto tp_days = std::chrono::floor(tp); + auto tp_hours = std::chrono::floor(tp - tp_days); + + auto prev_hours = (tp_hours / value) * value; + if (auto prev_time = offset.after(tp_days + prev_hours); prev_time > tp) + return prev_time; + + auto next_hours = (tp_hours / value + 1h) * value; + if (std::chrono::floor(next_hours - 1h) > ZERO_DAYS) + return offset.after(tp_days + ONE_DAY + std::chrono::hours{value}); + return offset.after(tp_days + next_hours); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToMinutes(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto tp_hours = std::chrono::floor(tp); + auto tp_minutes = std::chrono::floor(tp - tp_hours); + + auto prev_minutes = (tp_minutes / value) * value; + if (auto prev_time = offset.after(tp_hours + prev_minutes); prev_time > tp) + return prev_time; + + auto next_minutes = (tp_minutes / value + 1min) * value; + if (std::chrono::floor(next_minutes - 1min) > 0h) + return offset.after(tp_hours + 1h + std::chrono::minutes{value}); + return offset.after(tp_hours + next_minutes); +} + +std::chrono::sys_seconds RefreshEveryTimer::alignedToSeconds(std::chrono::system_clock::time_point tp) const +{ + using namespace std::chrono_literals; + + auto tp_minutes = std::chrono::floor(tp); + auto tp_seconds = std::chrono::floor(tp - tp_minutes); + + auto next_seconds= (tp_seconds / value + 1s) * value; + if (std::chrono::floor(next_seconds - 1s) > 0min) + return tp_minutes + 1min + std::chrono::seconds{value}; + return tp_minutes + next_seconds; +} + +} diff --git a/src/Storages/MaterializedView/RefreshTimers.h b/src/Storages/MaterializedView/RefreshTimers.h new file mode 100644 index 00000000000..0672782a3f9 --- /dev/null +++ b/src/Storages/MaterializedView/RefreshTimers.h @@ -0,0 +1,69 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class ASTTimeInterval; +class ASTTimePeriod; + +/// Schedule timer for MATERIALIZED VIEW ... REFRESH AFTER ... queries +class RefreshAfterTimer +{ +public: + explicit RefreshAfterTimer(const ASTTimeInterval * time_interval); + + std::chrono::sys_seconds after(std::chrono::system_clock::time_point tp) const; + + std::chrono::seconds getSeconds() const { return seconds; } + std::chrono::minutes getMinutes() const { return minutes; } + std::chrono::hours getHours() const { return hours; } + std::chrono::days getDays() const { return days; } + std::chrono::weeks getWeeks() const { return weeks; } + std::chrono::months getMonths() const { return months; } + std::chrono::years getYears() const { return years; } + +private: + void setWithKind(IntervalKind kind, UInt64 val); + + std::chrono::seconds seconds{0}; + std::chrono::minutes minutes{0}; + std::chrono::hours hours{0}; + std::chrono::days days{0}; + std::chrono::weeks weeks{0}; + std::chrono::months months{0}; + std::chrono::years years{0}; +}; + +/// Schedule timer for MATERIALIZED VIEW ... REFRESH EVERY ... queries +class RefreshEveryTimer +{ +public: + explicit RefreshEveryTimer(const ASTTimePeriod & time_period, const ASTTimeInterval * time_offset); + + std::chrono::sys_seconds next(std::chrono::system_clock::time_point tp) const; + +private: + std::chrono::sys_seconds alignedToYears(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToMonths(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToWeeks(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToDays(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToHours(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToMinutes(std::chrono::system_clock::time_point tp) const; + + std::chrono::sys_seconds alignedToSeconds(std::chrono::system_clock::time_point tp) const; + + RefreshAfterTimer offset; + UInt32 value{0}; + IntervalKind kind{IntervalKind::Second}; +}; + +} diff --git a/src/Storages/MaterializedView/tests/gtest_timers.cpp b/src/Storages/MaterializedView/tests/gtest_timers.cpp new file mode 100644 index 00000000000..2a9f30c57fa --- /dev/null +++ b/src/Storages/MaterializedView/tests/gtest_timers.cpp @@ -0,0 +1,27 @@ +#include + +#include +#include + +using namespace DB; + +TEST(Timers, AfterTimer) +{ + using namespace std::chrono; + + auto interval = std::make_shared(); + interval->kinds = { + {IntervalKind::Week, 2}, + {IntervalKind::Day, 3}, + {IntervalKind::Minute, 15}, + }; + RefreshAfterTimer timer(interval.get()); + + sys_days date_in = 2023y / January / 18d; + auto secs_in = date_in + 23h + 57min; + + sys_days date_out = 2023y / February / 5d; + auto secs_out = date_out + 0h + 12min; + + ASSERT_EQ(secs_out, timer.after(secs_in)); +} diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 2339fd11cf8..43e8e0d6f33 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include @@ -7,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +41,11 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW; } +namespace ActionLocks +{ + extern const StorageActionBlockType ViewRefresh; +} + static inline String generateInnerTableName(const StorageID & view_id) { if (view_id.hasUUID()) @@ -126,6 +134,12 @@ StorageMaterializedView::StorageMaterializedView( target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->getDatabase(), manual_create_query->getTable()}, getContext())->getStorageID(); } + + if (query.refresh_strategy) + refresher = RefreshTask::create( + *this, + getContext(), + *query.refresh_strategy); } QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage( @@ -236,23 +250,24 @@ void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_co /// See the comment in StorageMaterializedView::drop. /// DDL queries with StorageMaterializedView are fundamentally broken. /// Best-effort to make them work: the inner table name is almost always less than the MV name (so it's safe to lock DDLGuard) - bool may_lock_ddl_guard = getStorageID().getQualifiedName() < target_table_id.getQualifiedName(); + auto inner_table_id = getTargetTableId(); + bool may_lock_ddl_guard = getStorageID().getQualifiedName() < inner_table_id.getQualifiedName(); if (has_inner_table && tryGetTargetTable()) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, inner_table_id, sync, /* ignore_sync_setting */ true, may_lock_ddl_guard); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { if (has_inner_table) - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, target_table_id, true); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, getTargetTableId(), true); } void StorageMaterializedView::checkStatementCanBeForwarded() const { if (!has_inner_table) throw Exception(ErrorCodes::INCORRECT_QUERY, "MATERIALIZED VIEW targets existing table {}. " - "Execute the statement directly on it.", target_table_id.getNameForLogs()); + "Execute the statement directly on it.", getTargetTableId().getNameForLogs()); } bool StorageMaterializedView::optimize( @@ -270,6 +285,48 @@ bool StorageMaterializedView::optimize( return getTargetTable()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, local_context); } +std::shared_ptr StorageMaterializedView::prepareRefreshQuery() +{ + auto inner_table_id = getTargetTableId(); + auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); + + auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + + 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; + + auto create_ctx = Context::createCopy(getContext()); + InterpreterCreateQuery create_interpreter(create_table_query, create_ctx); + create_interpreter.setInternal(true); + create_interpreter.execute(); + + auto insert_query = std::make_shared(); + insert_query->setTable(new_table_name); + insert_query->setDatabase(db->getDatabaseName()); + insert_query->select = getInMemoryMetadataPtr()->getSelectQuery().select_query; + + return insert_query; +} + +void StorageMaterializedView::updateInnerTableAfterRefresh(std::shared_ptr refresh_query) +{ + auto inner_table_id = getTargetTableId(); + + auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); + auto target_db = DatabaseCatalog::instance().getDatabase(refresh_query->getDatabase()); + + auto rename_ctx = Context::createCopy(getContext()); + target_db->renameTable( + rename_ctx, refresh_query->getTable(), *db, inner_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); + + setTargetTableId(db->getTable(refresh_query->getTable(), getContext())->getStorageID()); +} + void StorageMaterializedView::alter( const AlterCommands & params, ContextPtr local_context, @@ -332,6 +389,7 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, ContextP void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) { auto old_table_id = getStorageID(); + auto inner_table_id = getTargetTableId(); auto metadata_snapshot = getInMemoryMetadataPtr(); bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID(); @@ -340,14 +398,14 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) auto new_target_table_name = generateInnerTableName(new_table_id); auto rename = std::make_shared(); - assert(target_table_id.database_name == old_table_id.database_name); + assert(inner_table_id.database_name == old_table_id.database_name); ASTRenameQuery::Element elem { ASTRenameQuery::Table { - target_table_id.database_name.empty() ? nullptr : std::make_shared(target_table_id.database_name), - std::make_shared(target_table_id.table_name) + 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 { @@ -358,15 +416,14 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id) rename->elements.emplace_back(std::move(elem)); InterpreterRenameQuery(rename, getContext()).execute(); - target_table_id.database_name = new_table_id.database_name; - target_table_id.table_name = new_target_table_name; + updateTargetTableId(new_table_id.database_name, new_target_table_name); } IStorage::renameInMemory(new_table_id); if (from_atomic_to_atomic_database && has_inner_table) { - assert(target_table_id.database_name == old_table_id.database_name); - target_table_id.database_name = new_table_id.database_name; + assert(inner_table_id.database_name == old_table_id.database_name); + updateTargetTableId(new_table_id.database_name, std::nullopt); } const auto & select_query = metadata_snapshot->getSelectQuery(); // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated @@ -379,10 +436,19 @@ void StorageMaterializedView::startup() const auto & select_query = metadata_snapshot->getSelectQuery(); if (!select_query.select_table_id.empty()) DatabaseCatalog::instance().addViewDependency(select_query.select_table_id, getStorageID()); + + if (refresher) + { + refresher->initialize(std::static_pointer_cast(shared_from_this())); + refresher->start(); + } } void StorageMaterializedView::shutdown(bool) { + if (refresher) + refresher->stop(); + auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & select_query = metadata_snapshot->getSelectQuery(); /// Make sure the dependency is removed after DETACH TABLE @@ -393,13 +459,13 @@ void StorageMaterializedView::shutdown(bool) StoragePtr StorageMaterializedView::getTargetTable() const { checkStackSize(); - return DatabaseCatalog::instance().getTable(target_table_id, getContext()); + return DatabaseCatalog::instance().getTable(getTargetTableId(), getContext()); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { checkStackSize(); - return DatabaseCatalog::instance().tryGetTable(target_table_id, getContext()); + return DatabaseCatalog::instance().tryGetTable(getTargetTableId(), getContext()); } NamesAndTypesList StorageMaterializedView::getVirtuals() const @@ -472,6 +538,8 @@ std::optional StorageMaterializedView::totalBytesUncompressed(const Sett ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type) { + if (type == ActionLocks::ViewRefresh && refresher) + refresher->stop(); if (has_inner_table) { if (auto target_table = tryGetTargetTable()) @@ -487,6 +555,34 @@ bool StorageMaterializedView::isRemote() const return false; } +void StorageMaterializedView::onActionLockRemove(StorageActionBlockType action_type) +{ + if (action_type == ActionLocks::ViewRefresh && refresher) + refresher->start(); + /// TODO: Do we need to release action lock on inner table? +} + +DB::StorageID StorageMaterializedView::getTargetTableId() const +{ + std::lock_guard guard(target_table_id_mutex); + return target_table_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); + if (database_name) + target_table_id.database_name = *std::move(database_name); + if (table_name) + target_table_id.table_name = *std::move(table_name); +} + void registerStorageMaterializedView(StorageFactory & factory) { factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 9ddcc458f3e..2dce8355b47 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -83,6 +84,7 @@ public: NamesAndTypesList getVirtuals() const override; ActionLock getActionLock(StorageActionBlockType type) override; + void onActionLockRemove(StorageActionBlockType action_type) override; void read( QueryPlan & query_plan, @@ -105,12 +107,25 @@ public: std::optional totalBytesUncompressed(const Settings & settings) const override; private: + mutable std::mutex target_table_id_mutex; /// Will be initialized in constructor StorageID target_table_id = StorageID::createEmpty(); + RefreshTaskHolder refresher; + bool has_inner_table = false; + friend class RefreshTask; + void checkStatementCanBeForwarded() const; + + std::shared_ptr prepareRefreshQuery(); + + void updateInnerTableAfterRefresh(std::shared_ptr refresh_query); + + StorageID getTargetTableId() const; + void setTargetTableId(StorageID id); + void updateTargetTableId(std::optional database_name, std::optional table_name); }; } diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp new file mode 100644 index 00000000000..0ddf8a48c9d --- /dev/null +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -0,0 +1,67 @@ +#include + +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemViewRefreshes::getNamesAndTypes() +{ + return { + {"database", std::make_shared()}, + {"view", std::make_shared()}, + {"refresh_status", std::make_shared()}, + {"last_refresh_status", std::make_shared()}, + {"last_refresh_time", std::make_shared()}, + {"next_refresh_time", std::make_shared()}, + {"progress", std::make_shared()}, + {"elapsed", std::make_shared()}, + {"read_rows", std::make_shared()}, + {"read_bytes", std::make_shared()}, + {"total_rows", std::make_shared()}, + {"total_bytes", std::make_shared()}, + {"written_rows", std::make_shared()}, + {"written_bytes", std::make_shared()}, + {"result_rows", std::make_shared()}, + {"result_bytes", std::make_shared()}, + }; +} + +void StorageSystemViewRefreshes::fillData( + MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + auto access = context->getAccess(); + // TODO: Do we need to add new access type? + auto valid_access = AccessType::SHOW_TABLES; + bool check_access_for_tables = !access->isGranted(valid_access); + + + for (const auto & refresh : context->getRefreshSet().getInfo()) + { + if (check_access_for_tables && !access->isGranted(valid_access, refresh.database, refresh.view_name)) + continue; + + std::size_t i = 0; + res_columns[i++]->insert(refresh.database); + res_columns[i++]->insert(refresh.view_name); + res_columns[i++]->insert(refresh.refresh_status); + res_columns[i++]->insert(refresh.last_refresh_status); + res_columns[i++]->insert(refresh.last_refresh_time); + res_columns[i++]->insert(refresh.next_refresh_time); + res_columns[i++]->insert(refresh.progress); + res_columns[i++]->insert(refresh.elapsed_ns); + res_columns[i++]->insert(refresh.read_rows); + res_columns[i++]->insert(refresh.read_bytes); + res_columns[i++]->insert(refresh.total_rows_to_read); + res_columns[i++]->insert(refresh.total_bytes_to_read); + res_columns[i++]->insert(refresh.written_rows); + res_columns[i++]->insert(refresh.written_bytes); + res_columns[i++]->insert(refresh.result_rows); + res_columns[i++]->insert(refresh.result_bytes); + } +} + +} diff --git a/src/Storages/System/StorageSystemViewRefreshes.h b/src/Storages/System/StorageSystemViewRefreshes.h new file mode 100644 index 00000000000..475ad45e68f --- /dev/null +++ b/src/Storages/System/StorageSystemViewRefreshes.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + + +class StorageSystemViewRefreshes final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemViewRefreshes"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} + diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index ffe74f1c94a..b907b97f0dd 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -87,6 +87,7 @@ #include #include #include +#include #if defined(__ELF__) && !defined(OS_FREEBSD) #include @@ -209,6 +210,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "jemalloc_bins"); attach(context, system_database, "s3queue"); attach(context, system_database, "dashboards"); + attach(context, system_database, "view_refreshes"); if (has_zookeeper) { From ba766edb530d481a9ed50aee36f4b46aa9b1d23b Mon Sep 17 00:00:00 2001 From: koloshmet Date: Mon, 13 Feb 2023 04:18:01 +0200 Subject: [PATCH 02/25] refreshable view query test --- .../0_stateless/02661_refreshable_materialized_views.reference | 1 + .../0_stateless/02661_refreshable_materialized_views.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02661_refreshable_materialized_views.reference create mode 100644 tests/queries/0_stateless/02661_refreshable_materialized_views.sql diff --git a/tests/queries/0_stateless/02661_refreshable_materialized_views.reference b/tests/queries/0_stateless/02661_refreshable_materialized_views.reference new file mode 100644 index 00000000000..663d951b8b5 --- /dev/null +++ b/tests/queries/0_stateless/02661_refreshable_materialized_views.reference @@ -0,0 +1 @@ +Scheduled Unknown diff --git a/tests/queries/0_stateless/02661_refreshable_materialized_views.sql b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql new file mode 100644 index 00000000000..0b8a6a5289e --- /dev/null +++ b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql @@ -0,0 +1,3 @@ +CREATE MATERIALIZED VIEW test REFRESH AFTER 15 SECOND ENGINE = MergeTree() ORDER BY number AS SELECT * FROM system.numbers LIMIT 10; + +SELECT refresh_status, last_refresh_status FROM system.view_refreshes WHERE view = 'test'; From 3556c15c51fa143d4590c936edea15e0b434f009 Mon Sep 17 00:00:00 2001 From: koloshmet Date: Mon, 13 Feb 2023 13:23:34 +0200 Subject: [PATCH 03/25] fixed tests --- src/Access/Common/AccessType.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 1f0e678461f..0188cbb5b99 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -177,7 +177,7 @@ enum class AccessType M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \ M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \ M(SYSTEM_CLEANUP, "SYSTEM STOP CLEANUP, SYSTEM START CLEANUP", TABLE, SYSTEM) \ - M(SYSTEM_VIEWS, "SYSTEM REFRESH VIEW, START VIEWS, STOP VIEWS, CANCEL VIEW, PAUSE VIEW, RESUME VIEW", VIEW, SYSTEM) \ + M(SYSTEM_VIEWS, "SYSTEM REFRESH VIEW, SYSTEM START VIEWS, SYSTEM STOP VIEWS, SYSTEM START VIEW, SYSTEM STOP VIEW, SYSTEM CANCEL VIEW, SYSTEM PAUSE VIEW, SYSTEM RESUME VIEW, REFRESH VIEW, START VIEWS, STOP VIEWS, START VIEW, STOP VIEW, CANCEL VIEW, PAUSE VIEW, RESUME VIEW", VIEW, SYSTEM) \ M(SYSTEM_DISTRIBUTED_SENDS, "SYSTEM STOP DISTRIBUTED SENDS, SYSTEM START DISTRIBUTED SENDS, STOP DISTRIBUTED SENDS, START DISTRIBUTED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP REPLICATED SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \ M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \ From d1932763f38f4740a4f67aa78039a07c8ceb0545 Mon Sep 17 00:00:00 2001 From: koloshmet Date: Mon, 13 Feb 2023 13:41:10 +0200 Subject: [PATCH 04/25] fixed style --- src/Parsers/ASTTimeInterval.h | 2 +- src/Storages/MaterializedView/RefreshAllCombiner.h | 2 +- src/Storages/MaterializedView/RefreshDependencies.h | 2 +- src/Storages/MaterializedView/RefreshSet.cpp | 5 +++++ src/Storages/MaterializedView/RefreshSet.h | 9 ++------- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/MaterializedView/RefreshTask.h | 2 +- src/Storages/System/StorageSystemViewRefreshes.cpp | 2 +- 8 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Parsers/ASTTimeInterval.h b/src/Parsers/ASTTimeInterval.h index 374d1e1ec55..a8f2518d180 100644 --- a/src/Parsers/ASTTimeInterval.h +++ b/src/Parsers/ASTTimeInterval.h @@ -1,4 +1,4 @@ - #pragma once +#pragma once #include diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.h b/src/Storages/MaterializedView/RefreshAllCombiner.h index f4faf073ae4..3b74018f6cf 100644 --- a/src/Storages/MaterializedView/RefreshAllCombiner.h +++ b/src/Storages/MaterializedView/RefreshAllCombiner.h @@ -6,7 +6,7 @@ namespace DB { /// Concurrent primitive for dependency completeness registration -/// When arrive methods return true, dependant task must be executed (or scheduled) +/// When arrive methods return true, dependent task must be executed (or scheduled) class RefreshAllCombiner { public: diff --git a/src/Storages/MaterializedView/RefreshDependencies.h b/src/Storages/MaterializedView/RefreshDependencies.h index a4488053adf..8d370f96d40 100644 --- a/src/Storages/MaterializedView/RefreshDependencies.h +++ b/src/Storages/MaterializedView/RefreshDependencies.h @@ -12,7 +12,7 @@ namespace DB class RefreshTask; -/// Concurrent primitive for managing list of dependant task and notifying them +/// Concurrent primitive for managing list of dependent task and notifying them class RefreshDependencies { using Container = std::list; diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index c38d3408495..8cae074d4a3 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -1,6 +1,11 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric Refresh; +} + namespace DB { diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index 48962d8c96d..255fa4dbd66 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -6,15 +6,10 @@ #include -namespace CurrentMetrics -{ - extern const Metric Refresh; -} - -namespace DB +namespace DB { -struct RefreshInfo +struct RefreshInfo { String database; String view_name; diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 579d3252865..163ab3362eb 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -7,7 +7,7 @@ #include #include -namespace DB +namespace DB { namespace ErrorCodes diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 5e9bb618372..ea289562c7e 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -66,7 +66,7 @@ public: /// Resume task execution void resume(); - /// Notify dependant task + /// Notify dependent task void notify(const StorageID & parent_id); private: diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index 0ddf8a48c9d..e78059e1a88 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -38,7 +38,7 @@ void StorageSystemViewRefreshes::fillData( auto valid_access = AccessType::SHOW_TABLES; bool check_access_for_tables = !access->isGranted(valid_access); - + for (const auto & refresh : context->getRefreshSet().getInfo()) { if (check_access_for_tables && !access->isGranted(valid_access, refresh.database, refresh.view_name)) From f14114dafc3559ab80f146e1c476a48622b8ccf4 Mon Sep 17 00:00:00 2001 From: koloshmet Date: Wed, 15 Feb 2023 04:58:26 +0200 Subject: [PATCH 05/25] proper tmp table cleanup --- src/Storages/MaterializedView/RefreshTask.cpp | 19 ++++++++++++-- src/Storages/MaterializedView/RefreshTask.h | 4 ++- src/Storages/StorageMaterializedView.cpp | 25 +++++++++++-------- src/Storages/StorageMaterializedView.h | 6 ++--- 4 files changed, 37 insertions(+), 17 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 163ab3362eb..fdf3948de70 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -168,12 +169,14 @@ void RefreshTask::refresh() storeLastState(LastTaskState::Finished); break; case ExecutionResult::Cancelled: + cancelRefresh(view); storeLastState(LastTaskState::Canceled); break; } refresh_executor.reset(); refresh_block.reset(); + refresh_query.reset(); storeLastRefresh(std::chrono::system_clock::now()); scheduleRefresh(last_refresh); @@ -193,9 +196,12 @@ RefreshTask::ExecutionResult RefreshTask::executeRefresh() } -void RefreshTask::initializeRefresh(std::shared_ptr view) +void RefreshTask::initializeRefresh(std::shared_ptr view) { + auto fresh_table = view->createFreshTable(); refresh_query = view->prepareRefreshQuery(); + refresh_query->setTable(fresh_table.table_name); + refresh_query->setDatabase(fresh_table.database_name); auto refresh_context = Context::createCopy(view->getContext()); refresh_block = InterpreterInsertQuery(refresh_query, refresh_context).execute(); refresh_block->pipeline.setProgressCallback([this](const Progress & progress){ progressCallback(progress); }); @@ -208,8 +214,17 @@ void RefreshTask::initializeRefresh(std::shared_ptr vie void RefreshTask::completeRefresh(std::shared_ptr view) { - view->updateInnerTableAfterRefresh(refresh_query); + auto stale_table = view->exchangeTargetTable(refresh_query->table_id); dependencies.notifyAll(view->getStorageID()); + + auto drop_context = Context::createCopy(view->getContext()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, stale_table, /*sync=*/true); +} + +void RefreshTask::cancelRefresh(std::shared_ptr view) +{ + auto drop_context = Context::createCopy(view->getContext()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, refresh_query->table_id, /*sync=*/true); } void RefreshTask::scheduleRefresh(std::chrono::system_clock::time_point now) diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index ea289562c7e..36c03e2c7df 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -85,10 +85,12 @@ private: ExecutionResult executeRefresh(); - void initializeRefresh(std::shared_ptr view); + void initializeRefresh(std::shared_ptr view); void completeRefresh(std::shared_ptr view); + void cancelRefresh(std::shared_ptr view); + std::chrono::sys_seconds calculateRefreshTime(std::chrono::system_clock::time_point now) const; std::chrono::seconds genSpreadSeconds(); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 43e8e0d6f33..42191bde90e 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -285,7 +285,7 @@ bool StorageMaterializedView::optimize( return getTargetTable()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, local_context); } -std::shared_ptr StorageMaterializedView::prepareRefreshQuery() +StorageID StorageMaterializedView::createFreshTable() const { auto inner_table_id = getTargetTableId(); auto new_table_name = ".tmp" + generateInnerTableName(getStorageID()); @@ -305,26 +305,29 @@ std::shared_ptr StorageMaterializedView::prepareRefreshQuery() create_interpreter.setInternal(true); create_interpreter.execute(); - auto insert_query = std::make_shared(); - insert_query->setTable(new_table_name); - insert_query->setDatabase(db->getDatabaseName()); - insert_query->select = getInMemoryMetadataPtr()->getSelectQuery().select_query; + return DatabaseCatalog::instance().getTable({create_query.getDatabase(), create_query.getTable()}, getContext())->getStorageID(); +} +std::shared_ptr StorageMaterializedView::prepareRefreshQuery() const +{ + auto insert_query = std::make_shared(); + insert_query->select = getInMemoryMetadataPtr()->getSelectQuery().select_query; return insert_query; } -void StorageMaterializedView::updateInnerTableAfterRefresh(std::shared_ptr refresh_query) +StorageID StorageMaterializedView::exchangeTargetTable(const StorageID & fresh_table) { - auto inner_table_id = getTargetTableId(); + auto stale_table_id = getTargetTableId(); - auto db = DatabaseCatalog::instance().getDatabase(inner_table_id.database_name); - auto target_db = DatabaseCatalog::instance().getDatabase(refresh_query->getDatabase()); + auto db = DatabaseCatalog::instance().getDatabase(stale_table_id.database_name); + auto target_db = DatabaseCatalog::instance().getDatabase(fresh_table.database_name); auto rename_ctx = Context::createCopy(getContext()); target_db->renameTable( - rename_ctx, refresh_query->getTable(), *db, inner_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); + rename_ctx, fresh_table.table_name, *db, stale_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); - setTargetTableId(db->getTable(refresh_query->getTable(), getContext())->getStorageID()); + setTargetTableId(fresh_table); + return stale_table_id; } void StorageMaterializedView::alter( diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 2dce8355b47..f3b777d34fa 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -119,9 +119,9 @@ private: void checkStatementCanBeForwarded() const; - std::shared_ptr prepareRefreshQuery(); - - void updateInnerTableAfterRefresh(std::shared_ptr refresh_query); + StorageID createFreshTable() const; + std::shared_ptr prepareRefreshQuery() const; + StorageID exchangeTargetTable(const StorageID & fresh_table); StorageID getTargetTableId() const; void setTargetTableId(StorageID id); From f1161566b46757492f01b995ae047fabd4810bcb Mon Sep 17 00:00:00 2001 From: koloshmet Date: Wed, 15 Feb 2023 07:04:13 +0200 Subject: [PATCH 06/25] proper tmp table cleanup --- src/Storages/MaterializedView/RefreshTask.cpp | 8 +++++--- src/Storages/StorageMaterializedView.cpp | 6 ++++-- src/Storages/StorageMaterializedView.h | 2 +- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index fdf3948de70..fa5b3df75a2 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -188,6 +188,11 @@ RefreshTask::ExecutionResult RefreshTask::executeRefresh() while (!interrupt_execution.load() && not_finished) not_finished = refresh_executor->executeStep(interrupt_execution); + auto defer = make_scope_guard([this] + { + canceled.store(false); + interrupt_execution.store(false); + }); if (!not_finished) return ExecutionResult::Finished; if (interrupt_execution.load() && !canceled.load()) @@ -206,9 +211,6 @@ void RefreshTask::initializeRefresh(std::shared_ptrpipeline.setProgressCallback([this](const Progress & progress){ progressCallback(progress); }); - canceled.store(false); - interrupt_execution.store(false); - refresh_executor.emplace(refresh_block->pipeline); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 42191bde90e..0939ddcd91a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -315,7 +315,7 @@ std::shared_ptr StorageMaterializedView::prepareRefreshQuery() c return insert_query; } -StorageID StorageMaterializedView::exchangeTargetTable(const StorageID & fresh_table) +StorageID StorageMaterializedView::exchangeTargetTable(StorageID fresh_table) { auto stale_table_id = getTargetTableId(); @@ -326,7 +326,9 @@ StorageID StorageMaterializedView::exchangeTargetTable(const StorageID & fresh_t target_db->renameTable( rename_ctx, fresh_table.table_name, *db, stale_table_id.table_name, /*exchange=*/true, /*dictionary=*/false); - setTargetTableId(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; } diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index f3b777d34fa..ff7b50340c7 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -121,7 +121,7 @@ private: StorageID createFreshTable() const; std::shared_ptr prepareRefreshQuery() const; - StorageID exchangeTargetTable(const StorageID & fresh_table); + StorageID exchangeTargetTable(StorageID fresh_table); StorageID getTargetTableId() const; void setTargetTableId(StorageID id); From 808cb0fa0514e1c4b624961a46c406b873609e68 Mon Sep 17 00:00:00 2001 From: koloshmet Date: Wed, 22 Feb 2023 05:01:21 +0200 Subject: [PATCH 07/25] fix fix fix --- src/Access/tests/gtest_access_rights_ops.cpp | 2 +- src/Parsers/ParserRefreshStrategy.cpp | 12 +-- src/Storages/MaterializedView/RefreshSet.cpp | 1 - src/Storages/MaterializedView/RefreshTask.cpp | 89 ++++++++++++++----- src/Storages/MaterializedView/RefreshTask.h | 19 +++- src/Storages/StorageMaterializedView.cpp | 2 +- .../integration/test_grant_and_revoke/test.py | 2 +- ...1_refreshable_materialized_views.reference | 8 +- .../02661_refreshable_materialized_views.sql | 21 ++++- 9 files changed, 119 insertions(+), 37 deletions(-) diff --git a/src/Access/tests/gtest_access_rights_ops.cpp b/src/Access/tests/gtest_access_rights_ops.cpp index b5a15513a89..47c01d66570 100644 --- a/src/Access/tests/gtest_access_rights_ops.cpp +++ b/src/Access/tests/gtest_access_rights_ops.cpp @@ -51,7 +51,7 @@ TEST(AccessRights, Union) "CREATE DICTIONARY, DROP DATABASE, DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, " "TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, " "SHOW ROW POLICIES, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, " - "SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " + "SYSTEM MOVES, SYSTEM VIEWS, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, " "SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, " "SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM SYNC DATABASE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON db1.*, GRANT NAMED COLLECTION ADMIN ON db1"); } diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index a448556bd4f..a6fbb373ed3 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -28,14 +28,16 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { refresh->schedule_kind = ASTRefreshStrategy::ScheduleKind::EVERY; ASTPtr period; - ASTPtr periodic_offset; if (!ParserTimePeriod{}.parse(pos, period, expected)) return false; - if (!ParserTimeInterval{}.parse(pos, periodic_offset, expected)) - return false; - refresh->set(refresh->period, period); - refresh->set(refresh->periodic_offset, periodic_offset); + if (ParserKeyword{"OFFSET"}.ignore(pos, expected)) + { + ASTPtr periodic_offset; + if (!ParserTimeInterval{}.parse(pos, periodic_offset, expected)) + return false; + refresh->set(refresh->periodic_offset, periodic_offset); + } } if (refresh->schedule_kind == ASTRefreshStrategy::ScheduleKind::UNKNOWN) return false; diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 8cae074d4a3..528375bb951 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -68,7 +68,6 @@ bool RefreshSetLess::operator()(const StorageID & l, const StorageID & r) const RefreshSet::Entry::Entry() : parent_set{nullptr} - , metric_increment{} {} RefreshSet::Entry::Entry(Entry && other) noexcept diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index fa5b3df75a2..84d5bbfb84e 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -47,9 +47,9 @@ RefreshTask::RefreshTask( const ASTRefreshStrategy & strategy) : refresh_timer(makeRefreshTimer(strategy)) , refresh_spread{makeSpreadDistribution(strategy.spread)} + , canceled{false} , refresh_immediately{false} , interrupt_execution{false} - , canceled{false} {} RefreshTaskHolder RefreshTask::create( @@ -92,7 +92,7 @@ void RefreshTask::start() void RefreshTask::stop() { refresh_task->deactivate(); - cancel(); + cancelSync(); storeState(TaskState::Disabled); } @@ -104,20 +104,36 @@ void RefreshTask::run() void RefreshTask::cancel() { - canceled.store(true); - interrupt_execution.store(true); + std::lock_guard guard(state_mutex); + cancelLocked(); +} + +void RefreshTask::cancelSync() +{ + std::unique_lock lock(state_mutex); + cancelLocked(); + sync_canceled.wait(lock, [this] { return !canceled; }); } void RefreshTask::pause() { - interrupt_execution.store(true); + std::lock_guard guard(state_mutex); + if (state == TaskState::Running) + { + interrupt_execution.store(true); + state = TaskState::Paused; + } } void RefreshTask::resume() { - interrupt_execution.store(false); - refresh_immediately.store(true); - refresh_task->schedule(); + std::lock_guard guard(state_mutex); + if (state == TaskState::Paused) + { + refresh_immediately.store(true); + refresh_task->schedule(); + state = TaskState::Scheduled; + } } void RefreshTask::notify(const StorageID & parent_id) @@ -154,15 +170,17 @@ void RefreshTask::refresh() if (!view) return; + std::unique_lock lock(state_mutex); + if (!refresh_executor) initializeRefresh(view); storeState(TaskState::Running); - switch (executeRefresh()) + switch (executeRefresh(lock)) { case ExecutionResult::Paused: - storeState(TaskState::Paused); + pauseRefresh(view); return; case ExecutionResult::Finished: completeRefresh(view); @@ -174,28 +192,24 @@ void RefreshTask::refresh() break; } - refresh_executor.reset(); - refresh_block.reset(); - refresh_query.reset(); + cleanState(); storeLastRefresh(std::chrono::system_clock::now()); scheduleRefresh(last_refresh); } -RefreshTask::ExecutionResult RefreshTask::executeRefresh() +RefreshTask::ExecutionResult RefreshTask::executeRefresh(std::unique_lock & state_lock) { + state_lock.unlock(); + bool not_finished{true}; while (!interrupt_execution.load() && not_finished) not_finished = refresh_executor->executeStep(interrupt_execution); - auto defer = make_scope_guard([this] - { - canceled.store(false); - interrupt_execution.store(false); - }); + state_lock.lock(); if (!not_finished) return ExecutionResult::Finished; - if (interrupt_execution.load() && !canceled.load()) + if (interrupt_execution.load() && !canceled) return ExecutionResult::Paused; return ExecutionResult::Cancelled; @@ -227,6 +241,14 @@ void RefreshTask::cancelRefresh(std::shared_ptr v { auto drop_context = Context::createCopy(view->getContext()); InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, refresh_query->table_id, /*sync=*/true); + interrupt_execution.store(false); + if (std::exchange(canceled, false)) + sync_canceled.notify_all(); +} + +void RefreshTask::pauseRefresh(std::shared_ptr /*view*/) +{ + interrupt_execution.store(false); } void RefreshTask::scheduleRefresh(std::chrono::system_clock::time_point now) @@ -275,6 +297,31 @@ void RefreshTask::progressCallback(const Progress & progress) set_entry->elapsed_ns.store(progress.elapsed_ns, std::memory_order_relaxed); } +void RefreshTask::cancelLocked() +{ + switch (state) + { + case TaskState::Running: + canceled = true; + interrupt_execution.store(true); + break; + case TaskState::Paused: + if (auto view = lockView()) + cancelRefresh(view); + cleanState(); + break; + default: + break; + } +} + +void RefreshTask::cleanState() +{ + refresh_executor.reset(); + refresh_block.reset(); + refresh_query.reset(); +} + std::shared_ptr RefreshTask::lockView() { return std::static_pointer_cast(view_to_refresh.lock()); @@ -282,7 +329,7 @@ std::shared_ptr RefreshTask::lockView() void RefreshTask::storeState(TaskState task_state) { - state.store(task_state); + state = task_state; set_entry->state.store(static_cast(task_state)); } diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 36c03e2c7df..894b7f2fea0 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -38,7 +38,7 @@ public: }; /// Never call it manual, public for shared_ptr construction only - RefreshTask(const ASTRefreshStrategy & strategy); + explicit RefreshTask(const ASTRefreshStrategy & strategy); /// The only proper way to construct task static RefreshTaskHolder create( @@ -60,6 +60,9 @@ public: /// Cancel task execution void cancel(); + /// Cancel task execution synchronously + void cancelSync(); + /// Pause task execution (must be either resumed or canceled later) void pause(); @@ -83,7 +86,7 @@ private: void refresh(); - ExecutionResult executeRefresh(); + ExecutionResult executeRefresh(std::unique_lock & state_lock); void initializeRefresh(std::shared_ptr view); @@ -91,6 +94,8 @@ private: void cancelRefresh(std::shared_ptr view); + void pauseRefresh(std::shared_ptr view); + std::chrono::sys_seconds calculateRefreshTime(std::chrono::system_clock::time_point now) const; std::chrono::seconds genSpreadSeconds(); @@ -106,6 +111,10 @@ private: }; } + void cancelLocked(); + + void cleanState(); + std::shared_ptr lockView(); void storeState(TaskState task_state); @@ -140,13 +149,15 @@ private: std::uniform_int_distribution refresh_spread; /// Task state - std::atomic state{TaskState::Disabled}; + std::mutex state_mutex; + std::condition_variable sync_canceled; + TaskState state{TaskState::Disabled}; LastTaskState last_state{LastTaskState::Unknown}; + bool canceled; /// Outer triggers std::atomic_bool refresh_immediately; std::atomic_bool interrupt_execution; - std::atomic_bool canceled; }; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 0939ddcd91a..9cb21bdb06f 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -282,7 +282,7 @@ bool StorageMaterializedView::optimize( checkStatementCanBeForwarded(); auto storage_ptr = getTargetTable(); auto metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); - return getTargetTable()->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, local_context); + return storage_ptr->optimize(query, metadata_snapshot, partition, final, deduplicate, deduplicate_by_columns, local_context); } StorageID StorageMaterializedView::createFreshTable() const diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index c8a0ee541e2..b257a551d57 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -188,7 +188,7 @@ def test_grant_all_on_table(): instance.query("SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER TABLE, ALTER VIEW, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, " "DROP TABLE, DROP VIEW, DROP DICTIONARY, UNDROP TABLE, TRUNCATE, OPTIMIZE, BACKUP, CREATE ROW POLICY, ALTER ROW POLICY, DROP ROW POLICY, SHOW ROW POLICIES, " - "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, " + "SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM VIEWS, SYSTEM PULLING REPLICATION LOG, SYSTEM CLEANUP, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, " "SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM WAIT LOADING PARTS, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" ) instance.query("REVOKE ALL ON test.table FROM B", user="A") diff --git a/tests/queries/0_stateless/02661_refreshable_materialized_views.reference b/tests/queries/0_stateless/02661_refreshable_materialized_views.reference index 663d951b8b5..bbb855348f7 100644 --- a/tests/queries/0_stateless/02661_refreshable_materialized_views.reference +++ b/tests/queries/0_stateless/02661_refreshable_materialized_views.reference @@ -1 +1,7 @@ -Scheduled Unknown +test +test +test1 +test2 +test Disabled +test1 Disabled +test2 Disabled diff --git a/tests/queries/0_stateless/02661_refreshable_materialized_views.sql b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql index 0b8a6a5289e..4e5420509c1 100644 --- a/tests/queries/0_stateless/02661_refreshable_materialized_views.sql +++ b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql @@ -1,3 +1,20 @@ -CREATE MATERIALIZED VIEW test REFRESH AFTER 15 SECOND ENGINE = MergeTree() ORDER BY number AS SELECT * FROM system.numbers LIMIT 10; +CREATE MATERIALIZED VIEW test REFRESH AFTER 15 SECOND ENGINE = MergeTree() ORDER BY number AS SELECT * FROM system.numbers LIMIT 10000000; -SELECT refresh_status, last_refresh_status FROM system.view_refreshes WHERE view = 'test'; +SELECT view FROM system.view_refreshes WHERE view = 'test'; + +CREATE MATERIALIZED VIEW test1 REFRESH EVERY 1 HOUR ENGINE = MergeTree() ORDER BY number AS SELECT * FROM test; + +CREATE MATERIALIZED VIEW test2 REFRESH EVERY 2 HOUR OFFSET 42 MINUTE 8 SECOND RANDOMIZE FOR 10 MINUTE ENGINE = MergeTree() ORDER BY number AS SELECT * FROM test; + +SELECT view FROM system.view_refreshes WHERE view LIKE 'test%' ORDER BY view; + +SYSTEM STOP VIEW test; +SYSTEM STOP VIEWS; + +SELECT view, refresh_status FROM system.view_refreshes WHERE view LIKE 'test%' ORDER BY view; + +SYSTEM START VIEWS; + +DROP VIEW test; +DROP VIEW test1; +DROP VIEW test2; From 5dc04a13a79c6f83600df5ec03e8d26f5717d2a2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 18 Nov 2023 00:45:05 +0000 Subject: [PATCH 08/25] Simple review comments --- src/Common/CurrentMetrics.cpp | 2 +- src/Parsers/ASTRefreshStrategy.cpp | 17 ++++++++--------- src/Parsers/ASTTimeInterval.cpp | 8 ++++---- src/Parsers/ParserTimeInterval.cpp | 10 +++++++++- src/Storages/MaterializedView/RefreshSet.cpp | 4 ++-- src/Storages/MaterializedView/RefreshTask.cpp | 4 ++-- src/Storages/MaterializedView/RefreshTimers.cpp | 4 ++-- .../System/StorageSystemViewRefreshes.cpp | 3 +++ .../0_stateless/01271_show_privileges.reference | 1 + 9 files changed, 32 insertions(+), 21 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index fccdeaa3c57..87de19b8907 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -253,7 +253,7 @@ M(MergeTreeAllRangesAnnouncementsSent, "The current number of announcement being sent in flight from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ M(CreatedTimersInQueryProfiler, "Number of Created thread local timers in QueryProfiler") \ M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \ - M(Refresh, "Number of active refreshes") \ + M(RefreshingViews, "Number of active refreshes") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp index f168bcc09c1..ed8216cf4aa 100644 --- a/src/Parsers/ASTRefreshStrategy.cpp +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -22,8 +22,6 @@ ASTPtr ASTRefreshStrategy::clone() const res->set(res->settings, settings->clone()); if (dependencies) res->set(res->dependencies, dependencies->clone()); - res->interval = interval; - res->spread = spread; res->schedule_kind = schedule_kind; return res; } @@ -33,40 +31,41 @@ void ASTRefreshStrategy::formatImpl( { frame.need_parens = false; - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << "REFRESH "; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << "REFRESH " << (f_settings.hilite ? hilite_none : ""); using enum ScheduleKind; switch (schedule_kind) { case AFTER: - f_settings.ostr << "AFTER "; + f_settings.ostr << "AFTER " << (f_settings.hilite ? hilite_none : ""); interval->formatImpl(f_settings, state, frame); break; case EVERY: - f_settings.ostr << "EVERY "; + f_settings.ostr << "EVERY " << (f_settings.hilite ? hilite_none : ""); period->formatImpl(f_settings, state, frame); if (periodic_offset) { - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " OFFSET "; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " OFFSET " << (f_settings.hilite ? hilite_none : ""); periodic_offset->formatImpl(f_settings, state, frame); } break; default: + f_settings.ostr << (f_settings.hilite ? hilite_none : ""); break; } if (spread) { - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " RANDOMIZE FOR "; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " RANDOMIZE FOR " << (f_settings.hilite ? hilite_none : ""); spread->formatImpl(f_settings, state, frame); } if (dependencies) { - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " DEPENDS ON "; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " DEPENDS ON " << (f_settings.hilite ? hilite_none : ""); dependencies->formatImpl(f_settings, state, frame); } if (settings) { - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS "; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " SETTINGS " << (f_settings.hilite ? hilite_none : ""); settings->formatImpl(f_settings, state, frame); } } diff --git a/src/Parsers/ASTTimeInterval.cpp b/src/Parsers/ASTTimeInterval.cpp index 4edda531202..13e07f4ed65 100644 --- a/src/Parsers/ASTTimeInterval.cpp +++ b/src/Parsers/ASTTimeInterval.cpp @@ -15,8 +15,8 @@ ASTPtr ASTTimePeriod::clone() const void ASTTimePeriod::formatImpl(const FormatSettings & f_settings, FormatState &, FormatStateStacked frame) const { frame.need_parens = false; - f_settings.ostr << (f_settings.hilite ? hilite_none : "") << value << ' '; - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword(); + f_settings.ostr << value << ' '; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword() << (f_settings.hilite ? hilite_none : ""); } ASTPtr ASTTimeInterval::clone() const @@ -32,8 +32,8 @@ void ASTTimeInterval::formatImpl(const FormatSettings & f_settings, FormatState { if (!std::exchange(is_first, false)) f_settings.ostr << ' '; - f_settings.ostr << (f_settings.hilite ? hilite_none : "") << value << ' '; - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword(); + f_settings.ostr << value << ' '; + f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword() << (f_settings.hilite ? hilite_none : ""); } } diff --git a/src/Parsers/ParserTimeInterval.cpp b/src/Parsers/ParserTimeInterval.cpp index dac66883083..8168ae61a33 100644 --- a/src/Parsers/ParserTimeInterval.cpp +++ b/src/Parsers/ParserTimeInterval.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + namespace { @@ -27,7 +32,10 @@ std::optional parseValKind(IParser::Pos & pos, Expected & expected) return ValKind{ .empty = true }; if (!parseIntervalKind(pos, expected, kind)) return {}; - return ValKind{ value->as().value.safeGet(), kind, false }; + UInt64 val; + if (!value->as().value.tryGet(val)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Time interval must be an integer"); + return ValKind{ val, kind, false }; } } diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 528375bb951..978fab26af0 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -3,7 +3,7 @@ namespace CurrentMetrics { - extern const Metric Refresh; + extern const Metric RefreshingViews; } namespace DB @@ -102,7 +102,7 @@ void RefreshSet::Entry::cleanup(RefreshSet * set) } RefreshSet::RefreshSet() - : set_metric(CurrentMetrics::Refresh) + : set_metric(CurrentMetrics::RefreshingViews) {} RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 84d5bbfb84e..6ab9fcb281c 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -37,7 +37,7 @@ std::variant makeRefreshTimer(const ASTRef case AFTER: return RefreshAfterTimer{strategy.interval}; default: - throw Exception("Unknown refresh strategy kind", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown refresh strategy kind"); } } @@ -64,7 +64,7 @@ RefreshTaskHolder RefreshTask::create( if (strategy.dependencies) { if (strategy.schedule_kind != ASTRefreshStrategy::ScheduleKind::AFTER) - throw Exception("Dependencies are allowed only for AFTER refresh kind", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dependencies are allowed only for AFTER refresh kind"); task->deps_entries.reserve(strategy.dependencies->children.size()); for (auto && dependency : strategy.dependencies->children) diff --git a/src/Storages/MaterializedView/RefreshTimers.cpp b/src/Storages/MaterializedView/RefreshTimers.cpp index 973eba46057..ebef561fc29 100644 --- a/src/Storages/MaterializedView/RefreshTimers.cpp +++ b/src/Storages/MaterializedView/RefreshTimers.cpp @@ -116,7 +116,7 @@ std::chrono::sys_seconds RefreshEveryTimer::alignedToYears(std::chrono::system_c if (auto prev_time = offset.after(prev_years); prev_time > tp) return prev_time; - auto next_years = normalize_years(tp_ymd.year() + std::chrono::years{1}); + auto next_years = normalize_years(std::chrono::year((int(tp_ymd.year()) / value + 1) * value)); return offset.after(next_years); } @@ -234,7 +234,7 @@ std::chrono::sys_seconds RefreshEveryTimer::alignedToSeconds(std::chrono::system auto tp_minutes = std::chrono::floor(tp); auto tp_seconds = std::chrono::floor(tp - tp_minutes); - auto next_seconds= (tp_seconds / value + 1s) * value; + auto next_seconds = (tp_seconds / value + 1s) * value; if (std::chrono::floor(next_seconds - 1s) > 0min) return tp_minutes + 1min + std::chrono::seconds{value}; return tp_minutes + next_seconds; diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index e78059e1a88..f30e2863c4a 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -1,6 +1,9 @@ #include #include +#include +#include +#include #include #include diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index e2c0655b2bc..b482cf7bd13 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -127,6 +127,7 @@ SYSTEM FETCHES ['SYSTEM STOP FETCHES','SYSTEM START FETCHES','STOP FETCHES','STA SYSTEM MOVES ['SYSTEM STOP MOVES','SYSTEM START MOVES','STOP MOVES','START MOVES'] TABLE SYSTEM SYSTEM PULLING REPLICATION LOG ['SYSTEM STOP PULLING REPLICATION LOG','SYSTEM START PULLING REPLICATION LOG'] TABLE SYSTEM SYSTEM CLEANUP ['SYSTEM STOP CLEANUP','SYSTEM START CLEANUP'] TABLE SYSTEM +SYSTEM VIEWS ['SYSTEM REFRESH VIEW','SYSTEM START VIEWS','SYSTEM STOP VIEWS','SYSTEM START VIEW','SYSTEM STOP VIEW','SYSTEM CANCEL VIEW','SYSTEM PAUSE VIEW','SYSTEM RESUME VIEW','REFRESH VIEW','START VIEWS','STOP VIEWS','START VIEW','STOP VIEW','CANCEL VIEW','PAUSE VIEW','RESUME VIEW'] VIEW SYSTEM SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBUTED SENDS','STOP DISTRIBUTED SENDS','START DISTRIBUTED SENDS'] TABLE SYSTEM SENDS SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP REPLICATED SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM From 01345981e286e7601a2ad22516af7d9f90922598 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 23 Nov 2023 05:08:44 +0000 Subject: [PATCH 09/25] Overhaul RefreshTask --- src/Interpreters/ActionLocksManager.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 8 +- .../MaterializedView/RefreshAllCombiner.h | 1 + src/Storages/MaterializedView/RefreshSet.cpp | 12 +- src/Storages/MaterializedView/RefreshSet.h | 40 +- src/Storages/MaterializedView/RefreshTask.cpp | 419 +++++++++++------- src/Storages/MaterializedView/RefreshTask.h | 141 +++--- src/Storages/StorageMaterializedView.cpp | 3 +- .../System/StorageSystemViewRefreshes.cpp | 4 +- .../02661_refreshable_materialized_views.sql | 2 +- 10 files changed, 347 insertions(+), 285 deletions(-) diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 43b49b024aa..65f13ebd66c 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -18,7 +18,7 @@ namespace ActionLocks extern const StorageActionBlockType PartsMove = 7; extern const StorageActionBlockType PullReplicationLog = 8; extern const StorageActionBlockType Cleanup = 9; - extern const StorageActionBlockType ViewRefresh = 8; + extern const StorageActionBlockType ViewRefresh = 10; } diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 2f504e97857..7df6499fd39 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -609,16 +609,12 @@ BlockIO InterpreterSystemQuery::execute() case Type::START_CLEANUP: startStopAction(ActionLocks::Cleanup, true); break; + case Type::START_VIEW: case Type::START_VIEWS: startStopAction(ActionLocks::ViewRefresh, true); break; - case Type::STOP_VIEWS: - startStopAction(ActionLocks::ViewRefresh, false); - break; - case Type::START_VIEW: - startStopAction(ActionLocks::ViewRefresh, true); - break; case Type::STOP_VIEW: + case Type::STOP_VIEWS: startStopAction(ActionLocks::ViewRefresh, false); break; case Type::REFRESH_VIEW: diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.h b/src/Storages/MaterializedView/RefreshAllCombiner.h index 3b74018f6cf..f9f3a8d319c 100644 --- a/src/Storages/MaterializedView/RefreshAllCombiner.h +++ b/src/Storages/MaterializedView/RefreshAllCombiner.h @@ -7,6 +7,7 @@ namespace DB /// Concurrent primitive for dependency completeness registration /// When arrive methods return true, dependent task must be executed (or scheduled) +/// TODO: Doesn't need to be thread safe. class RefreshAllCombiner { public: diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 978fab26af0..9efd82e1afc 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -9,7 +9,7 @@ namespace CurrentMetrics namespace DB { -RefreshSetElement::RefreshSetElement(RefreshTaskHolder task, StorageID id) +RefreshSetElement::RefreshSetElement(StorageID id, RefreshTaskHolder task) : corresponding_task(task) , view_id(std::move(id)) {} @@ -19,8 +19,8 @@ RefreshInfo RefreshSetElement::getInfo() const return { .database = view_id.getDatabaseName(), .view_name = view_id.getTableName(), - .refresh_status = toString(RefreshTask::TaskState{state.load()}), - .last_refresh_status = toString(RefreshTask::LastTaskState{last_state.load()}), + .refresh_status = toString(RefreshTask::RefreshState{state.load()}), + .last_refresh_result = toString(RefreshTask::LastTaskResult{last_result.load()}), .last_refresh_time = static_cast(last_s.load(std::memory_order_relaxed)), .next_refresh_time = static_cast(next_s.load(std::memory_order_relaxed)), .progress = static_cast(written_rows) / total_rows_to_read, @@ -108,8 +108,8 @@ RefreshSet::RefreshSet() RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const { std::lock_guard lock(elements_mutex); - if (auto element = elements.find(id); element != elements.end()) - return element->getTask(); + if (auto element = elements.find(id.uuid); element != elements.end()) + return element->second.getTask(); return nullptr; } @@ -119,7 +119,7 @@ RefreshSet::InfoContainer RefreshSet::getInfo() const InfoContainer res; res.reserve(elements.size()); for (auto && element : elements) - res.emplace_back(element.getInfo()); + res.emplace_back(element.second.getInfo()); return res; } diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index 255fa4dbd66..cc5b0006218 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -14,7 +14,7 @@ struct RefreshInfo String database; String view_name; String refresh_status; - String last_refresh_status; + String last_refresh_result; UInt32 last_refresh_time; UInt32 next_refresh_time; Float64 progress; @@ -33,7 +33,7 @@ class RefreshSetElement { friend class RefreshTask; public: - RefreshSetElement(RefreshTaskHolder task, StorageID id); + RefreshSetElement(StorageID id, RefreshTaskHolder task); RefreshSetElement(const RefreshSetElement &) = delete; RefreshSetElement & operator=(const RefreshSetElement &) = delete; @@ -48,19 +48,19 @@ private: RefreshTaskObserver corresponding_task; StorageID view_id; - mutable std::atomic read_rows{0}; - mutable std::atomic read_bytes{0}; - mutable std::atomic total_rows_to_read{0}; - mutable std::atomic total_bytes_to_read{0}; - mutable std::atomic written_rows{0}; - mutable std::atomic written_bytes{0}; - mutable std::atomic result_rows{0}; - mutable std::atomic result_bytes{0}; - mutable std::atomic elapsed_ns{0}; - mutable std::atomic last_s{0}; - mutable std::atomic next_s{0}; - mutable std::atomic state{0}; - mutable std::atomic last_state{0}; + std::atomic read_rows{0}; + std::atomic read_bytes{0}; + std::atomic total_rows_to_read{0}; + std::atomic total_bytes_to_read{0}; + std::atomic written_rows{0}; + std::atomic written_bytes{0}; + std::atomic result_rows{0}; + std::atomic result_bytes{0}; + std::atomic elapsed_ns{0}; + std::atomic last_s{0}; + std::atomic next_s{0}; + std::atomic state{0}; + std::atomic last_result{0}; }; struct RefreshSetLess @@ -77,7 +77,7 @@ struct RefreshSetLess class RefreshSet { private: - using Container = std::set; + using Container = std::map; using ContainerIter = typename Container::iterator; public: @@ -92,7 +92,7 @@ public: ~Entry(); - const RefreshSetElement * operator->() const { return std::to_address(iter); } + RefreshSetElement * operator->() { return &iter->second; } private: RefreshSet * parent_set; @@ -111,11 +111,11 @@ public: RefreshSet(); - template - std::optional emplace(Args &&... args) + std::optional emplace(StorageID id, RefreshTaskHolder task) { std::lock_guard guard(elements_mutex); - if (auto [it, is_inserted] = elements.emplace(std::forward(args)...); is_inserted) + auto [it, is_inserted] = elements.emplace(std::piecewise_construct, std::forward_as_tuple(id.uuid), std::forward_as_tuple(id, std::move(task))); + if (is_inserted) return Entry(*this, std::move(it), set_metric); return {}; } diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 6ab9fcb281c..0a85f533a27 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -45,11 +45,9 @@ std::variant makeRefreshTimer(const ASTRef RefreshTask::RefreshTask( const ASTRefreshStrategy & strategy) - : refresh_timer(makeRefreshTimer(strategy)) + : log(&Poco::Logger::get("RefreshTask")) + , refresh_timer(makeRefreshTimer(strategy)) , refresh_spread{makeSpreadDistribution(strategy.spread)} - , canceled{false} - , refresh_immediately{false} - , interrupt_execution{false} {} RefreshTaskHolder RefreshTask::create( @@ -59,8 +57,13 @@ RefreshTaskHolder RefreshTask::create( { auto task = std::make_shared(strategy); - task->refresh_task = context->getSchedulePool().createTask("MaterializedViewRefresherTask", task->makePoolTask()); - task->set_entry = context->getRefreshSet().emplace(task, view.getStorageID()).value(); + task->refresh_task = context->getSchedulePool().createTask("MaterializedViewRefresherTask", + [self = task->weak_from_this()] + { + if (auto t = self.lock()) + t->refreshTask(); + }); + task->set_entry = context->getRefreshSet().emplace(view.getStorageID(), task).value(); if (strategy.dependencies) { if (strategy.schedule_kind != ASTRefreshStrategy::ScheduleKind::AFTER) @@ -70,153 +73,224 @@ RefreshTaskHolder RefreshTask::create( for (auto && dependency : strategy.dependencies->children) { StorageID dep_id(dependency->as()); + /// TODO: + /// * This depends on the order in which different tables are initialized. + /// Is the order guaranteed on startup? + /// * At what point does the table name from the query get mapped to the table's UUID? + /// Does it work at all? Is it reliable? + /// * Don't silently ignore if the table is missing. if (auto dep_task = context->getRefreshSet().getTask(dep_id)) - task->deps_entries.push_back(dep_task->dependencies.add(task)); + task->deps_entries.push_back(dep_task->dependencies.add(task)); } + + /// TODO: Initialize combiner. } return task; } -void RefreshTask::initialize(std::shared_ptr view) +void RefreshTask::initializeAndStart(std::shared_ptr view) { view_to_refresh = view; + /// TODO: Add a setting to stop views on startup, set `stop_requested = true` in that case. + calculateNextRefreshTime(std::chrono::system_clock::now()); + refresh_task->schedule(); } void RefreshTask::start() { - storeState(TaskState::Scheduled); - refresh_task->activateAndSchedule(); + std::lock_guard guard(mutex); + if (!std::exchange(stop_requested, false)) + return; + refresh_task->schedule(); } void RefreshTask::stop() { - refresh_task->deactivate(); - cancelSync(); - storeState(TaskState::Disabled); + std::lock_guard guard(mutex); + if (std::exchange(stop_requested, true)) + return; + interrupt_execution.store(true); + refresh_task->schedule(); } void RefreshTask::run() { - refresh_immediately.store(true); - refresh_task->activateAndSchedule(); + std::lock_guard guard(mutex); + if (std::exchange(refresh_immediately, true)) + return; + refresh_task->schedule(); } void RefreshTask::cancel() { - std::lock_guard guard(state_mutex); - cancelLocked(); -} - -void RefreshTask::cancelSync() -{ - std::unique_lock lock(state_mutex); - cancelLocked(); - sync_canceled.wait(lock, [this] { return !canceled; }); + std::lock_guard guard(mutex); + if (std::exchange(cancel_requested, true)) + return; + interrupt_execution.store(true); + refresh_task->schedule(); } void RefreshTask::pause() { - std::lock_guard guard(state_mutex); - if (state == TaskState::Running) - { - interrupt_execution.store(true); - state = TaskState::Paused; - } + std::lock_guard guard(mutex); + if (std::exchange(pause_requested, true)) + return; + interrupt_execution.store(true); + refresh_task->schedule(); } void RefreshTask::resume() { - std::lock_guard guard(state_mutex); - if (state == TaskState::Paused) - { - refresh_immediately.store(true); - refresh_task->schedule(); - state = TaskState::Scheduled; - } + std::lock_guard guard(mutex); + if (!std::exchange(pause_requested, false)) + return; + refresh_task->schedule(); } void RefreshTask::notify(const StorageID & parent_id) { - if (combiner.arriveParent(parent_id)) - { - refresh_immediately.store(true); - refresh_task->schedule(); - } -} - -void RefreshTask::doRefresh() -{ - if (refresh_immediately.exchange(false)) - { - refresh(); - } - else - { - auto now = std::chrono::system_clock::now(); - if (now >= next_refresh) - { - if (combiner.arriveTime()) - refresh(); - } - else - scheduleRefresh(now); - } -} - -void RefreshTask::refresh() -{ - auto view = lockView(); - if (!view) + std::lock_guard guard(mutex); + if (!combiner.arriveParent(parent_id)) return; - - std::unique_lock lock(state_mutex); - - if (!refresh_executor) - initializeRefresh(view); - - storeState(TaskState::Running); - - switch (executeRefresh(lock)) - { - case ExecutionResult::Paused: - pauseRefresh(view); - return; - case ExecutionResult::Finished: - completeRefresh(view); - storeLastState(LastTaskState::Finished); - break; - case ExecutionResult::Cancelled: - cancelRefresh(view); - storeLastState(LastTaskState::Canceled); - break; - } - - cleanState(); - - storeLastRefresh(std::chrono::system_clock::now()); - scheduleRefresh(last_refresh); + if (std::exchange(refresh_immediately, true)) + return; + refresh_task->schedule(); } -RefreshTask::ExecutionResult RefreshTask::executeRefresh(std::unique_lock & state_lock) +void RefreshTask::refreshTask() { - state_lock.unlock(); + try + { + std::unique_lock lock(mutex); - bool not_finished{true}; - while (!interrupt_execution.load() && not_finished) - not_finished = refresh_executor->executeStep(interrupt_execution); + /// Whoever breaks out of this loop should call reportState() first. + while (true) + { + chassert(lock.owns_lock()); - state_lock.lock(); - if (!not_finished) - return ExecutionResult::Finished; - if (interrupt_execution.load() && !canceled) - return ExecutionResult::Paused; - return ExecutionResult::Cancelled; + interrupt_execution.store(false); + /// Discard the active refresh if requested. + if ((stop_requested || cancel_requested) && refresh_executor) + { + lock.unlock(); + cancelRefresh(LastTaskResult::Canceled); + lock.lock(); + + if (cancel_requested) + { + /// Advance to the next refresh time according to schedule. + /// Otherwise we'd start another refresh immediately after canceling this one. + auto now = std::chrono::system_clock::now(); + if (now >= next_refresh_with_spread) + calculateNextRefreshTime(std::chrono::system_clock::now()); + } + } + + cancel_requested = false; + + if (pause_requested && !refresh_executor) + pause_requested = false; // no refresh to pause + + if (stop_requested || pause_requested) + { + /// Exit the task and wait for the user to start or resume, which will schedule the task again. + reportState(stop_requested ? RefreshState::Disabled : RefreshState::Paused); + break; + } + + if (!refresh_immediately && !refresh_executor) + { + auto now = std::chrono::system_clock::now(); + if (now >= next_refresh_with_spread) + { + if (combiner.arriveTime()) + refresh_immediately = true; + else + { + /// TODO: Put the array of remaining dependencies in RefreshSet, report it in the system table (update it from notify() too). + reportState(RefreshState::WaitingForDependencies); + break; + } + } + else + { + refresh_task->scheduleAfter( + std::chrono::duration_cast( + next_refresh_with_spread - now).count()); + reportState(RefreshState::Scheduled); + break; + } + } + + /// Perform a refresh. + + refresh_immediately = false; + + auto view = lockView(); + if (!view) + { + /// The view was dropped. This RefreshTask should be destroyed soon too. + /// (Maybe this is unreachable.) + reportState(RefreshState::Disabled); + break; + } + + reportState(RefreshState::Running); + + lock.unlock(); + + bool finished = false; + + try + { + if (!refresh_executor) + initializeRefresh(view); + + finished = executeRefresh(); + + if (finished) + completeRefresh(view, LastTaskResult::Finished); + } + catch (...) + { + tryLogCurrentException(log, "Refresh failed"); + + /// Don't leave a trash table. + if (!finished && refresh_query) + cancelRefresh(LastTaskResult::Exception); + + /// TODO: Put the exception message into RefreshSet, report it in the system table. + /// TODO: Backoff. Maybe just assigning next_refresh_* will work. + } + + lock.lock(); + + if (finished) + { + auto now = std::chrono::system_clock::now(); + reportLastRefreshTime(now); + calculateNextRefreshTime(now); + } + } + } + catch (...) + { + std::unique_lock lock(mutex); + stop_requested = true; + tryLogCurrentException(log, + "Unexpected exception in refresh scheduling, please investigate. The view will be stopped."); +#ifdef ABORT_ON_LOGICAL_ERROR + abortOnFailedAssertion("Unexpected exception in refresh scheduling"); +#endif + } } void RefreshTask::initializeRefresh(std::shared_ptr view) { + chassert(!refresh_query); + auto fresh_table = view->createFreshTable(); refresh_query = view->prepareRefreshQuery(); refresh_query->setTable(fresh_table.table_name); @@ -228,37 +302,53 @@ void RefreshTask::initializeRefresh(std::shared_ptrpipeline); } -void RefreshTask::completeRefresh(std::shared_ptr view) +bool RefreshTask::executeRefresh() +{ + bool not_finished{true}; + while (!interrupt_execution.load() && not_finished) + not_finished = refresh_executor->executeStep(interrupt_execution); + + return !not_finished; +} + +void RefreshTask::completeRefresh(std::shared_ptr view, LastTaskResult result) { auto stale_table = view->exchangeTargetTable(refresh_query->table_id); dependencies.notifyAll(view->getStorageID()); auto drop_context = Context::createCopy(view->getContext()); InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, stale_table, /*sync=*/true); + + cleanState(); + reportLastResult(result); } -void RefreshTask::cancelRefresh(std::shared_ptr view) +void RefreshTask::cancelRefresh(LastTaskResult result) { - auto drop_context = Context::createCopy(view->getContext()); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, refresh_query->table_id, /*sync=*/true); - interrupt_execution.store(false); - if (std::exchange(canceled, false)) - sync_canceled.notify_all(); + if (auto view = lockView()) + { + try + { + auto drop_context = Context::createCopy(view->getContext()); + InterpreterDropQuery::executeDropQuery( + ASTDropQuery::Kind::Drop, drop_context, drop_context, refresh_query->table_id, /*sync=*/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. + } + } + + cleanState(); + reportLastResult(result); } -void RefreshTask::pauseRefresh(std::shared_ptr /*view*/) +void RefreshTask::cleanState() { - interrupt_execution.store(false); -} - -void RefreshTask::scheduleRefresh(std::chrono::system_clock::time_point now) -{ - using namespace std::chrono_literals; - auto scheduled_refresh = calculateRefreshTime(now) + genSpreadSeconds(); - storeNextRefresh(scheduled_refresh); - auto schedule_time = std::chrono::ceil(scheduled_refresh - now); - storeState(TaskState::Scheduled); - refresh_task->scheduleAfter(std::max(schedule_time, 0ms).count()); + refresh_executor.reset(); + refresh_block.reset(); + refresh_query.reset(); } namespace @@ -271,17 +361,38 @@ CombinedVisitor(Ts...) -> CombinedVisitor; } -std::chrono::sys_seconds RefreshTask::calculateRefreshTime(std::chrono::system_clock::time_point now) const +void RefreshTask::calculateNextRefreshTime(std::chrono::system_clock::time_point now) { - CombinedVisitor refresh_time_visitor{ - [now](const RefreshAfterTimer & timer) { return timer.after(now); }, - [now](const RefreshEveryTimer & timer) { return timer.next(now); }}; - return std::visit(std::move(refresh_time_visitor), refresh_timer); + /// TODO: Add a setting to randomize initial delay in case of AFTER, for the case when the server + /// is restarted more often than the refresh period. + /// TODO: Maybe do something like skip_update_after_seconds and skip_update_after_ratio. + /// Unclear if that's useful at all if the last refresh timestamp is not remembered across restarts. + + auto advance = [&](std::chrono::system_clock::time_point t) + { + CombinedVisitor refresh_time_visitor{ + [t](const RefreshAfterTimer & timer) { return timer.after(t); }, + [t](const RefreshEveryTimer & timer) { return timer.next(t); }}; + auto r = std::visit(std::move(refresh_time_visitor), refresh_timer); + chassert(r > t); + return r; + }; + + /// It's important to use time without spread here, otherwise we would do multiple refreshes instead + /// of one, if the generated spread is negative and the first refresh completes faster than the spread. + std::chrono::sys_seconds next = advance(next_refresh_without_spread); + if (next < now) + next = advance(now); // fell behind, skip to current time + + next_refresh_without_spread = next; + next_refresh_with_spread = next + std::chrono::seconds{refresh_spread(thread_local_rng)}; + + reportNextRefreshTime(next_refresh_with_spread); } -std::chrono::seconds RefreshTask::genSpreadSeconds() +std::shared_ptr RefreshTask::lockView() { - return std::chrono::seconds{refresh_spread(thread_local_rng)}; + return std::static_pointer_cast(view_to_refresh.lock()); } void RefreshTask::progressCallback(const Progress & progress) @@ -297,58 +408,24 @@ void RefreshTask::progressCallback(const Progress & progress) set_entry->elapsed_ns.store(progress.elapsed_ns, std::memory_order_relaxed); } -void RefreshTask::cancelLocked() +void RefreshTask::reportState(RefreshState s) { - switch (state) - { - case TaskState::Running: - canceled = true; - interrupt_execution.store(true); - break; - case TaskState::Paused: - if (auto view = lockView()) - cancelRefresh(view); - cleanState(); - break; - default: - break; - } + set_entry->state.store(static_cast(s)); } -void RefreshTask::cleanState() +void RefreshTask::reportLastResult(LastTaskResult r) { - refresh_executor.reset(); - refresh_block.reset(); - refresh_query.reset(); + set_entry->last_result.store(static_cast(r)); } -std::shared_ptr RefreshTask::lockView() +void RefreshTask::reportLastRefreshTime(std::chrono::system_clock::time_point last) { - return std::static_pointer_cast(view_to_refresh.lock()); -} - -void RefreshTask::storeState(TaskState task_state) -{ - state = task_state; - set_entry->state.store(static_cast(task_state)); -} - -void RefreshTask::storeLastState(LastTaskState task_state) -{ - last_state = task_state; - set_entry->last_state.store(static_cast(task_state)); -} - -void RefreshTask::storeLastRefresh(std::chrono::system_clock::time_point last) -{ - last_refresh = last; auto secs = std::chrono::floor(last); set_entry->last_s.store(secs.time_since_epoch().count()); } -void RefreshTask::storeNextRefresh(std::chrono::system_clock::time_point next) +void RefreshTask::reportNextRefreshTime(std::chrono::system_clock::time_point next) { - next_refresh = next; auto secs = std::chrono::floor(next); set_entry->next_s.store(secs.time_since_epoch().count()); } diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 894b7f2fea0..192a4776be0 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -22,18 +22,22 @@ class ASTRefreshStrategy; class RefreshTask : public std::enable_shared_from_this { public: - enum class TaskState : RefreshTaskStateUnderlying + /// Just for observability. + enum class RefreshState : RefreshTaskStateUnderlying { Disabled = 0, Scheduled, + WaitingForDependencies, Running, Paused }; - enum class LastTaskState : RefreshTaskStateUnderlying + /// Just for observability. + enum class LastTaskResult : RefreshTaskStateUnderlying { Unknown = 0, Canceled, + Exception, Finished }; @@ -46,7 +50,7 @@ public: ContextMutablePtr context, const DB::ASTRefreshStrategy & strategy); - void initialize(std::shared_ptr view); + void initializeAndStart(std::shared_ptr view); /// Enable task scheduling void start(); @@ -60,9 +64,6 @@ public: /// Cancel task execution void cancel(); - /// Cancel task execution synchronously - void cancelSync(); - /// Pause task execution (must be either resumed or canceled later) void pause(); @@ -73,64 +74,16 @@ public: void notify(const StorageID & parent_id); private: - enum class ExecutionResult : UInt8 - { - Finished, - Paused, - Cancelled - }; - - void doRefresh(); - - void scheduleRefresh(std::chrono::system_clock::time_point now); - - void refresh(); - - ExecutionResult executeRefresh(std::unique_lock & state_lock); - - void initializeRefresh(std::shared_ptr view); - - void completeRefresh(std::shared_ptr view); - - void cancelRefresh(std::shared_ptr view); - - void pauseRefresh(std::shared_ptr view); - - std::chrono::sys_seconds calculateRefreshTime(std::chrono::system_clock::time_point now) const; - - std::chrono::seconds genSpreadSeconds(); - - void progressCallback(const Progress & progress); - - auto makePoolTask() - { - return [self = this->weak_from_this()] - { - if (auto task = self.lock()) - task->doRefresh(); - }; - } - - void cancelLocked(); - - void cleanState(); - - std::shared_ptr lockView(); - - void storeState(TaskState task_state); - - void storeLastState(LastTaskState task_state); - - void storeLastRefresh(std::chrono::system_clock::time_point last); - - void storeNextRefresh(std::chrono::system_clock::time_point next); - - /// Task ownership - BackgroundSchedulePool::TaskHolder refresh_task; + Poco::Logger * log = nullptr; std::weak_ptr view_to_refresh; RefreshSet::Entry set_entry; - /// Task execution + /// Refresh schedule + std::variant refresh_timer; + std::uniform_int_distribution refresh_spread; + + /// Task execution. Non-empty iff a refresh is in progress (possibly paused). + /// Whoever unsets these should also call storeLastState(). std::optional refresh_executor; std::optional refresh_block; std::shared_ptr refresh_query; @@ -140,24 +93,60 @@ private: RefreshDependencies dependencies; std::vector deps_entries; - /// Refresh time settings and data - std::chrono::system_clock::time_point last_refresh; - std::chrono::system_clock::time_point next_refresh; - std::variant refresh_timer; + /// Protects all fields below (they're accessed by both refreshTask() and public methods). + /// Never locked for blocking operations (e.g. creating or dropping the internal table). + std::mutex mutex; - /// Refresh time randomization - std::uniform_int_distribution refresh_spread; + /// Refreshes are stopped (e.g. by SYSTEM STOP VIEW). + bool stop_requested = false; + /// Refreshes are paused (e.g. by SYSTEM PAUSE VIEW). + bool pause_requested = false; + /// Cancel current refresh, then reset this flag. + bool cancel_requested = false; - /// Task state - std::mutex state_mutex; - std::condition_variable sync_canceled; - TaskState state{TaskState::Disabled}; - LastTaskState last_state{LastTaskState::Unknown}; - bool canceled; + /// If true, we should start a refresh right away. All refreshes go through this flag. + bool refresh_immediately = false; - /// Outer triggers - std::atomic_bool refresh_immediately; - std::atomic_bool interrupt_execution; + /// If true, the refresh task should interrupt its query execution and reconsider what to do, + /// re-checking `stop_requested`, `cancel_requested`, etc. + std::atomic_bool interrupt_execution {false}; + + /// When to refresh next. Updated when a refresh is finished or canceled. + std::chrono::system_clock::time_point next_refresh_without_spread; + std::chrono::system_clock::time_point next_refresh_with_spread; + + /// Calls refreshTask() from background thread. + BackgroundSchedulePool::TaskHolder refresh_task; + + /// 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, + /// when needed, by public methods or by timer. + /// + /// Public methods just provide inputs for the refreshTask()'s decisions + /// (e.g. stop_requested, cancel_requested), they don't do anything significant themselves. + /// This adds some inefficiency: even trivial or no-op requests have to schedule a background + /// task instead of directly performing the operation; but the simplicity seems worth it, I had + /// a really hard time trying to organize this code in any other way. + void refreshTask(); + + /// Methods that do the actual work: creating/dropping internal table, executing the query. + void initializeRefresh(std::shared_ptr view); + bool executeRefresh(); + void completeRefresh(std::shared_ptr view, LastTaskResult result); + void cancelRefresh(LastTaskResult result); + void cleanState(); + + /// Assigns next_refresh_* + void calculateNextRefreshTime(std::chrono::system_clock::time_point now); + + std::shared_ptr lockView(); + + /// Methods that push information to RefreshSet, for observability. + void progressCallback(const Progress & progress); + void reportState(RefreshState s); + void reportLastResult(LastTaskResult r); + void reportLastRefreshTime(std::chrono::system_clock::time_point last); + void reportNextRefreshTime(std::chrono::system_clock::time_point next); }; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9cb21bdb06f..4f2ffb38017 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -444,8 +444,7 @@ void StorageMaterializedView::startup() if (refresher) { - refresher->initialize(std::static_pointer_cast(shared_from_this())); - refresher->start(); + refresher->initializeAndStart(std::static_pointer_cast(shared_from_this())); } } diff --git a/src/Storages/System/StorageSystemViewRefreshes.cpp b/src/Storages/System/StorageSystemViewRefreshes.cpp index f30e2863c4a..1e763f30991 100644 --- a/src/Storages/System/StorageSystemViewRefreshes.cpp +++ b/src/Storages/System/StorageSystemViewRefreshes.cpp @@ -17,7 +17,7 @@ NamesAndTypesList StorageSystemViewRefreshes::getNamesAndTypes() {"database", std::make_shared()}, {"view", std::make_shared()}, {"refresh_status", std::make_shared()}, - {"last_refresh_status", std::make_shared()}, + {"last_refresh_result", std::make_shared()}, {"last_refresh_time", std::make_shared()}, {"next_refresh_time", std::make_shared()}, {"progress", std::make_shared()}, @@ -51,7 +51,7 @@ void StorageSystemViewRefreshes::fillData( res_columns[i++]->insert(refresh.database); res_columns[i++]->insert(refresh.view_name); res_columns[i++]->insert(refresh.refresh_status); - res_columns[i++]->insert(refresh.last_refresh_status); + res_columns[i++]->insert(refresh.last_refresh_result); res_columns[i++]->insert(refresh.last_refresh_time); res_columns[i++]->insert(refresh.next_refresh_time); res_columns[i++]->insert(refresh.progress); diff --git a/tests/queries/0_stateless/02661_refreshable_materialized_views.sql b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql index 4e5420509c1..531000b3fe6 100644 --- a/tests/queries/0_stateless/02661_refreshable_materialized_views.sql +++ b/tests/queries/0_stateless/02661_refreshable_materialized_views.sql @@ -4,7 +4,7 @@ SELECT view FROM system.view_refreshes WHERE view = 'test'; CREATE MATERIALIZED VIEW test1 REFRESH EVERY 1 HOUR ENGINE = MergeTree() ORDER BY number AS SELECT * FROM test; -CREATE MATERIALIZED VIEW test2 REFRESH EVERY 2 HOUR OFFSET 42 MINUTE 8 SECOND RANDOMIZE FOR 10 MINUTE ENGINE = MergeTree() ORDER BY number AS SELECT * FROM test; +CREATE MATERIALIZED VIEW test2 REFRESH EVERY 2 HOUR OFFSET 42 MINUTE 8 SECOND RANDOMIZE FOR 10 MINUTE ENGINE = MergeTree() ORDER BY number AS SELECT * FROM test; SELECT view FROM system.view_refreshes WHERE view LIKE 'test%' ORDER BY view; From 01369a0a8a1742679de85a7e507980b2ab610f3b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 24 Nov 2023 01:32:45 +0000 Subject: [PATCH 10/25] Overhaul dependencies --- src/Common/CurrentMetrics.cpp | 3 +- src/Databases/TablesDependencyGraph.h | 2 +- src/Interpreters/AddDefaultDatabaseVisitor.h | 13 ++ src/Interpreters/InterpreterCreateQuery.cpp | 9 + src/Parsers/ParserRefreshStrategy.cpp | 6 +- .../MaterializedView/RefreshAllCombiner.cpp | 58 ------ .../MaterializedView/RefreshAllCombiner.h | 34 --- .../MaterializedView/RefreshDependencies.cpp | 60 ------ .../MaterializedView/RefreshDependencies.h | 56 ----- src/Storages/MaterializedView/RefreshSet.cpp | 100 +++++---- src/Storages/MaterializedView/RefreshSet.h | 66 +++--- src/Storages/MaterializedView/RefreshTask.cpp | 193 +++++++++++------- src/Storages/MaterializedView/RefreshTask.h | 23 ++- .../MaterializedView/RefreshTimers.cpp | 61 +++++- src/Storages/MaterializedView/RefreshTimers.h | 19 ++ src/Storages/StorageMaterializedView.cpp | 2 +- 16 files changed, 334 insertions(+), 371 deletions(-) delete mode 100644 src/Storages/MaterializedView/RefreshAllCombiner.cpp delete mode 100644 src/Storages/MaterializedView/RefreshAllCombiner.h delete mode 100644 src/Storages/MaterializedView/RefreshDependencies.cpp delete mode 100644 src/Storages/MaterializedView/RefreshDependencies.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 87de19b8907..2613e9ec116 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -253,7 +253,8 @@ M(MergeTreeAllRangesAnnouncementsSent, "The current number of announcement being sent in flight from the remote server to the initiator server about the set of data parts (for MergeTree tables). Measured on the remote server side.") \ M(CreatedTimersInQueryProfiler, "Number of Created thread local timers in QueryProfiler") \ M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \ - M(RefreshingViews, "Number of active refreshes") \ + M(RefreshableViews, "Number materialized views with periodic refreshing (REFRESH)") \ + M(RefreshingViews, "Number of materialized views currently executing a refresh") \ #ifdef APPLY_FOR_EXTERNAL_METRICS #define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M) diff --git a/src/Databases/TablesDependencyGraph.h b/src/Databases/TablesDependencyGraph.h index e71d5ecc5fc..50be3bbf969 100644 --- a/src/Databases/TablesDependencyGraph.h +++ b/src/Databases/TablesDependencyGraph.h @@ -60,7 +60,7 @@ public: /// Removes all dependencies of "table_id", returns those dependencies. std::vector removeDependencies(const StorageID & table_id, bool remove_isolated_tables = false); - /// Removes a table from the graph and removes all references to in from the graph (both from its dependencies and dependents). + /// Removes a table from the graph and removes all references to it from the graph (both from its dependencies and dependents). bool removeTable(const StorageID & table_id); /// Removes tables from the graph by a specified filter. diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 27639c4b813..e6354467938 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -87,6 +88,12 @@ public: visit(child); } + void visit(ASTRefreshStrategy & refresh) const + { + ASTPtr unused; + visit(refresh, unused); + } + private: ContextPtr context; @@ -229,6 +236,12 @@ private: } } + void visit(ASTRefreshStrategy & refresh, ASTPtr &) const + { + for (auto & table : refresh.children) + tryVisit(table); + } + void visitChildren(IAST & ast) const { for (auto & child : ast.children) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1eadb325e95..801a46f4167 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1210,6 +1210,15 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) visitor.visit(*create.select); } + if (create.refresh_strategy) + { + /// TODO: This doesn't work for some reason. + AddDefaultDatabaseVisitor visitor(getContext(), current_database); + visitor.visit(*create.refresh_strategy); + + /// TODO: For DEPENDS ON, check that the specified tables exist. + } + if (create.columns_list) { AddDefaultDatabaseVisitor visitor(getContext(), current_database); diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index a6fbb373ed3..05dd081e61d 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -54,10 +54,12 @@ bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (ParserKeyword{"DEPENDS ON"}.ignore(pos, expected)) { ASTPtr dependencies; + auto list_parser = ParserList{ - std::make_unique(), + std::make_unique( + /*table_name_with_optional_uuid_*/ true, /*allow_query_parameter_*/ false), std::make_unique(TokenType::Comma), - /* allow_empty= */ false}; + /*allow_empty*/ false}; if (!list_parser.parse(pos, dependencies, expected)) return false; refresh->set(refresh->dependencies, dependencies); diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.cpp b/src/Storages/MaterializedView/RefreshAllCombiner.cpp deleted file mode 100644 index 5cb06ade9c7..00000000000 --- a/src/Storages/MaterializedView/RefreshAllCombiner.cpp +++ /dev/null @@ -1,58 +0,0 @@ -#include - -#include - -namespace DB -{ - -RefreshAllCombiner::RefreshAllCombiner() - : time_arrived{false} -{} - -RefreshAllCombiner::RefreshAllCombiner(const std::vector & parents) - : time_arrived{false} -{ - parents_arrived.reserve(parents.size()); - for (auto && parent : parents) - parents_arrived.emplace(parent.uuid, false); -} - -bool RefreshAllCombiner::arriveTime() -{ - std::lock_guard lock(combiner_mutex); - time_arrived = true; - return allArrivedLocked(); -} - -bool RefreshAllCombiner::arriveParent(const StorageID & id) -{ - std::lock_guard lock(combiner_mutex); - parents_arrived[id.uuid] = true; - return allArrivedLocked(); -} - -void RefreshAllCombiner::flush() -{ - std::lock_guard lock(combiner_mutex); - flushLocked(); -} - -bool RefreshAllCombiner::allArrivedLocked() -{ - auto is_value = [](auto && key_value) { return key_value.second; }; - if (time_arrived && std::ranges::all_of(parents_arrived, is_value)) - { - flushLocked(); - return true; - } - return false; -} - -void RefreshAllCombiner::flushLocked() -{ - for (auto & [parent, arrived] : parents_arrived) - arrived = false; - time_arrived = false; -} - -} diff --git a/src/Storages/MaterializedView/RefreshAllCombiner.h b/src/Storages/MaterializedView/RefreshAllCombiner.h deleted file mode 100644 index f9f3a8d319c..00000000000 --- a/src/Storages/MaterializedView/RefreshAllCombiner.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/// Concurrent primitive for dependency completeness registration -/// When arrive methods return true, dependent task must be executed (or scheduled) -/// TODO: Doesn't need to be thread safe. -class RefreshAllCombiner -{ -public: - RefreshAllCombiner(); - - explicit RefreshAllCombiner(const std::vector & parents); - - bool arriveTime(); - - bool arriveParent(const StorageID & id); - - void flush(); - -private: - bool allArrivedLocked(); - - void flushLocked(); - - std::mutex combiner_mutex; - std::unordered_map parents_arrived; - bool time_arrived; -}; - -} diff --git a/src/Storages/MaterializedView/RefreshDependencies.cpp b/src/Storages/MaterializedView/RefreshDependencies.cpp deleted file mode 100644 index f1a834a6b3a..00000000000 --- a/src/Storages/MaterializedView/RefreshDependencies.cpp +++ /dev/null @@ -1,60 +0,0 @@ -#include - -#include - -namespace DB -{ - -RefreshDependencies::Entry::Entry(RefreshDependencies & deps, ContainerIter it) - : dependencies{&deps} - , entry_it{it} -{} - -RefreshDependencies::Entry::Entry(Entry && other) noexcept - : dependencies(std::exchange(other.dependencies, nullptr)) - , entry_it(std::move(other.entry_it)) -{} - -RefreshDependencies::Entry & RefreshDependencies::Entry::operator=(Entry && other) noexcept -{ - if (this == &other) - return *this; - cleanup(std::exchange(dependencies, std::exchange(other.dependencies, nullptr))); - entry_it = std::move(other.entry_it); - return *this; -} - -RefreshDependencies::Entry::~Entry() -{ - cleanup(dependencies); -} - -void RefreshDependencies::Entry::cleanup(RefreshDependencies * deps) -{ - if (deps) - deps->erase(entry_it); -} - -RefreshDependenciesEntry RefreshDependencies::add(RefreshTaskHolder dependency) -{ - std::lock_guard lock(dependencies_mutex); - return Entry(*this, dependencies.emplace(dependencies.end(), dependency)); -} - -void RefreshDependencies::notifyAll(const StorageID & id) -{ - std::lock_guard lock(dependencies_mutex); - for (auto && dep : dependencies) - { - if (auto task = dep.lock()) - task->notify(id); - } -} - -void RefreshDependencies::erase(ContainerIter it) -{ - std::lock_guard lock(dependencies_mutex); - dependencies.erase(it); -} - -} diff --git a/src/Storages/MaterializedView/RefreshDependencies.h b/src/Storages/MaterializedView/RefreshDependencies.h deleted file mode 100644 index 8d370f96d40..00000000000 --- a/src/Storages/MaterializedView/RefreshDependencies.h +++ /dev/null @@ -1,56 +0,0 @@ -#pragma once - -#include - -#include - -#include - - -namespace DB -{ - -class RefreshTask; - -/// Concurrent primitive for managing list of dependent task and notifying them -class RefreshDependencies -{ - using Container = std::list; - using ContainerIter = typename Container::iterator; - -public: - class Entry - { - friend class RefreshDependencies; - - public: - Entry(Entry &&) noexcept; - Entry & operator=(Entry &&) noexcept; - - ~Entry(); - - private: - Entry(RefreshDependencies & deps, ContainerIter it); - - void cleanup(RefreshDependencies * deps); - - RefreshDependencies * dependencies; - ContainerIter entry_it; - }; - - RefreshDependencies() = default; - - Entry add(RefreshTaskHolder dependency); - - void notifyAll(const StorageID & id); - -private: - void erase(ContainerIter it); - - std::mutex dependencies_mutex; - std::list dependencies; -}; - -using RefreshDependenciesEntry = RefreshDependencies::Entry; - -} diff --git a/src/Storages/MaterializedView/RefreshSet.cpp b/src/Storages/MaterializedView/RefreshSet.cpp index 9efd82e1afc..bef628bc42b 100644 --- a/src/Storages/MaterializedView/RefreshSet.cpp +++ b/src/Storages/MaterializedView/RefreshSet.cpp @@ -3,15 +3,16 @@ namespace CurrentMetrics { - extern const Metric RefreshingViews; + extern const Metric RefreshableViews; } namespace DB { -RefreshSetElement::RefreshSetElement(StorageID id, RefreshTaskHolder task) +RefreshSetElement::RefreshSetElement(StorageID id, std::vector deps, RefreshTaskHolder task) : corresponding_task(task) , view_id(std::move(id)) + , dependencies(std::move(deps)) {} RefreshInfo RefreshSetElement::getInfo() const @@ -36,40 +37,21 @@ RefreshInfo RefreshSetElement::getInfo() const }; } -const StorageID & RefreshSetElement::getID() const -{ - return view_id; -} - RefreshTaskHolder RefreshSetElement::getTask() const { return corresponding_task.lock(); } -bool RefreshSetLess::operator()(const RefreshSetElement & l, const RefreshSetElement & r) const +const StorageID & RefreshSetElement::getID() const { - return l.getID().uuid < r.getID().uuid; + return view_id; } -bool RefreshSetLess::operator()(const StorageID & l, const RefreshSetElement & r) const +const std::vector & RefreshSetElement::getDependencies() const { - return l.uuid < r.getID().uuid; + return dependencies; } -bool RefreshSetLess::operator()(const RefreshSetElement & l, const StorageID & r) const -{ - return l.getID().uuid < r.uuid; -} - -bool RefreshSetLess::operator()(const StorageID & l, const StorageID & r) const -{ - return l.uuid < r.uuid; -} - -RefreshSet::Entry::Entry() - : parent_set{nullptr} -{} - RefreshSet::Entry::Entry(Entry && other) noexcept : parent_set{std::exchange(other.parent_set, nullptr)} , iter(std::move(other.iter)) @@ -80,7 +62,8 @@ RefreshSet::Entry & RefreshSet::Entry::operator=(Entry && other) noexcept { if (this == &other) return *this; - cleanup(std::exchange(parent_set, std::exchange(other.parent_set, nullptr))); + reset(); + parent_set = std::exchange(other.parent_set, nullptr); iter = std::move(other.iter); metric_increment = std::move(other.metric_increment); return *this; @@ -88,34 +71,51 @@ RefreshSet::Entry & RefreshSet::Entry::operator=(Entry && other) noexcept RefreshSet::Entry::~Entry() { - cleanup(parent_set); + reset(); } -RefreshSet::Entry::Entry(RefreshSet & set, ContainerIter it, const CurrentMetrics::Metric & metric) - : parent_set{&set}, iter(std::move(it)), metric_increment(metric) +RefreshSet::Entry::Entry(RefreshSet & set, ElementMapIter it) + : parent_set{&set}, iter(std::move(it)), metric_increment(CurrentMetrics::RefreshableViews) {} -void RefreshSet::Entry::cleanup(RefreshSet * set) +void RefreshSet::Entry::reset() { - if (set) - set->erase(iter); + if (!parent_set) + return; + std::exchange(parent_set, nullptr)->erase(iter); + metric_increment.reset(); } -RefreshSet::RefreshSet() - : set_metric(CurrentMetrics::RefreshingViews) -{} +RefreshSet::RefreshSet() {} + +RefreshSet::Entry RefreshSet::emplace(StorageID id, std::vector dependencies, RefreshTaskHolder task) +{ + std::lock_guard guard(mutex); + auto [it, is_inserted] = elements.emplace(std::piecewise_construct, std::forward_as_tuple(id), std::forward_as_tuple(id, dependencies, std::move(task))); + if (!is_inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Refresh set entry already exists for table {}", id.getFullTableName()); + + for (const StorageID & dep : dependencies) + { + auto [unused, dep_inserted] = dependents[dep].insert(id); + if (!dep_inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Refresh set entry already contains dependency of {} on {}", id.getFullTableName(), dep.getFullTableName()); + } + + return Entry(*this, std::move(it)); +} RefreshTaskHolder RefreshSet::getTask(const StorageID & id) const { - std::lock_guard lock(elements_mutex); - if (auto element = elements.find(id.uuid); element != elements.end()) + std::lock_guard lock(mutex); + if (auto element = elements.find(id); element != elements.end()) return element->second.getTask(); return nullptr; } RefreshSet::InfoContainer RefreshSet::getInfo() const { - std::lock_guard lock(elements_mutex); + std::lock_guard lock(mutex); InfoContainer res; res.reserve(elements.size()); for (auto && element : elements) @@ -123,9 +123,29 @@ RefreshSet::InfoContainer RefreshSet::getInfo() const return res; } -void RefreshSet::erase(ContainerIter it) +std::vector RefreshSet::getDependents(const StorageID & id) const { - std::lock_guard lock(elements_mutex); + std::lock_guard lock(mutex); + std::vector res; + auto it = dependents.find(id); + if (it == dependents.end()) + return {}; + for (auto & dep_id : it->second) + if (auto element = elements.find(dep_id); element != elements.end()) + res.push_back(element->second.getTask()); + return res; +} + +void RefreshSet::erase(ElementMapIter it) +{ + std::lock_guard lock(mutex); + for (const StorageID & dep : it->second.getDependencies()) + { + auto & set = dependents[dep]; + set.erase(it->second.getID()); + if (set.empty()) + dependents.erase(dep); + } elements.erase(it); } diff --git a/src/Storages/MaterializedView/RefreshSet.h b/src/Storages/MaterializedView/RefreshSet.h index cc5b0006218..b2685d67883 100644 --- a/src/Storages/MaterializedView/RefreshSet.h +++ b/src/Storages/MaterializedView/RefreshSet.h @@ -9,6 +9,8 @@ namespace DB { +using DatabaseAndTableNameSet = std::unordered_set; + struct RefreshInfo { String database; @@ -33,7 +35,7 @@ class RefreshSetElement { friend class RefreshTask; public: - RefreshSetElement(StorageID id, RefreshTaskHolder task); + RefreshSetElement(StorageID id, std::vector deps, RefreshTaskHolder task); RefreshSetElement(const RefreshSetElement &) = delete; RefreshSetElement & operator=(const RefreshSetElement &) = delete; @@ -41,12 +43,13 @@ public: RefreshInfo getInfo() const; RefreshTaskHolder getTask() const; - const StorageID & getID() const; + const std::vector & getDependencies() const; private: RefreshTaskObserver corresponding_task; StorageID view_id; + std::vector dependencies; std::atomic read_rows{0}; std::atomic read_bytes{0}; @@ -63,73 +66,62 @@ private: std::atomic last_result{0}; }; -struct RefreshSetLess -{ - using is_transparent = std::true_type; - - bool operator()(const RefreshSetElement & l, const RefreshSetElement & r) const; - bool operator()(const StorageID & l, const RefreshSetElement & r) const; - bool operator()(const RefreshSetElement & l, const StorageID & r) const; - bool operator()(const StorageID & l, const StorageID & r) const; -}; - /// Set of refreshable views class RefreshSet { private: - using Container = std::map; - using ContainerIter = typename Container::iterator; + using ElementMap = std::unordered_map; + using ElementMapIter = typename ElementMap::iterator; public: class Entry { friend class RefreshSet; public: - Entry(); + Entry() = default; Entry(Entry &&) noexcept; Entry & operator=(Entry &&) noexcept; ~Entry(); - RefreshSetElement * operator->() { return &iter->second; } + explicit operator bool() const { return parent_set != nullptr; } + RefreshSetElement * operator->() { chassert(parent_set); return &iter->second; } + + void reset(); private: - RefreshSet * parent_set; - ContainerIter iter; + RefreshSet * parent_set = nullptr; + ElementMapIter iter; std::optional metric_increment; - Entry( - RefreshSet & set, - ContainerIter it, - const CurrentMetrics::Metric & metric); - - void cleanup(RefreshSet * set); + Entry(RefreshSet & set, ElementMapIter it); }; using InfoContainer = std::vector; RefreshSet(); - std::optional emplace(StorageID id, RefreshTaskHolder task) - { - std::lock_guard guard(elements_mutex); - auto [it, is_inserted] = elements.emplace(std::piecewise_construct, std::forward_as_tuple(id.uuid), std::forward_as_tuple(id, std::move(task))); - if (is_inserted) - return Entry(*this, std::move(it), set_metric); - return {}; - } + Entry emplace(StorageID id, std::vector dependencies, RefreshTaskHolder task); RefreshTaskHolder getTask(const StorageID & id) const; InfoContainer getInfo() const; -private: - mutable std::mutex elements_mutex; - Container elements; - CurrentMetrics::Metric set_metric; + /// Get tasks that depend on the given one. + std::vector getDependents(const StorageID & id) const; - void erase(ContainerIter it); +private: + using DependentsMap = std::unordered_map; + + /// Protects the two maps below, not locked for any nontrivial operations (e.g. operations that + /// block or lock other mutexes). + mutable std::mutex mutex; + + ElementMap elements; + DependentsMap dependents; + + void erase(ElementMapIter it); }; using RefreshSetEntry = RefreshSet::Entry; diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 0a85f533a27..000ee7aa1bd 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -2,20 +2,21 @@ #include +#include #include #include #include #include #include +namespace CurrentMetrics +{ + extern const Metric RefreshingViews; +} + namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - namespace { @@ -27,26 +28,12 @@ std::uniform_int_distribution makeSpreadDistribution(const ASTTimePeriod return std::uniform_int_distribution(-limit, limit); } -std::variant makeRefreshTimer(const ASTRefreshStrategy & strategy) -{ - using enum ASTRefreshStrategy::ScheduleKind; - switch (strategy.schedule_kind) - { - case EVERY: - return RefreshEveryTimer{*strategy.period, strategy.interval}; - case AFTER: - return RefreshAfterTimer{strategy.interval}; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown refresh strategy kind"); - } -} - } RefreshTask::RefreshTask( const ASTRefreshStrategy & strategy) : log(&Poco::Logger::get("RefreshTask")) - , refresh_timer(makeRefreshTimer(strategy)) + , refresh_timer(strategy) , refresh_spread{makeSpreadDistribution(strategy.spread)} {} @@ -63,28 +50,13 @@ RefreshTaskHolder RefreshTask::create( if (auto t = self.lock()) t->refreshTask(); }); - task->set_entry = context->getRefreshSet().emplace(view.getStorageID(), task).value(); + + std::vector deps; if (strategy.dependencies) - { - if (strategy.schedule_kind != ASTRefreshStrategy::ScheduleKind::AFTER) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dependencies are allowed only for AFTER refresh kind"); - - task->deps_entries.reserve(strategy.dependencies->children.size()); for (auto && dependency : strategy.dependencies->children) - { - StorageID dep_id(dependency->as()); - /// TODO: - /// * This depends on the order in which different tables are initialized. - /// Is the order guaranteed on startup? - /// * At what point does the table name from the query get mapped to the table's UUID? - /// Does it work at all? Is it reliable? - /// * Don't silently ignore if the table is missing. - if (auto dep_task = context->getRefreshSet().getTask(dep_id)) - task->deps_entries.push_back(dep_task->dependencies.add(task)); - } + deps.emplace_back(dependency->as()); - /// TODO: Initialize combiner. - } + task->set_entry = context->getRefreshSet().emplace(view.getStorageID(), deps, task); return task; } @@ -93,6 +65,7 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi { view_to_refresh = view; /// TODO: Add a setting to stop views on startup, set `stop_requested = true` in that case. + populateDependencies(); calculateNextRefreshTime(std::chrono::system_clock::now()); refresh_task->schedule(); } @@ -148,14 +121,78 @@ void RefreshTask::resume() refresh_task->schedule(); } -void RefreshTask::notify(const StorageID & parent_id) +void RefreshTask::shutdown() +{ + { + std::lock_guard guard(mutex); + stop_requested = true; + interrupt_execution.store(true); + } + + /// Wait for the task to return and prevent it from being scheduled in future. + refresh_task->deactivate(); + + /// Remove from RefreshSet on DROP, without waiting for the IStorage to be destroyed. + /// This matters because a table may get dropped and immediately created again with the same name, + /// while the old table's IStorage still exists (pinned by ongoing queries). + std::lock_guard guard(mutex); + set_entry.reset(); +} + +void RefreshTask::notify(const StorageID & parent_id, std::chrono::system_clock::time_point scheduled_time_without_spread, const RefreshTimer & parent_timer) { std::lock_guard guard(mutex); - if (!combiner.arriveParent(parent_id)) + if (!set_entry) + return; // we've shut down + + /// In the general case, it's not clear what the meaning of dependencies should be. + /// E.g. what behavior would the user want/expect in the following cases?: + /// * REFRESH EVERY 3 HOUR depends on REFRESH EVERY 2 HOUR + /// * REFRESH AFTER 3 HOUR depends on REFRESH AFTER 2 HOUR + /// * REFRESH AFTER 3 HOUR depends on REFRESH EVERY 1 DAY + /// I don't know. + /// + /// Cases that are important to support well include: + /// (1) REFRESH EVERY 1 DAY depends on REFRESH EVERY 1 DAY + /// Here the second refresh should start only after the first refresh completed *for the same day*. + /// Yesterday's refresh of the dependency shouldn't trigger today's refresh of the dependent, + /// even if it completed today. + /// (2) REFRESH EVERY 1 DAY OFFSET 2 HOUR depends on REFRESH EVERY 1 DAY OFFSET 1 HOUR + /// (3) REFRESH EVERY 1 DAY OFFSET 1 HOUR depends on REFRESH EVERY 1 DAY OFFSET 23 HOUR + /// Here the dependency's refresh on day X should trigger dependent's refresh on day X+1. + /// (4) REFRESH EVERY 2 HOUR depends on REFRESH EVERY 1 HOUR + /// The 2 HOUR refresh should happen after the 1 HOUR refresh for every other hour, e.g. + /// after the 2pm refresh, then after the 4pm refresh, etc. + /// (5) REFRESH AFTER 1 HOUR depends on REFRESH AFTER 1 HOUR + /// Here the two views should try to synchronize their schedules instead of arbitrarily drifting + /// apart. In particular, consider the case where the dependency refreshes slightly faster than + /// the dependent. If we don't do anything special, the DEPENDS ON will have pretty much no effect. + /// To apply some synchronization pressure, we reduce the dependent's delay by some percentage + /// after the dependent completed. + /// (6) REFRESH AFTER 1 HOUR depends on REFRESH AFTER 2 HOUR + /// REFRESH EVERY 1 HOUR depends on REFRESH EVERY 2 HOUR + /// Not sure about these. Currently we just make the dependent refresh at the same rate as + /// the dependency, i.e. the 1 HOUR table will actually be refreshed every 2 hours. + + /// Only accept the dependency's refresh if its next refresh time is after ours. + /// This takes care of cases (1)-(4), and seems harmless in all other cases. + /// Might be mildly helpful in weird cases like REFRESH AFTER 3 HOUR depends on REFRESH AFTER 2 HOUR. + if (parent_timer.next(scheduled_time_without_spread) <= next_refresh_without_spread) return; - if (std::exchange(refresh_immediately, true)) - return; - refresh_task->schedule(); + + if (arriveDependency(parent_id) && !std::exchange(refresh_immediately, true)) + refresh_task->schedule(); + + /// Decrease delay in case (5). + /// Maybe we should do it for all AFTER-AFTER dependencies, even if periods are different. + if (refresh_timer == parent_timer && refresh_timer.tryGetAfter()) + { + /// TODO: Implement this: + /// * Add setting max_after_delay_adjustment_pct + /// * Decrease both next_refresh_without_spread and next_refresh_with_spread, + /// but only if they haven't already been decreased this way during current period + /// * refresh_task->schedule() + } } void RefreshTask::refreshTask() @@ -205,7 +242,7 @@ void RefreshTask::refreshTask() auto now = std::chrono::system_clock::now(); if (now >= next_refresh_with_spread) { - if (combiner.arriveTime()) + if (arriveTime()) refresh_immediately = true; else { @@ -239,6 +276,9 @@ void RefreshTask::refreshTask() reportState(RefreshState::Running); + CurrentMetrics::Increment metric_inc(CurrentMetrics::RefreshingViews); + auto scheduled_time_without_spread = next_refresh_without_spread; + lock.unlock(); bool finished = false; @@ -251,7 +291,7 @@ void RefreshTask::refreshTask() finished = executeRefresh(); if (finished) - completeRefresh(view, LastTaskResult::Finished); + completeRefresh(view, LastTaskResult::Finished, scheduled_time_without_spread); } catch (...) { @@ -311,12 +351,17 @@ bool RefreshTask::executeRefresh() return !not_finished; } -void RefreshTask::completeRefresh(std::shared_ptr view, LastTaskResult result) +void RefreshTask::completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::system_clock::time_point scheduled_time_without_spread) { auto stale_table = view->exchangeTargetTable(refresh_query->table_id); - dependencies.notifyAll(view->getStorageID()); - auto drop_context = Context::createCopy(view->getContext()); + auto context = view->getContext(); + StorageID my_id = set_entry->getID(); + auto dependents = context->getRefreshSet().getDependents(my_id); + for (const RefreshTaskHolder & dep_task : dependents) + dep_task->notify(my_id, scheduled_time_without_spread, refresh_timer); + + auto drop_context = Context::createCopy(context); InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, stale_table, /*sync=*/true); cleanState(); @@ -351,16 +396,6 @@ void RefreshTask::cleanState() refresh_query.reset(); } -namespace -{ - -template -struct CombinedVisitor : Ts... { using Ts::operator()...; }; -template -CombinedVisitor(Ts...) -> CombinedVisitor; - -} - void RefreshTask::calculateNextRefreshTime(std::chrono::system_clock::time_point now) { /// TODO: Add a setting to randomize initial delay in case of AFTER, for the case when the server @@ -368,21 +403,11 @@ void RefreshTask::calculateNextRefreshTime(std::chrono::system_clock::time_point /// TODO: Maybe do something like skip_update_after_seconds and skip_update_after_ratio. /// Unclear if that's useful at all if the last refresh timestamp is not remembered across restarts. - auto advance = [&](std::chrono::system_clock::time_point t) - { - CombinedVisitor refresh_time_visitor{ - [t](const RefreshAfterTimer & timer) { return timer.after(t); }, - [t](const RefreshEveryTimer & timer) { return timer.next(t); }}; - auto r = std::visit(std::move(refresh_time_visitor), refresh_timer); - chassert(r > t); - return r; - }; - /// It's important to use time without spread here, otherwise we would do multiple refreshes instead /// of one, if the generated spread is negative and the first refresh completes faster than the spread. - std::chrono::sys_seconds next = advance(next_refresh_without_spread); + std::chrono::sys_seconds next = refresh_timer.next(next_refresh_without_spread); if (next < now) - next = advance(now); // fell behind, skip to current time + next = refresh_timer.next(now); // fell behind, skip to current time next_refresh_without_spread = next; next_refresh_with_spread = next + std::chrono::seconds{refresh_spread(thread_local_rng)}; @@ -390,6 +415,32 @@ void RefreshTask::calculateNextRefreshTime(std::chrono::system_clock::time_point reportNextRefreshTime(next_refresh_with_spread); } +bool RefreshTask::arriveDependency(const StorageID & parent_table_or_timer) +{ + remaining_dependencies.erase(parent_table_or_timer); + if (!remaining_dependencies.empty() || !time_arrived) + return false; + populateDependencies(); + return true; +} + +bool RefreshTask::arriveTime() +{ + time_arrived = true; + if (!remaining_dependencies.empty() || !time_arrived) + return false; + populateDependencies(); + return true; +} + +void RefreshTask::populateDependencies() +{ + chassert(remaining_dependencies.empty()); + auto deps = set_entry->getDependencies(); + remaining_dependencies.insert(deps.begin(), deps.end()); + time_arrived = false; +} + std::shared_ptr RefreshTask::lockView() { return std::static_pointer_cast(view_to_refresh.lock()); diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index 192a4776be0..cdb0d22342e 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include #include @@ -70,8 +68,11 @@ public: /// Resume task execution void resume(); + /// Permanently disable task scheduling and remove this table from RefreshSet. + void shutdown(); + /// Notify dependent task - void notify(const StorageID & parent_id); + void notify(const StorageID & parent_id, std::chrono::system_clock::time_point scheduled_time_without_spread, const RefreshTimer & parent_timer); private: Poco::Logger * log = nullptr; @@ -79,7 +80,7 @@ private: RefreshSet::Entry set_entry; /// Refresh schedule - std::variant refresh_timer; + RefreshTimer refresh_timer; std::uniform_int_distribution refresh_spread; /// Task execution. Non-empty iff a refresh is in progress (possibly paused). @@ -88,10 +89,9 @@ private: std::optional refresh_block; std::shared_ptr refresh_query; - /// Concurrent dependency management - RefreshAllCombiner combiner; - RefreshDependencies dependencies; - std::vector deps_entries; + /// StorageIDs of our dependencies that we're waiting for. + DatabaseAndTableNameSet remaining_dependencies; + bool time_arrived = false; /// Protects all fields below (they're accessed by both refreshTask() and public methods). /// Never locked for blocking operations (e.g. creating or dropping the internal table). @@ -132,13 +132,18 @@ private: /// Methods that do the actual work: creating/dropping internal table, executing the query. void initializeRefresh(std::shared_ptr view); bool executeRefresh(); - void completeRefresh(std::shared_ptr view, LastTaskResult result); + void completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::system_clock::time_point scheduled_time_without_spread); void cancelRefresh(LastTaskResult result); void cleanState(); /// Assigns next_refresh_* void calculateNextRefreshTime(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_table_or_timer); + bool arriveTime(); + void populateDependencies(); + std::shared_ptr lockView(); /// Methods that push information to RefreshSet, for observability. diff --git a/src/Storages/MaterializedView/RefreshTimers.cpp b/src/Storages/MaterializedView/RefreshTimers.cpp index ebef561fc29..0331bad82c3 100644 --- a/src/Storages/MaterializedView/RefreshTimers.cpp +++ b/src/Storages/MaterializedView/RefreshTimers.cpp @@ -1,10 +1,15 @@ #include -#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + namespace { constexpr std::chrono::days ZERO_DAYS{0}; @@ -68,6 +73,14 @@ void RefreshAfterTimer::setWithKind(IntervalKind kind, UInt64 val) } } +bool RefreshAfterTimer::operator==(const RefreshAfterTimer & rhs) const +{ + /// (Or maybe different implementations of standard library have different sizes of chrono types. + /// If so, feel free to just remove this assert.) + static_assert(sizeof(*this) == 40, "RefreshAfterTimer fields appear to have changed. Please update this operator==() here."); + return std::tie(seconds, minutes, hours, days, weeks, months, years) == std::tie(rhs.seconds, rhs.minutes, rhs.hours, rhs.days, rhs.weeks, rhs.months, rhs.years); +} + RefreshEveryTimer::RefreshEveryTimer(const ASTTimePeriod & time_period, const ASTTimeInterval * time_offset) : offset(time_offset) , value{static_cast(time_period.value)} @@ -240,4 +253,50 @@ std::chrono::sys_seconds RefreshEveryTimer::alignedToSeconds(std::chrono::system return tp_minutes + next_seconds; } +bool RefreshEveryTimer::operator==(const RefreshEveryTimer & rhs) const +{ + static_assert(sizeof(*this) == sizeof(offset) + 8, "RefreshEveryTimer fields appear to have changed. Please update this operator==() here."); + return std::tie(offset, value, kind) == std::tie(rhs.offset, rhs.value, rhs.kind); +} + +std::variant makeTimer(const ASTRefreshStrategy & strategy) +{ + using enum ASTRefreshStrategy::ScheduleKind; + switch (strategy.schedule_kind) + { + case EVERY: + return RefreshEveryTimer{*strategy.period, strategy.interval}; + case AFTER: + return RefreshAfterTimer{strategy.interval}; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown refresh strategy kind"); + } +} + +RefreshTimer::RefreshTimer(const ASTRefreshStrategy & strategy) : timer(makeTimer(strategy)) {} + +namespace +{ + +template +struct CombinedVisitor : Ts... { using Ts::operator()...; }; +template +CombinedVisitor(Ts...) -> CombinedVisitor; + +} + +std::chrono::sys_seconds RefreshTimer::next(std::chrono::system_clock::time_point tp) const +{ + CombinedVisitor visitor{ + [tp](const RefreshAfterTimer & timer_) { return timer_.after(tp); }, + [tp](const RefreshEveryTimer & timer_) { return timer_.next(tp); }}; + auto r = std::visit(std::move(visitor), timer); + chassert(r > tp); + return r; +} + +bool RefreshTimer::operator==(const RefreshTimer & rhs) const { return timer == rhs.timer; } +const RefreshAfterTimer * RefreshTimer::tryGetAfter() const { return std::get_if(&timer); } +const RefreshEveryTimer * RefreshTimer::tryGetEvery() const { return std::get_if(&timer); } + } diff --git a/src/Storages/MaterializedView/RefreshTimers.h b/src/Storages/MaterializedView/RefreshTimers.h index 0672782a3f9..4625e8cd344 100644 --- a/src/Storages/MaterializedView/RefreshTimers.h +++ b/src/Storages/MaterializedView/RefreshTimers.h @@ -9,6 +9,7 @@ namespace DB class ASTTimeInterval; class ASTTimePeriod; +class ASTRefreshStrategy; /// Schedule timer for MATERIALIZED VIEW ... REFRESH AFTER ... queries class RefreshAfterTimer @@ -26,6 +27,8 @@ public: std::chrono::months getMonths() const { return months; } std::chrono::years getYears() const { return years; } + bool operator==(const RefreshAfterTimer & rhs) const; + private: void setWithKind(IntervalKind kind, UInt64 val); @@ -46,6 +49,8 @@ public: std::chrono::sys_seconds next(std::chrono::system_clock::time_point tp) const; + bool operator==(const RefreshEveryTimer & rhs) const; + private: std::chrono::sys_seconds alignedToYears(std::chrono::system_clock::time_point tp) const; @@ -66,4 +71,18 @@ private: IntervalKind kind{IntervalKind::Second}; }; +struct RefreshTimer +{ + std::variant timer; + + explicit RefreshTimer(const ASTRefreshStrategy & strategy); + + std::chrono::sys_seconds next(std::chrono::system_clock::time_point tp) const; + + bool operator==(const RefreshTimer & rhs) const; + + const RefreshAfterTimer * tryGetAfter() const; + const RefreshEveryTimer * tryGetEvery() const; +}; + } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 4f2ffb38017..6504bfa313b 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -451,7 +451,7 @@ void StorageMaterializedView::startup() void StorageMaterializedView::shutdown(bool) { if (refresher) - refresher->stop(); + refresher->shutdown(); auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & select_query = metadata_snapshot->getSelectQuery(); From a7c369e14f61566b2919f04e09ebe2f8cbe4954f Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 25 Nov 2023 03:00:32 +0000 Subject: [PATCH 11/25] Overhaul timestamp arithmetic --- src/Common/CalendarTimeInterval.cpp | 139 ++++++++ src/Common/CalendarTimeInterval.h | 64 ++++ src/Parsers/ASTRefreshStrategy.cpp | 14 +- src/Parsers/ASTRefreshStrategy.h | 23 +- src/Parsers/ASTTimeInterval.cpp | 14 +- src/Parsers/ASTTimeInterval.h | 17 +- src/Parsers/ParserRefreshStrategy.cpp | 31 +- src/Parsers/ParserTimeInterval.cpp | 76 ++--- src/Parsers/ParserTimeInterval.h | 20 +- .../MaterializedView/RefreshSchedule.cpp | 57 ++++ .../MaterializedView/RefreshSchedule.h | 28 ++ src/Storages/MaterializedView/RefreshTask.cpp | 64 ++-- src/Storages/MaterializedView/RefreshTask.h | 29 +- .../MaterializedView/RefreshTimers.cpp | 302 ------------------ src/Storages/MaterializedView/RefreshTimers.h | 88 ----- 15 files changed, 411 insertions(+), 555 deletions(-) create mode 100644 src/Common/CalendarTimeInterval.cpp create mode 100644 src/Common/CalendarTimeInterval.h create mode 100644 src/Storages/MaterializedView/RefreshSchedule.cpp create mode 100644 src/Storages/MaterializedView/RefreshSchedule.h delete mode 100644 src/Storages/MaterializedView/RefreshTimers.cpp delete mode 100644 src/Storages/MaterializedView/RefreshTimers.h diff --git a/src/Common/CalendarTimeInterval.cpp b/src/Common/CalendarTimeInterval.cpp new file mode 100644 index 00000000000..bcedf63b3ff --- /dev/null +++ b/src/Common/CalendarTimeInterval.cpp @@ -0,0 +1,139 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +CalendarTimeInterval::CalendarTimeInterval(const CalendarTimeInterval::Intervals & intervals) +{ + for (auto [kind, val] : intervals) + { + switch (kind.kind) + { + case IntervalKind::Nanosecond: + case IntervalKind::Microsecond: + case IntervalKind::Millisecond: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sub-second intervals are not supported here"); + + case IntervalKind::Second: + case IntervalKind::Minute: + case IntervalKind::Hour: + case IntervalKind::Day: + case IntervalKind::Week: + seconds += val * kind.toAvgSeconds(); + break; + + case IntervalKind::Month: + months += val; + break; + case IntervalKind::Quarter: + months += val * 3; + break; + case IntervalKind::Year: + months += val * 12; + break; + } + } +} + +CalendarTimeInterval::Intervals CalendarTimeInterval::toIntervals() const +{ + Intervals res; + auto greedy = [&](UInt64 x, std::initializer_list> kinds) + { + for (auto [kind, count] : kinds) + { + UInt64 k = x / count; + if (k == 0) + continue; + x -= k * count; + res.emplace_back(kind, k); + } + chassert(x == 0); + }; + greedy(months, {{IntervalKind::Year, 12}, {IntervalKind::Month, 1}}); + greedy(seconds, {{IntervalKind::Week, 3600*24*7}, {IntervalKind::Day, 3600*24}, {IntervalKind::Hour, 3600}, {IntervalKind::Minute, 60}, {IntervalKind::Second, 1}}); + return res; +} + +UInt64 CalendarTimeInterval::minSeconds() const +{ + return 3600*24 * (365 * months/12 + 28 * months%12) + seconds; +} + +UInt64 CalendarTimeInterval::maxSeconds() const +{ + return 3600*24 * (366 * months/12 + 31 * months%12) + seconds; +} + +void CalendarTimeInterval::assertSingleUnit() const +{ + if (seconds && months) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interval shouldn't contain both calendar units and clock units (e.g. months and days)"); +} + +void CalendarTimeInterval::assertPositive() const +{ + if (!seconds && !months) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interval must be positive"); +} + +/// Number of whole months between 1970-01-01 and `t`. +static Int64 toAbsoluteMonth(std::chrono::system_clock::time_point t) +{ + std::chrono::year_month_day ymd(std::chrono::floor(t)); + return (Int64(int(ymd.year())) - 1970) * 12 + Int64(unsigned(ymd.month()) - 1); +} + +static std::chrono::sys_seconds startOfAbsoluteMonth(Int64 absolute_month) +{ + Int64 year = absolute_month >= 0 ? absolute_month/12 : -((-absolute_month+11)/12); + Int64 month = absolute_month - year*12; + chassert(month >= 0 && month < 12); + std::chrono::year_month_day ymd( + std::chrono::year(int(year + 1970)), + std::chrono::month(unsigned(month + 1)), + std::chrono::day(1)); + return std::chrono::sys_days(ymd); +} + +std::chrono::sys_seconds CalendarTimeInterval::advance(std::chrono::system_clock::time_point tp) const +{ + auto t = std::chrono::sys_seconds(std::chrono::floor(tp)); + if (months) + { + auto m = toAbsoluteMonth(t); + auto s = t - startOfAbsoluteMonth(m); + t = startOfAbsoluteMonth(m + Int64(months)) + s; + } + return t + std::chrono::seconds(Int64(seconds)); +} + +std::chrono::sys_seconds CalendarTimeInterval::floor(std::chrono::system_clock::time_point tp) const +{ + assertSingleUnit(); + assertPositive(); + + if (months) + return startOfAbsoluteMonth(toAbsoluteMonth(tp) / months * months); + else + { + constexpr std::chrono::seconds epoch(-3600*24*3); + auto t = std::chrono::sys_seconds(std::chrono::floor(tp)); + /// We want to align with weeks, but 1970-01-01 is a Thursday, so align with 1969-12-29 instead. + return std::chrono::sys_seconds((t.time_since_epoch() - epoch) / seconds * seconds + epoch); + } +} + +bool CalendarTimeInterval::operator==(const CalendarTimeInterval & rhs) const +{ + return std::tie(months, seconds) == std::tie(rhs.months, rhs.seconds); +} + +} diff --git a/src/Common/CalendarTimeInterval.h b/src/Common/CalendarTimeInterval.h new file mode 100644 index 00000000000..40a390736bc --- /dev/null +++ b/src/Common/CalendarTimeInterval.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/// Represents a duration of calendar time, e.g.: +/// * 2 weeks + 5 minutes + and 21 seconds (aka 605121 seconds), +/// * 1 (calendar) month - not equivalent to any number of seconds! +/// * 3 years + 2 weeks (aka 36 months + 604800 seconds). +/// +/// Be careful with calendar arithmetic: it's missing many familiar properties of numbers. +/// E.g. x + y - y is not always equal to x (October 31 + 1 month - 1 month = November 1). +struct CalendarTimeInterval +{ + UInt64 seconds = 0; + UInt64 months = 0; + + using Intervals = std::vector>; + + CalendarTimeInterval() = default; + + /// Year, Quarter, Month are converted to months. + /// Week, Day, Hour, Minute, Second are converted to seconds. + /// Millisecond, Microsecond, Nanosecond throw exception. + explicit CalendarTimeInterval(const Intervals & intervals); + + /// E.g. for {36 months, 604801 seconds} returns {3 years, 2 weeks, 1 second}. + Intervals toIntervals() const; + + /// Approximate shortest and longest duration in seconds. E.g. a month is [28, 31] days. + UInt64 minSeconds() const; + UInt64 maxSeconds() const; + + /// Checks that the interval has only months or only seconds, throws otherwise. + void assertSingleUnit() const; + void assertPositive() const; + + /// Add this interval to the timestamp. First months, then seconds. + /// Gets weird near month boundaries: October 31 + 1 month = December 1. + /// Gets weird with leap years: 2004-03-15 + 1 year = 2005-03-16, + /// 2004-12-31 + 1 year = 2006-01-01, + std::chrono::sys_seconds advance(std::chrono::system_clock::time_point t) const; + + /// Rounds the timestamp down to the nearest timestamp "aligned" with this interval. + /// The interval must satisfy assertSingleUnit() and assertPositive(). + /// * For months, rounds to the start of a month whose abosolute index is divisible by `months`. + /// The month index is 0-based starting from January 1970. + /// E.g. if the interval is 1 month, rounds down to the start of the month. + /// * For seconds, rounds to a timestamp x such that (x - December 29 1969 (Monday)) is divisible + /// by this interval. + /// E.g. if the interval is 1 week, rounds down to the start of the week. + /// + /// Guarantees: + /// * advance(floor(x)) > x + /// * floor(advance(floor(x))) = advance(floor(x)) + std::chrono::sys_seconds floor(std::chrono::system_clock::time_point t) const; + + bool operator==(const CalendarTimeInterval & rhs) const; +}; + +} diff --git a/src/Parsers/ASTRefreshStrategy.cpp b/src/Parsers/ASTRefreshStrategy.cpp index ed8216cf4aa..2e0c6ee4638 100644 --- a/src/Parsers/ASTRefreshStrategy.cpp +++ b/src/Parsers/ASTRefreshStrategy.cpp @@ -10,12 +10,10 @@ ASTPtr ASTRefreshStrategy::clone() const auto res = std::make_shared(*this); res->children.clear(); - if (interval) - res->set(res->interval, interval->clone()); if (period) res->set(res->period, period->clone()); - if (periodic_offset) - res->set(res->periodic_offset, periodic_offset->clone()); + if (offset) + res->set(res->offset, offset->clone()); if (spread) res->set(res->spread, spread->clone()); if (settings) @@ -32,20 +30,20 @@ void ASTRefreshStrategy::formatImpl( frame.need_parens = false; f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << "REFRESH " << (f_settings.hilite ? hilite_none : ""); - using enum ScheduleKind; + using enum RefreshScheduleKind; switch (schedule_kind) { case AFTER: f_settings.ostr << "AFTER " << (f_settings.hilite ? hilite_none : ""); - interval->formatImpl(f_settings, state, frame); + period->formatImpl(f_settings, state, frame); break; case EVERY: f_settings.ostr << "EVERY " << (f_settings.hilite ? hilite_none : ""); period->formatImpl(f_settings, state, frame); - if (periodic_offset) + if (offset) { f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << " OFFSET " << (f_settings.hilite ? hilite_none : ""); - periodic_offset->formatImpl(f_settings, state, frame); + offset->formatImpl(f_settings, state, frame); } break; default: diff --git a/src/Parsers/ASTRefreshStrategy.h b/src/Parsers/ASTRefreshStrategy.h index 0df6a6e0e10..ca248b76b40 100644 --- a/src/Parsers/ASTRefreshStrategy.h +++ b/src/Parsers/ASTRefreshStrategy.h @@ -7,24 +7,23 @@ namespace DB { +enum class RefreshScheduleKind : UInt8 +{ + UNKNOWN = 0, + AFTER, + EVERY +}; + /// Strategy for MATERIALIZED VIEW ... REFRESH .. class ASTRefreshStrategy : public IAST { public: - enum class ScheduleKind : UInt8 - { - UNKNOWN = 0, - AFTER, - EVERY - }; - ASTSetQuery * settings = nullptr; ASTExpressionList * dependencies = nullptr; - ASTTimeInterval * interval = nullptr; - ASTTimePeriod * period = nullptr; - ASTTimeInterval * periodic_offset = nullptr; - ASTTimePeriod * spread = nullptr; - ScheduleKind schedule_kind{ScheduleKind::UNKNOWN}; + ASTTimeInterval * period = nullptr; + ASTTimeInterval * offset = nullptr; + ASTTimeInterval * spread = nullptr; + RefreshScheduleKind schedule_kind{RefreshScheduleKind::UNKNOWN}; String getID(char) const override { return "Refresh strategy definition"; } diff --git a/src/Parsers/ASTTimeInterval.cpp b/src/Parsers/ASTTimeInterval.cpp index 13e07f4ed65..44f6e577e94 100644 --- a/src/Parsers/ASTTimeInterval.cpp +++ b/src/Parsers/ASTTimeInterval.cpp @@ -7,18 +7,6 @@ namespace DB { -ASTPtr ASTTimePeriod::clone() const -{ - return std::make_shared(*this); -} - -void ASTTimePeriod::formatImpl(const FormatSettings & f_settings, FormatState &, FormatStateStacked frame) const -{ - frame.need_parens = false; - f_settings.ostr << value << ' '; - f_settings.ostr << (f_settings.hilite ? hilite_keyword : "") << kind.toKeyword() << (f_settings.hilite ? hilite_none : ""); -} - ASTPtr ASTTimeInterval::clone() const { return std::make_shared(*this); @@ -28,7 +16,7 @@ void ASTTimeInterval::formatImpl(const FormatSettings & f_settings, FormatState { frame.need_parens = false; - for (bool is_first = true; auto [kind, value] : kinds | std::views::reverse) + for (bool is_first = true; auto [kind, value] : interval.toIntervals()) { if (!std::exchange(is_first, false)) f_settings.ostr << ' '; diff --git a/src/Parsers/ASTTimeInterval.h b/src/Parsers/ASTTimeInterval.h index a8f2518d180..a68acd0f8ea 100644 --- a/src/Parsers/ASTTimeInterval.h +++ b/src/Parsers/ASTTimeInterval.h @@ -2,31 +2,18 @@ #include -#include +#include #include namespace DB { -/// Simple periodic time interval like 10 SECOND -class ASTTimePeriod : public IAST -{ -public: - UInt64 value{0}; - IntervalKind kind{IntervalKind::Second}; - - String getID(char) const override { return "TimePeriod"; } - - ASTPtr clone() const override; - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; -}; - /// Compound time interval like 1 YEAR 3 DAY 15 MINUTE class ASTTimeInterval : public IAST { public: - std::map kinds; + CalendarTimeInterval interval; String getID(char) const override { return "TimeInterval"; } diff --git a/src/Parsers/ParserRefreshStrategy.cpp b/src/Parsers/ParserRefreshStrategy.cpp index 05dd081e61d..af518ee203e 100644 --- a/src/Parsers/ParserRefreshStrategy.cpp +++ b/src/Parsers/ParserRefreshStrategy.cpp @@ -11,41 +11,52 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + bool ParserRefreshStrategy::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto refresh = std::make_shared(); if (ParserKeyword{"AFTER"}.ignore(pos, expected)) { - refresh->schedule_kind = ASTRefreshStrategy::ScheduleKind::AFTER; - ASTPtr interval; - if (!ParserTimeInterval{}.parse(pos, interval, expected)) + refresh->schedule_kind = RefreshScheduleKind::AFTER; + ASTPtr period; + if (!ParserTimeInterval{}.parse(pos, period, expected)) return false; - refresh->set(refresh->interval, interval); + refresh->set(refresh->period, period); } else if (ParserKeyword{"EVERY"}.ignore(pos, expected)) { - refresh->schedule_kind = ASTRefreshStrategy::ScheduleKind::EVERY; + refresh->schedule_kind = RefreshScheduleKind::EVERY; ASTPtr period; - if (!ParserTimePeriod{}.parse(pos, period, expected)) + if (!ParserTimeInterval{{.allow_mixing_calendar_and_clock_units = false}}.parse(pos, period, expected)) return false; refresh->set(refresh->period, period); if (ParserKeyword{"OFFSET"}.ignore(pos, expected)) { ASTPtr periodic_offset; - if (!ParserTimeInterval{}.parse(pos, periodic_offset, expected)) + if (!ParserTimeInterval{{.allow_zero = true}}.parse(pos, periodic_offset, expected)) return false; - refresh->set(refresh->periodic_offset, periodic_offset); + + if (periodic_offset->as()->interval.maxSeconds() + >= period->as()->interval.minSeconds()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "OFFSET must be less than the period"); + + refresh->set(refresh->offset, periodic_offset); } } - if (refresh->schedule_kind == ASTRefreshStrategy::ScheduleKind::UNKNOWN) + if (refresh->schedule_kind == RefreshScheduleKind::UNKNOWN) return false; if (ParserKeyword{"RANDOMIZE FOR"}.ignore(pos, expected)) { ASTPtr spread; - if (!ParserTimePeriod{}.parse(pos, spread, expected)) + if (!ParserTimeInterval{{.allow_zero = true}}.parse(pos, spread, expected)) return false; refresh->set(refresh->spread, spread); diff --git a/src/Parsers/ParserTimeInterval.cpp b/src/Parsers/ParserTimeInterval.cpp index 8168ae61a33..8454eb27e1d 100644 --- a/src/Parsers/ParserTimeInterval.cpp +++ b/src/Parsers/ParserTimeInterval.cpp @@ -14,64 +14,40 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -namespace -{ - -struct ValKind -{ - UInt64 val; - IntervalKind kind; - bool empty; -}; - -std::optional parseValKind(IParser::Pos & pos, Expected & expected) -{ - ASTPtr value; - IntervalKind kind; - if (!ParserNumber{}.parse(pos, value, expected)) - return ValKind{ .empty = true }; - if (!parseIntervalKind(pos, expected, kind)) - return {}; - UInt64 val; - if (!value->as().value.tryGet(val)) - throw Exception(ErrorCodes::SYNTAX_ERROR, "Time interval must be an integer"); - return ValKind{ val, kind, false }; -} - -} - -bool ParserTimePeriod::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - auto parsed = parseValKind(pos, expected); - - if (!parsed || parsed->empty || parsed->val == 0) - return false; - - auto time_period = std::make_shared(); - time_period->value = parsed->val; - time_period->kind = parsed->kind; - - node = time_period; - return true; -} +ParserTimeInterval::ParserTimeInterval(Options opt) : options(opt) {} +ParserTimeInterval::ParserTimeInterval() = default; bool ParserTimeInterval::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - auto time_interval = std::make_shared(); - - auto parsed = parseValKind(pos, expected); - while (parsed && !parsed->empty) + CalendarTimeInterval::Intervals intervals; + while (true) { - if (parsed->val == 0) + ASTPtr value; + IntervalKind kind; + if (!ParserNumber{}.parse(pos, value, expected)) + break; + if (!parseIntervalKind(pos, expected, kind)) return false; - auto [it, inserted] = time_interval->kinds.emplace(parsed->kind, parsed->val); - if (!inserted) - return false; - parsed = parseValKind(pos, expected); + + UInt64 val; + if (!value->as().value.tryGet(val)) + throw Exception(ErrorCodes::SYNTAX_ERROR, "Time interval must be an integer"); + intervals.emplace_back(kind, val); } - if (!parsed || time_interval->kinds.empty()) + if (intervals.empty()) return false; + + CalendarTimeInterval interval(intervals); + + if (!options.allow_zero) + interval.assertPositive(); + if (!options.allow_mixing_calendar_and_clock_units) + interval.assertSingleUnit(); + + auto time_interval = std::make_shared(); + time_interval->interval = interval; + node = time_interval; return true; } diff --git a/src/Parsers/ParserTimeInterval.h b/src/Parsers/ParserTimeInterval.h index 6eae1fa4133..2a6d7fd2534 100644 --- a/src/Parsers/ParserTimeInterval.h +++ b/src/Parsers/ParserTimeInterval.h @@ -5,18 +5,22 @@ namespace DB { -/// Parser for ASTTimePeriod -class ParserTimePeriod : public IParserBase -{ -protected: - const char * getName() const override { return "time period"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - /// Parser for ASTTimeInterval class ParserTimeInterval : public IParserBase { +public: + struct Options + { + bool allow_mixing_calendar_and_clock_units = true; + bool allow_zero = false; + }; + + ParserTimeInterval(); + explicit ParserTimeInterval(Options opt); + protected: + Options options; + const char * getName() const override { return "time interval"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; diff --git a/src/Storages/MaterializedView/RefreshSchedule.cpp b/src/Storages/MaterializedView/RefreshSchedule.cpp new file mode 100644 index 00000000000..fab6877848c --- /dev/null +++ b/src/Storages/MaterializedView/RefreshSchedule.cpp @@ -0,0 +1,57 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +RefreshSchedule::RefreshSchedule(const ASTRefreshStrategy & strategy) +{ + kind = strategy.schedule_kind; + period = strategy.period->interval; + if (strategy.offset) + offset = strategy.offset->interval; + if (strategy.spread) + spread = strategy.spread->interval; +} + +static std::chrono::sys_seconds advanceEvery(std::chrono::system_clock::time_point prev, CalendarTimeInterval period, CalendarTimeInterval offset) +{ + auto period_start = period.floor(prev); + auto t = offset.advance(period_start); + if (t > prev) + return t; + t = offset.advance(period.advance(period_start)); + chassert(t > prev); + return t; +} + +std::chrono::sys_seconds RefreshSchedule::prescribeNext( + std::chrono::system_clock::time_point last_prescribed, std::chrono::system_clock::time_point now) const +{ + if (kind == RefreshScheduleKind::AFTER) + return period.advance(now); + + /// It's important to use prescribed instead of actual time here, otherwise we would do multiple + /// refreshes instead of one if the generated spread is negative and the the refresh completes + /// faster than the spread. + auto res = advanceEvery(last_prescribed, period, offset); + if (res < now) + res = advanceEvery(now, period, offset); // fell behind by a whole period, skip to current time + + return res; +} + +std::chrono::system_clock::time_point RefreshSchedule::addRandomSpread(std::chrono::sys_seconds prescribed_time) const +{ + Int64 ms = Int64(spread.minSeconds() * 1000 / 2); + auto add = std::uniform_int_distribution(-ms, ms)(thread_local_rng); + return prescribed_time + std::chrono::milliseconds(add); +} + +} diff --git a/src/Storages/MaterializedView/RefreshSchedule.h b/src/Storages/MaterializedView/RefreshSchedule.h new file mode 100644 index 00000000000..2a6a5afc3ad --- /dev/null +++ b/src/Storages/MaterializedView/RefreshSchedule.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ASTRefreshStrategy; + +struct RefreshSchedule +{ + RefreshScheduleKind kind; + CalendarTimeInterval period; + CalendarTimeInterval offset; + CalendarTimeInterval spread; + + explicit RefreshSchedule(const ASTRefreshStrategy & strategy); + + /// Tells when to do the next refresh (without random spread). + std::chrono::sys_seconds prescribeNext( + std::chrono::system_clock::time_point last_prescribed, std::chrono::system_clock::time_point now) const; + + std::chrono::system_clock::time_point addRandomSpread(std::chrono::sys_seconds prescribed_time) const; +}; + +} diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 000ee7aa1bd..710d48efdb8 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -17,24 +17,10 @@ namespace CurrentMetrics namespace DB { -namespace -{ - -std::uniform_int_distribution makeSpreadDistribution(const ASTTimePeriod * spread) -{ - if (!spread) - return std::uniform_int_distribution(0, 0); - Int64 limit = spread->kind.toAvgSeconds() * spread->value / 2; - return std::uniform_int_distribution(-limit, limit); -} - -} - RefreshTask::RefreshTask( const ASTRefreshStrategy & strategy) : log(&Poco::Logger::get("RefreshTask")) - , refresh_timer(strategy) - , refresh_spread{makeSpreadDistribution(strategy.spread)} + , refresh_schedule(strategy) {} RefreshTaskHolder RefreshTask::create( @@ -66,7 +52,7 @@ void RefreshTask::initializeAndStart(std::shared_ptr vi view_to_refresh = view; /// TODO: Add a setting to stop views on startup, set `stop_requested = true` in that case. populateDependencies(); - calculateNextRefreshTime(std::chrono::system_clock::now()); + advanceNextRefreshTime(std::chrono::system_clock::now()); refresh_task->schedule(); } @@ -139,7 +125,7 @@ void RefreshTask::shutdown() set_entry.reset(); } -void RefreshTask::notify(const StorageID & parent_id, std::chrono::system_clock::time_point scheduled_time_without_spread, const RefreshTimer & parent_timer) +void RefreshTask::notify(const StorageID & parent_id, std::chrono::sys_seconds prescribed_time, const RefreshSchedule & parent_schedule) { std::lock_guard guard(mutex); if (!set_entry) @@ -177,7 +163,7 @@ void RefreshTask::notify(const StorageID & parent_id, std::chrono::system_clock: /// Only accept the dependency's refresh if its next refresh time is after ours. /// This takes care of cases (1)-(4), and seems harmless in all other cases. /// Might be mildly helpful in weird cases like REFRESH AFTER 3 HOUR depends on REFRESH AFTER 2 HOUR. - if (parent_timer.next(scheduled_time_without_spread) <= next_refresh_without_spread) + if (parent_schedule.prescribeNext(prescribed_time, std::chrono::system_clock::now()) <= next_refresh_prescribed) return; if (arriveDependency(parent_id) && !std::exchange(refresh_immediately, true)) @@ -185,11 +171,13 @@ void RefreshTask::notify(const StorageID & parent_id, std::chrono::system_clock: /// Decrease delay in case (5). /// Maybe we should do it for all AFTER-AFTER dependencies, even if periods are different. - if (refresh_timer == parent_timer && refresh_timer.tryGetAfter()) + if (refresh_schedule.kind == RefreshScheduleKind::AFTER && + parent_schedule.kind == RefreshScheduleKind::AFTER && + refresh_schedule.period == parent_schedule.period) { - /// TODO: Implement this: + /// TODO: Implement this. /// * Add setting max_after_delay_adjustment_pct - /// * Decrease both next_refresh_without_spread and next_refresh_with_spread, + /// * Decrease both next_refresh_prescribed and next_refresh_with_spread, /// but only if they haven't already been decreased this way during current period /// * refresh_task->schedule() } @@ -217,11 +205,11 @@ void RefreshTask::refreshTask() if (cancel_requested) { - /// Advance to the next refresh time according to schedule. + /// Move on to the next refresh time according to schedule. /// Otherwise we'd start another refresh immediately after canceling this one. auto now = std::chrono::system_clock::now(); if (now >= next_refresh_with_spread) - calculateNextRefreshTime(std::chrono::system_clock::now()); + advanceNextRefreshTime(now); } } @@ -277,7 +265,7 @@ void RefreshTask::refreshTask() reportState(RefreshState::Running); CurrentMetrics::Increment metric_inc(CurrentMetrics::RefreshingViews); - auto scheduled_time_without_spread = next_refresh_without_spread; + auto prescribed_time = next_refresh_prescribed; lock.unlock(); @@ -291,7 +279,7 @@ void RefreshTask::refreshTask() finished = executeRefresh(); if (finished) - completeRefresh(view, LastTaskResult::Finished, scheduled_time_without_spread); + completeRefresh(view, LastTaskResult::Finished, prescribed_time); } catch (...) { @@ -311,7 +299,7 @@ void RefreshTask::refreshTask() { auto now = std::chrono::system_clock::now(); reportLastRefreshTime(now); - calculateNextRefreshTime(now); + advanceNextRefreshTime(now); } } } @@ -351,7 +339,7 @@ bool RefreshTask::executeRefresh() return !not_finished; } -void RefreshTask::completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::system_clock::time_point scheduled_time_without_spread) +void RefreshTask::completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::sys_seconds prescribed_time) { auto stale_table = view->exchangeTargetTable(refresh_query->table_id); @@ -359,7 +347,7 @@ void RefreshTask::completeRefresh(std::shared_ptr view, StorageID my_id = set_entry->getID(); auto dependents = context->getRefreshSet().getDependents(my_id); for (const RefreshTaskHolder & dep_task : dependents) - dep_task->notify(my_id, scheduled_time_without_spread, refresh_timer); + dep_task->notify(my_id, prescribed_time, refresh_schedule); auto drop_context = Context::createCopy(context); InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, drop_context, drop_context, stale_table, /*sync=*/true); @@ -396,28 +384,24 @@ void RefreshTask::cleanState() refresh_query.reset(); } -void RefreshTask::calculateNextRefreshTime(std::chrono::system_clock::time_point now) +void RefreshTask::advanceNextRefreshTime(std::chrono::system_clock::time_point now) { /// TODO: Add a setting to randomize initial delay in case of AFTER, for the case when the server /// is restarted more often than the refresh period. /// TODO: Maybe do something like skip_update_after_seconds and skip_update_after_ratio. - /// Unclear if that's useful at all if the last refresh timestamp is not remembered across restarts. + /// Or maybe that should be checked in refreshTask(), just before starting a refresh. + /// Probably only useful after we have concurrency limits. Or maybe it's not useful even then? - /// It's important to use time without spread here, otherwise we would do multiple refreshes instead - /// of one, if the generated spread is negative and the first refresh completes faster than the spread. - std::chrono::sys_seconds next = refresh_timer.next(next_refresh_without_spread); - if (next < now) - next = refresh_timer.next(now); // fell behind, skip to current time - - next_refresh_without_spread = next; - next_refresh_with_spread = next + std::chrono::seconds{refresh_spread(thread_local_rng)}; + std::chrono::sys_seconds next = refresh_schedule.prescribeNext(next_refresh_prescribed, now); + next_refresh_prescribed = next; + next_refresh_with_spread = refresh_schedule.addRandomSpread(next); reportNextRefreshTime(next_refresh_with_spread); } -bool RefreshTask::arriveDependency(const StorageID & parent_table_or_timer) +bool RefreshTask::arriveDependency(const StorageID & parent) { - remaining_dependencies.erase(parent_table_or_timer); + remaining_dependencies.erase(parent); if (!remaining_dependencies.empty() || !time_arrived) return false; populateDependencies(); diff --git a/src/Storages/MaterializedView/RefreshTask.h b/src/Storages/MaterializedView/RefreshTask.h index cdb0d22342e..342fe3fc514 100644 --- a/src/Storages/MaterializedView/RefreshTask.h +++ b/src/Storages/MaterializedView/RefreshTask.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include @@ -72,16 +72,14 @@ public: void shutdown(); /// Notify dependent task - void notify(const StorageID & parent_id, std::chrono::system_clock::time_point scheduled_time_without_spread, const RefreshTimer & parent_timer); + void notify(const StorageID & parent_id, std::chrono::sys_seconds prescribed_time, const RefreshSchedule & parent_schedule); private: Poco::Logger * log = nullptr; std::weak_ptr view_to_refresh; RefreshSet::Entry set_entry; - /// Refresh schedule - RefreshTimer refresh_timer; - std::uniform_int_distribution refresh_spread; + RefreshSchedule refresh_schedule; /// Task execution. Non-empty iff a refresh is in progress (possibly paused). /// Whoever unsets these should also call storeLastState(). @@ -112,7 +110,20 @@ private: std::atomic_bool interrupt_execution {false}; /// When to refresh next. Updated when a refresh is finished or canceled. - std::chrono::system_clock::time_point next_refresh_without_spread; + /// We maintain the distinction between: + /// * The "prescribed" time of the refresh, dictated by the refresh schedule. + /// E.g. for REFERSH EVERY 1 DAY, the prescribed time is always at the exact start of a day. + /// * Actual wall clock timestamps, e.g. when the refresh is scheduled to happen + /// (including random spread) or when a refresh completed. + /// The prescribed time is required for: + /// * Doing REFRESH EVERY correctly if the random spread came up negative, and a refresh completed + /// before the prescribed time. E.g. suppose a refresh was prescribed at 05:00, which was randomly + /// adjusted to 4:50, and the refresh completed at 4:55; we shouldn't schedule another refresh + /// at 5:00, so we should remember that the 4:50-4:55 refresh actually had prescribed time 5:00. + /// * Similarly, for dependencies between REFRESH EVERY tables, using actual time would be unreliable. + /// 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; /// Calls refreshTask() from background thread. @@ -132,15 +143,15 @@ private: /// Methods that do the actual work: creating/dropping internal table, executing the query. void initializeRefresh(std::shared_ptr view); bool executeRefresh(); - void completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::system_clock::time_point scheduled_time_without_spread); + void completeRefresh(std::shared_ptr view, LastTaskResult result, std::chrono::sys_seconds prescribed_time); void cancelRefresh(LastTaskResult result); void cleanState(); /// Assigns next_refresh_* - void calculateNextRefreshTime(std::chrono::system_clock::time_point now); + void advanceNextRefreshTime(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_table_or_timer); + bool arriveDependency(const StorageID & parent); bool arriveTime(); void populateDependencies(); diff --git a/src/Storages/MaterializedView/RefreshTimers.cpp b/src/Storages/MaterializedView/RefreshTimers.cpp deleted file mode 100644 index 0331bad82c3..00000000000 --- a/src/Storages/MaterializedView/RefreshTimers.cpp +++ /dev/null @@ -1,302 +0,0 @@ -#include - -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -namespace -{ - constexpr std::chrono::days ZERO_DAYS{0}; - constexpr std::chrono::days ONE_DAY{1}; -} - -RefreshAfterTimer::RefreshAfterTimer(const ASTTimeInterval * time_interval) -{ - if (time_interval) - { - for (auto && [kind, value] : time_interval->kinds) - setWithKind(kind, value); - } -} - -std::chrono::sys_seconds RefreshAfterTimer::after(std::chrono::system_clock::time_point tp) const -{ - auto tp_date = std::chrono::floor(tp); - auto tp_time_offset = std::chrono::floor(tp - tp_date); - std::chrono::year_month_day ymd(tp_date); - ymd += years; - ymd += months; - std::chrono::sys_days date = ymd; - date += weeks; - date += days; - auto result = std::chrono::time_point_cast(date); - result += tp_time_offset; - result += hours; - result += minutes; - result += seconds; - return result; -} - -void RefreshAfterTimer::setWithKind(IntervalKind kind, UInt64 val) -{ - switch (kind) - { - case IntervalKind::Second: - seconds = std::chrono::seconds{val}; - break; - case IntervalKind::Minute: - minutes = std::chrono::minutes{val}; - break; - case IntervalKind::Hour: - hours = std::chrono::hours{val}; - break; - case IntervalKind::Day: - days = std::chrono::days{val}; - break; - case IntervalKind::Week: - weeks = std::chrono::weeks{val}; - break; - case IntervalKind::Month: - months = std::chrono::months{val}; - break; - case IntervalKind::Year: - years = std::chrono::years{val}; - break; - default: - break; - } -} - -bool RefreshAfterTimer::operator==(const RefreshAfterTimer & rhs) const -{ - /// (Or maybe different implementations of standard library have different sizes of chrono types. - /// If so, feel free to just remove this assert.) - static_assert(sizeof(*this) == 40, "RefreshAfterTimer fields appear to have changed. Please update this operator==() here."); - return std::tie(seconds, minutes, hours, days, weeks, months, years) == std::tie(rhs.seconds, rhs.minutes, rhs.hours, rhs.days, rhs.weeks, rhs.months, rhs.years); -} - -RefreshEveryTimer::RefreshEveryTimer(const ASTTimePeriod & time_period, const ASTTimeInterval * time_offset) - : offset(time_offset) - , value{static_cast(time_period.value)} - , kind{time_period.kind} -{ - // TODO: validate invariants -} - -std::chrono::sys_seconds RefreshEveryTimer::next(std::chrono::system_clock::time_point tp) const -{ - if (value == 0) - return std::chrono::floor(tp); - switch (kind) - { - case IntervalKind::Second: - return alignedToSeconds(tp); - case IntervalKind::Minute: - return alignedToMinutes(tp); - case IntervalKind::Hour: - return alignedToHours(tp); - case IntervalKind::Day: - return alignedToDays(tp); - case IntervalKind::Week: - return alignedToWeeks(tp); - case IntervalKind::Month: - return alignedToMonths(tp); - case IntervalKind::Year: - return alignedToYears(tp); - default: - return std::chrono::ceil(tp); - } -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToYears(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto tp_days = std::chrono::floor(tp); - std::chrono::year_month_day tp_ymd(tp_days); - auto normalize_years = [](std::chrono::year year) -> std::chrono::sys_days - { - return year / std::chrono::January / 1d; - }; - - auto prev_years = normalize_years(tp_ymd.year()); - if (auto prev_time = offset.after(prev_years); prev_time > tp) - return prev_time; - - auto next_years = normalize_years(std::chrono::year((int(tp_ymd.year()) / value + 1) * value)); - return offset.after(next_years); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToMonths(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto tp_days = std::chrono::floor(tp); - std::chrono::year_month_day tp_ymd(tp_days); - auto normalize_months = [](const std::chrono::year_month_day & ymd, unsigned month_value) -> std::chrono::sys_days - { - return ymd.year() / std::chrono::month{month_value} / 1d; - }; - - auto prev_month_value = static_cast(tp_ymd.month()) / value * value; - auto prev_months = normalize_months(tp_ymd, prev_month_value); - if (auto prev_time = offset.after(prev_months); prev_time > tp) - return prev_time; - - auto next_month_value = (static_cast(tp_ymd.month()) / value + 1) * value; - auto next_months = normalize_months(tp_ymd, next_month_value); - std::chrono::year_month_day next_ymd(next_months); - if (next_ymd.year() > tp_ymd.year()) - return offset.after(normalize_months(next_ymd, value)); - return offset.after(next_months); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToWeeks(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto cpp_weekday = offset.getDays() + ONE_DAY; - std::chrono::weekday offset_weekday((cpp_weekday - std::chrono::floor(cpp_weekday)).count()); - - auto tp_days = std::chrono::floor(tp); - std::chrono::year_month_weekday tp_ymd(tp_days); - auto normalize_weeks = [offset_weekday](const std::chrono::year_month_weekday & ymd, unsigned week_value) - { - return std::chrono::sys_days(ymd.year() / ymd.month() / std::chrono::weekday{offset_weekday}[week_value]); - }; - - auto prev_week_value = tp_ymd.index() / value * value; - auto prev_days = normalize_weeks(tp_ymd, prev_week_value); - if (auto prev_time = offset.after(prev_days - offset.getDays()); prev_time > tp) - return prev_time; - - auto next_day_value = (tp_ymd.index() / value + 1) * value; - auto next_days = normalize_weeks(tp_ymd, next_day_value); - std::chrono::year_month_weekday next_ymd(next_days); - if (next_ymd.year() > tp_ymd.year() || next_ymd.month() > tp_ymd.month()) - return offset.after(normalize_weeks(next_ymd, value) - offset.getDays()); - return offset.after(next_days); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToDays(std::chrono::system_clock::time_point tp) const -{ - auto tp_days = std::chrono::floor(tp); - std::chrono::year_month_day tp_ymd(tp_days); - auto normalize_days = [](const std::chrono::year_month_day & ymd, unsigned day_value) -> std::chrono::sys_days - { - return ymd.year() / ymd.month() / std::chrono::day{day_value}; - }; - - auto prev_day_value = static_cast(tp_ymd.day()) / value * value; - auto prev_days = normalize_days(tp_ymd, prev_day_value); - if (auto prev_time = offset.after(prev_days); prev_time > tp) - return prev_time; - - auto next_day_value = (static_cast(tp_ymd.day()) / value + 1) * value; - auto next_days = normalize_days(tp_ymd, next_day_value); - std::chrono::year_month_day next_ymd(next_days); - if (next_ymd.year() > tp_ymd.year() || next_ymd.month() > tp_ymd.month()) - return offset.after(normalize_days(next_ymd, value)); - return offset.after(next_days); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToHours(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto tp_days = std::chrono::floor(tp); - auto tp_hours = std::chrono::floor(tp - tp_days); - - auto prev_hours = (tp_hours / value) * value; - if (auto prev_time = offset.after(tp_days + prev_hours); prev_time > tp) - return prev_time; - - auto next_hours = (tp_hours / value + 1h) * value; - if (std::chrono::floor(next_hours - 1h) > ZERO_DAYS) - return offset.after(tp_days + ONE_DAY + std::chrono::hours{value}); - return offset.after(tp_days + next_hours); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToMinutes(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto tp_hours = std::chrono::floor(tp); - auto tp_minutes = std::chrono::floor(tp - tp_hours); - - auto prev_minutes = (tp_minutes / value) * value; - if (auto prev_time = offset.after(tp_hours + prev_minutes); prev_time > tp) - return prev_time; - - auto next_minutes = (tp_minutes / value + 1min) * value; - if (std::chrono::floor(next_minutes - 1min) > 0h) - return offset.after(tp_hours + 1h + std::chrono::minutes{value}); - return offset.after(tp_hours + next_minutes); -} - -std::chrono::sys_seconds RefreshEveryTimer::alignedToSeconds(std::chrono::system_clock::time_point tp) const -{ - using namespace std::chrono_literals; - - auto tp_minutes = std::chrono::floor(tp); - auto tp_seconds = std::chrono::floor(tp - tp_minutes); - - auto next_seconds = (tp_seconds / value + 1s) * value; - if (std::chrono::floor(next_seconds - 1s) > 0min) - return tp_minutes + 1min + std::chrono::seconds{value}; - return tp_minutes + next_seconds; -} - -bool RefreshEveryTimer::operator==(const RefreshEveryTimer & rhs) const -{ - static_assert(sizeof(*this) == sizeof(offset) + 8, "RefreshEveryTimer fields appear to have changed. Please update this operator==() here."); - return std::tie(offset, value, kind) == std::tie(rhs.offset, rhs.value, rhs.kind); -} - -std::variant makeTimer(const ASTRefreshStrategy & strategy) -{ - using enum ASTRefreshStrategy::ScheduleKind; - switch (strategy.schedule_kind) - { - case EVERY: - return RefreshEveryTimer{*strategy.period, strategy.interval}; - case AFTER: - return RefreshAfterTimer{strategy.interval}; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown refresh strategy kind"); - } -} - -RefreshTimer::RefreshTimer(const ASTRefreshStrategy & strategy) : timer(makeTimer(strategy)) {} - -namespace -{ - -template -struct CombinedVisitor : Ts... { using Ts::operator()...; }; -template -CombinedVisitor(Ts...) -> CombinedVisitor; - -} - -std::chrono::sys_seconds RefreshTimer::next(std::chrono::system_clock::time_point tp) const -{ - CombinedVisitor visitor{ - [tp](const RefreshAfterTimer & timer_) { return timer_.after(tp); }, - [tp](const RefreshEveryTimer & timer_) { return timer_.next(tp); }}; - auto r = std::visit(std::move(visitor), timer); - chassert(r > tp); - return r; -} - -bool RefreshTimer::operator==(const RefreshTimer & rhs) const { return timer == rhs.timer; } -const RefreshAfterTimer * RefreshTimer::tryGetAfter() const { return std::get_if(&timer); } -const RefreshEveryTimer * RefreshTimer::tryGetEvery() const { return std::get_if(&timer); } - -} diff --git a/src/Storages/MaterializedView/RefreshTimers.h b/src/Storages/MaterializedView/RefreshTimers.h deleted file mode 100644 index 4625e8cd344..00000000000 --- a/src/Storages/MaterializedView/RefreshTimers.h +++ /dev/null @@ -1,88 +0,0 @@ -#pragma once - -#include - -#include - -namespace DB -{ - -class ASTTimeInterval; -class ASTTimePeriod; -class ASTRefreshStrategy; - -/// Schedule timer for MATERIALIZED VIEW ... REFRESH AFTER ... queries -class RefreshAfterTimer -{ -public: - explicit RefreshAfterTimer(const ASTTimeInterval * time_interval); - - std::chrono::sys_seconds after(std::chrono::system_clock::time_point tp) const; - - std::chrono::seconds getSeconds() const { return seconds; } - std::chrono::minutes getMinutes() const { return minutes; } - std::chrono::hours getHours() const { return hours; } - std::chrono::days getDays() const { return days; } - std::chrono::weeks getWeeks() const { return weeks; } - std::chrono::months getMonths() const { return months; } - std::chrono::years getYears() const { return years; } - - bool operator==(const RefreshAfterTimer & rhs) const; - -private: - void setWithKind(IntervalKind kind, UInt64 val); - - std::chrono::seconds seconds{0}; - std::chrono::minutes minutes{0}; - std::chrono::hours hours{0}; - std::chrono::days days{0}; - std::chrono::weeks weeks{0}; - std::chrono::months months{0}; - std::chrono::years years{0}; -}; - -/// Schedule timer for MATERIALIZED VIEW ... REFRESH EVERY ... queries -class RefreshEveryTimer -{ -public: - explicit RefreshEveryTimer(const ASTTimePeriod & time_period, const ASTTimeInterval * time_offset); - - std::chrono::sys_seconds next(std::chrono::system_clock::time_point tp) const; - - bool operator==(const RefreshEveryTimer & rhs) const; - -private: - std::chrono::sys_seconds alignedToYears(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToMonths(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToWeeks(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToDays(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToHours(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToMinutes(std::chrono::system_clock::time_point tp) const; - - std::chrono::sys_seconds alignedToSeconds(std::chrono::system_clock::time_point tp) const; - - RefreshAfterTimer offset; - UInt32 value{0}; - IntervalKind kind{IntervalKind::Second}; -}; - -struct RefreshTimer -{ - std::variant timer; - - explicit RefreshTimer(const ASTRefreshStrategy & strategy); - - std::chrono::sys_seconds next(std::chrono::system_clock::time_point tp) const; - - bool operator==(const RefreshTimer & rhs) const; - - const RefreshAfterTimer * tryGetAfter() const; - const RefreshEveryTimer * tryGetEvery() const; -}; - -} From ef4cc5ec7fe5b690acfaa453f117cc50800e4cc2 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 29 Nov 2023 02:32:41 +0000 Subject: [PATCH 12/25] Things --- src/Access/Common/AccessType.h | 3 +- src/Common/CalendarTimeInterval.cpp | 4 +- src/Common/CalendarTimeInterval.h | 4 +- src/Core/BackgroundSchedulePool.cpp | 4 +- src/Core/BackgroundSchedulePool.h | 6 +- src/Core/Settings.h | 1 + src/Interpreters/AddDefaultDatabaseVisitor.h | 5 +- src/Interpreters/InterpreterAlterQuery.cpp | 5 + src/Interpreters/InterpreterCreateQuery.cpp | 8 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 + src/Parsers/ASTAlterQuery.cpp | 6 + src/Parsers/ASTAlterQuery.h | 4 + src/Parsers/ASTSystemQuery.h | 5 + src/Parsers/ParserAlterQuery.cpp | 9 + src/Parsers/ParserSystemQuery.cpp | 24 ++ src/Storages/AlterCommands.cpp | 14 +- src/Storages/AlterCommands.h | 4 + .../MaterializedView/RefreshSchedule.cpp | 6 + .../MaterializedView/RefreshSchedule.h | 1 + src/Storages/MaterializedView/RefreshSet.cpp | 148 ++++++------ src/Storages/MaterializedView/RefreshSet.h | 110 ++++----- src/Storages/MaterializedView/RefreshTask.cpp | 219 ++++++++++++------ src/Storages/MaterializedView/RefreshTask.h | 68 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 3 + src/Storages/SelectQueryDescription.cpp | 13 +- src/Storages/SelectQueryDescription.h | 4 +- src/Storages/StorageInMemoryMetadata.cpp | 7 + src/Storages/StorageInMemoryMetadata.h | 6 + src/Storages/StorageMaterializedView.cpp | 26 ++- .../System/StorageSystemViewRefreshes.cpp | 54 +++-- .../02661_refreshable_materialized_views.sql | 25 ++ 31 files changed, 490 insertions(+), 310 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 0188cbb5b99..307dc8aa1cc 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -82,7 +82,8 @@ enum class AccessType \ M(ALTER_VIEW_REFRESH, "ALTER LIVE VIEW REFRESH, REFRESH VIEW", VIEW, ALTER_VIEW) \ M(ALTER_VIEW_MODIFY_QUERY, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \ - M(ALTER_VIEW, "", GROUP, ALTER) /* allows to execute ALTER VIEW REFRESH, ALTER VIEW MODIFY QUERY; + M(ALTER_VIEW_MODIFY_REFRESH, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \ + M(ALTER_VIEW, "", GROUP, ALTER) /* allows to execute ALTER VIEW REFRESH, ALTER VIEW MODIFY QUERY, ALTER VIEW MODIFY REFRESH; implicitly enabled by the grant ALTER_TABLE */\ \ M(ALTER, "", GROUP, ALL) /* allows to execute ALTER {TABLE|LIVE VIEW} */\ diff --git a/src/Common/CalendarTimeInterval.cpp b/src/Common/CalendarTimeInterval.cpp index bcedf63b3ff..de1ef76f8ab 100644 --- a/src/Common/CalendarTimeInterval.cpp +++ b/src/Common/CalendarTimeInterval.cpp @@ -64,12 +64,12 @@ CalendarTimeInterval::Intervals CalendarTimeInterval::toIntervals() const UInt64 CalendarTimeInterval::minSeconds() const { - return 3600*24 * (365 * months/12 + 28 * months%12) + seconds; + return 3600*24 * (months/12 * 365 + months%12 * 28) + seconds; } UInt64 CalendarTimeInterval::maxSeconds() const { - return 3600*24 * (366 * months/12 + 31 * months%12) + seconds; + return 3600*24 * (months/12 * 366 + months%12 * 31) + seconds; } void CalendarTimeInterval::assertSingleUnit() const diff --git a/src/Common/CalendarTimeInterval.h b/src/Common/CalendarTimeInterval.h index 40a390736bc..c68449de0b6 100644 --- a/src/Common/CalendarTimeInterval.h +++ b/src/Common/CalendarTimeInterval.h @@ -40,8 +40,6 @@ struct CalendarTimeInterval /// Add this interval to the timestamp. First months, then seconds. /// Gets weird near month boundaries: October 31 + 1 month = December 1. - /// Gets weird with leap years: 2004-03-15 + 1 year = 2005-03-16, - /// 2004-12-31 + 1 year = 2006-01-01, std::chrono::sys_seconds advance(std::chrono::system_clock::time_point t) const; /// Rounds the timestamp down to the nearest timestamp "aligned" with this interval. @@ -51,7 +49,7 @@ struct CalendarTimeInterval /// E.g. if the interval is 1 month, rounds down to the start of the month. /// * For seconds, rounds to a timestamp x such that (x - December 29 1969 (Monday)) is divisible /// by this interval. - /// E.g. if the interval is 1 week, rounds down to the start of the week. + /// E.g. if the interval is 1 week, rounds down to the start of the week (Monday). /// /// Guarantees: /// * advance(floor(x)) > x diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index ec1ae047d05..fa892bc3c84 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -31,7 +31,7 @@ bool BackgroundSchedulePoolTaskInfo::schedule() return true; } -bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool overwrite) +bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool overwrite, bool only_if_scheduled) { std::lock_guard lock(schedule_mutex); @@ -39,6 +39,8 @@ bool BackgroundSchedulePoolTaskInfo::scheduleAfter(size_t milliseconds, bool ove return false; if (delayed && !overwrite) return false; + if (!delayed && only_if_scheduled) + return false; pool.scheduleDelayedTask(shared_from_this(), milliseconds, lock); return true; diff --git a/src/Core/BackgroundSchedulePool.h b/src/Core/BackgroundSchedulePool.h index e97b02e976f..eca93353283 100644 --- a/src/Core/BackgroundSchedulePool.h +++ b/src/Core/BackgroundSchedulePool.h @@ -106,8 +106,10 @@ public: bool schedule(); /// Schedule for execution after specified delay. - /// If overwrite is set then the task will be re-scheduled (if it was already scheduled, i.e. delayed == true). - bool scheduleAfter(size_t milliseconds, bool overwrite = true); + /// If overwrite is set, and the task is already scheduled with a delay (delayed == true), + /// the task will be re-scheduled with the new delay. + /// If only_if_scheduled is set, don't do anything unless the task is already scheduled with a delay. + bool scheduleAfter(size_t milliseconds, bool overwrite = true, bool only_if_scheduled = false); /// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task. void deactivate(); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d96b1b9fc10..7b84cfa6f5d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -584,6 +584,7 @@ class IColumn; M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 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, 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) \ 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/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index e6354467938..b977a73d461 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -238,8 +238,9 @@ private: void visit(ASTRefreshStrategy & refresh, ASTPtr &) const { - for (auto & table : refresh.children) - tryVisit(table); + if (refresh.dependencies) + for (auto & table : refresh.dependencies->children) + tryVisit(table); } void visitChildren(IAST & ast) const diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index db93467c0a4..2a34932d950 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -460,6 +460,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_QUERY, database, table); break; } + case ASTAlterCommand::MODIFY_REFRESH: + { + required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_REFRESH, database, table); + break; + } case ASTAlterCommand::LIVE_VIEW_REFRESH: { required_access.emplace_back(AccessType::ALTER_VIEW_REFRESH, database, table); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 801a46f4167..823f04e0580 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1089,6 +1089,11 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data "{} UUID specified, but engine of database {} is not Atomic", kind, create.getDatabase()); } + if (create.refresh_strategy && !internal) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Refreshable materialized view requires Atomic database engine"); + /// ... because it needs to atomically replace the inner table after refresh + /// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either /// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or /// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts. @@ -1212,11 +1217,8 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.refresh_strategy) { - /// TODO: This doesn't work for some reason. AddDefaultDatabaseVisitor visitor(getContext(), current_database); visitor.visit(*create.refresh_strategy); - - /// TODO: For DEPENDS ON, check that the specified tables exist. } if (create.columns_list) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 7df6499fd39..6998a6ef978 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -629,6 +629,9 @@ BlockIO InterpreterSystemQuery::execute() case Type::RESUME_VIEW: getRefreshTask()->resume(); break; + case Type::TEST_VIEW: + getRefreshTask()->setFakeTime(query.fake_time_for_view); + break; case Type::DROP_REPLICA: dropReplica(query); break; @@ -1284,6 +1287,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::CANCEL_VIEW: case Type::PAUSE_VIEW: case Type::RESUME_VIEW: + case Type::TEST_VIEW: { if (!query.table) required_access.emplace_back(AccessType::SYSTEM_VIEWS); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index ed9de6a46eb..84355817b2c 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -453,6 +453,12 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::MODIFY_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY REFRESH " << settings.nl_or_ws + << (settings.hilite ? hilite_none : ""); + refresh->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::LIVE_VIEW_REFRESH) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "REFRESH " << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 77c540aed33..0b115537a6d 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -40,6 +40,7 @@ public: MODIFY_SETTING, RESET_SETTING, MODIFY_QUERY, + MODIFY_REFRESH, REMOVE_TTL, REMOVE_SAMPLE_BY, @@ -166,6 +167,9 @@ public: */ ASTPtr values; + /// For MODIFY REFRESH + ASTPtr refresh; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART, DROP DETACHED PART and MOVE diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index ec8e47f9513..fa849800ba6 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -98,6 +98,7 @@ public: CANCEL_VIEW, PAUSE_VIEW, RESUME_VIEW, + TEST_VIEW, END }; @@ -141,6 +142,10 @@ public: ServerType server_type; + /// For SYSTEM TEST VIEW (SET FAKE TIME