mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Simple review comments
This commit is contained in:
parent
808cb0fa05
commit
5dc04a13a7
@ -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)
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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 : "");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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 };
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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>
|
||||
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user