Simple review comments

This commit is contained in:
Michael Kolupaev 2023-11-18 00:45:05 +00:00
parent 808cb0fa05
commit 5dc04a13a7
9 changed files with 32 additions and 21 deletions

View File

@ -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)

View File

@ -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);
}
}

View File

@ -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 : "");
}
}

View File

@ -9,6 +9,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
namespace
{
@ -27,7 +32,10 @@ std::optional<ValKind> parseValKind(IParser::Pos & pos, Expected & expected)
return ValKind{ .empty = true };
if (!parseIntervalKind(pos, expected, kind))
return {};
return ValKind{ value->as<ASTLiteral &>().value.safeGet<UInt64>(), kind, false };
UInt64 val;
if (!value->as<ASTLiteral &>().value.tryGet(val))
throw Exception(ErrorCodes::SYNTAX_ERROR, "Time interval must be an integer");
return ValKind{ val, kind, false };
}
}

View File

@ -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

View File

@ -37,7 +37,7 @@ std::variant<RefreshEveryTimer, RefreshAfterTimer> 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)

View File

@ -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<std::chrono::minutes>(tp);
auto tp_seconds = std::chrono::floor<std::chrono::seconds>(tp - tp_minutes);
auto next_seconds= (tp_seconds / value + 1s) * value;
auto next_seconds = (tp_seconds / value + 1s) * value;
if (std::chrono::floor<std::chrono::minutes>(next_seconds - 1s) > 0min)
return tp_minutes + 1min + std::chrono::seconds{value};
return tp_minutes + next_seconds;

View File

@ -1,6 +1,9 @@
#include <Storages/System/StorageSystemViewRefreshes.h>
#include <Access/ContextAccess.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Storages/MaterializedView/RefreshSet.h>

View File

@ -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