Merge remote-tracking branch 'origin/master' into remove-old-code-5

This commit is contained in:
Alexey Milovidov 2024-03-04 00:12:03 +01:00
commit da2481c5c8
7 changed files with 133 additions and 9 deletions

View File

@ -582,11 +582,18 @@ ReturnType parseDateTimeBestEffortImpl(
day_of_month = 1;
if (!month)
month = 1;
if (!year)
{
/// If year is not specified, it will be the current year if the date is unknown or not greater than today,
/// otherwise it will be the previous year.
/// This convoluted logic is needed to parse the syslog format, which looks as follows: "Mar 3 01:33:48".
/// If you have questions, ask Victor Krasnov, https://www.linkedin.com/in/vickr/
time_t now = time(nullptr);
UInt16 curr_year = local_time_zone.toYear(now);
year = now < local_time_zone.makeDateTime(curr_year, month, day_of_month, hour, minute, second) ? curr_year - 1 : curr_year;
auto today = local_time_zone.toDayNum(now);
UInt16 curr_year = local_time_zone.toYear(today);
year = local_time_zone.makeDayNum(curr_year, month, day_of_month) <= today ? curr_year : curr_year - 1;
}
auto is_leap_year = (year % 400 == 0) || (year % 100 != 0 && year % 4 == 0);

View File

@ -1209,7 +1209,12 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
break;
}
case Type::DROP_DISK_METADATA_CACHE:
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Not implemented");
case Type::DROP_DISTRIBUTED_CACHE:
case Type::STOP_VIRTUAL_PARTS_UPDATE:
case Type::START_VIRTUAL_PARTS_UPDATE:
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only available in ClickHouse Cloud, https://clickhouse.cloud/");
}
case Type::RELOAD_DICTIONARY:
case Type::RELOAD_DICTIONARIES:
case Type::RELOAD_EMBEDDED_DICTIONARIES:

View File

@ -172,6 +172,8 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s
case Type::START_PULLING_REPLICATION_LOG:
case Type::STOP_CLEANUP:
case Type::START_CLEANUP:
case Type::START_VIRTUAL_PARTS_UPDATE:
case Type::STOP_VIRTUAL_PARTS_UPDATE:
{
if (table)
{
@ -294,6 +296,12 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState & s
}
break;
}
case Type::DROP_DISTRIBUTED_CACHE:
{
if (!distributed_cache_servive_id.empty())
settings.ostr << (settings.hilite ? hilite_none : "") << " " << distributed_cache_servive_id;
break;
}
case Type::UNFREEZE:
{
print_keyword(" WITH NAME ");

View File

@ -30,6 +30,7 @@ public:
DROP_QUERY_CACHE,
DROP_COMPILED_EXPRESSION_CACHE,
DROP_FILESYSTEM_CACHE,
DROP_DISTRIBUTED_CACHE,
DROP_DISK_METADATA_CACHE,
DROP_SCHEMA_CACHE,
DROP_FORMAT_SCHEMA_CACHE,
@ -98,6 +99,8 @@ public:
STOP_VIEWS,
CANCEL_VIEW,
TEST_VIEW,
STOP_VIRTUAL_PARTS_UPDATE,
START_VIRTUAL_PARTS_UPDATE,
END
};
@ -126,6 +129,8 @@ public:
UInt64 seconds{};
String filesystem_cache_name;
String distributed_cache_servive_id;
std::string key_to_drop;
std::optional<size_t> offset_to_drop;

View File

@ -14,11 +14,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
[[nodiscard]] static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
Expected & expected, bool require_table, bool allow_string_literal)
{
@ -397,6 +392,8 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::START_PULLING_REPLICATION_LOG:
case Type::STOP_CLEANUP:
case Type::START_CLEANUP:
case Type::STOP_VIRTUAL_PARTS_UPDATE:
case Type::START_VIRTUAL_PARTS_UPDATE:
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
@ -470,6 +467,15 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
return false;
break;
}
case Type::DROP_DISTRIBUTED_CACHE:
{
ParserLiteral parser;
ASTPtr ast;
if (!parser.parse(pos, ast, expected))
return false;
res->distributed_cache_servive_id = ast->as<ASTLiteral>()->value.safeGet<String>();
break;
}
case Type::SYNC_FILESYSTEM_CACHE:
{
ParserLiteral path_parser;
@ -482,7 +488,9 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
}
case Type::DROP_DISK_METADATA_CACHE:
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Not implemented");
if (!parseQueryWithOnClusterAndTarget(res, pos, expected, SystemQueryTargetType::Disk))
return false;
break;
}
case Type::DROP_SCHEMA_CACHE:
{

View File

@ -0,0 +1,37 @@
The reference time point is 2023-06-30 23:59:30
───────────────────────────────────────────────
The argument is before the reference time point
───────────────────────────────────────────────
Row 1:
──────
syslog_arg: Jun 29 23:59:30
res: 2023-06-29 23:59:30
res_null: 2023-06-29 23:59:30
res_zero: 2023-06-29 23:59:30
res_us: 2023-06-29 23:59:30
res_us_null: 2023-06-29 23:59:30
res_us_zero: 2023-06-29 23:59:30
res64: 2023-06-29 23:59:30.000
res64_null: 2023-06-29 23:59:30.000
res64_zero: 2023-06-29 23:59:30.000
res64_us: 2023-06-29 23:59:30.000
res64_us_null: 2023-06-29 23:59:30.000
res64_us_zero: 2023-06-29 23:59:30.000
──────────────────────────────────────────────
The argument is after the reference time point
──────────────────────────────────────────────
Row 1:
──────
syslog_arg: Jul 1 23:59:30
res: 2022-06-30 23:59:30
res_null: 2022-06-30 23:59:30
res_zero: 2022-06-30 23:59:30
res_us: 2022-06-30 23:59:30
res_us_null: 2022-06-30 23:59:30
res_us_zero: 2022-06-30 23:59:30
res64: 2022-06-30 23:59:30.000
res64_null: 2022-06-30 23:59:30.000
res64_zero: 2022-06-30 23:59:30.000
res64_us: 2022-06-30 23:59:30.000
res64_us_null: 2022-06-30 23:59:30.000
res64_us_zero: 2022-06-30 23:59:30.000

View File

@ -0,0 +1,54 @@
SET session_timezone = 'UTC';
SELECT 'The reference time point is 2023-06-30 23:59:30';
SELECT '───────────────────────────────────────────────';
SELECT 'The argument is before the reference time point';
SELECT '───────────────────────────────────────────────';
WITH
toDateTime('2023-06-30 23:59:30') AS dt_ref,
now() AS dt_now,
date_sub(DAY, 1, dt_now) as dt_before,
dateDiff(SECOND, dt_ref, dt_now) AS time_shift,
formatDateTime(dt_before, '%b %e %T') AS syslog_before
SELECT
formatDateTime(dt_before - time_shift, '%b %e %T') AS syslog_arg,
parseDateTimeBestEffort(syslog_before) - time_shift AS res,
parseDateTimeBestEffortOrNull(syslog_before) - time_shift AS res_null,
parseDateTimeBestEffortOrZero(syslog_before) - time_shift AS res_zero,
parseDateTimeBestEffortUS(syslog_before) - time_shift AS res_us,
parseDateTimeBestEffortUSOrNull(syslog_before) - time_shift AS res_us_null,
parseDateTimeBestEffortUSOrZero(syslog_before) - time_shift AS res_us_zero,
parseDateTime64BestEffort(syslog_before) - time_shift AS res64,
parseDateTime64BestEffortOrNull(syslog_before) - time_shift AS res64_null,
parseDateTime64BestEffortOrZero(syslog_before) - time_shift AS res64_zero,
parseDateTime64BestEffortUS(syslog_before) - time_shift AS res64_us,
parseDateTime64BestEffortUSOrNull(syslog_before) - time_shift AS res64_us_null,
parseDateTime64BestEffortUSOrZero(syslog_before) - time_shift AS res64_us_zero
FORMAT Vertical;
SELECT '──────────────────────────────────────────────';
SELECT 'The argument is after the reference time point';
SELECT '──────────────────────────────────────────────';
WITH
toDateTime('2023-06-30 23:59:30') AS dt_ref,
now() AS dt_now,
date_add(DAY, 1, dt_now) as dt_after,
dateDiff(SECOND, dt_ref, dt_now) AS time_shift,
formatDateTime(dt_after, '%b %e %T') AS syslog_after
SELECT
formatDateTime(dt_after - time_shift, '%b %e %T') AS syslog_arg,
parseDateTimeBestEffort(syslog_after) - time_shift AS res,
parseDateTimeBestEffortOrNull(syslog_after) - time_shift AS res_null,
parseDateTimeBestEffortOrZero(syslog_after) - time_shift AS res_zero,
parseDateTimeBestEffortUS(syslog_after) - time_shift AS res_us,
parseDateTimeBestEffortUSOrNull(syslog_after) - time_shift AS res_us_null,
parseDateTimeBestEffortUSOrZero(syslog_after) - time_shift AS res_us_zero,
parseDateTime64BestEffort(syslog_after) - time_shift AS res64,
parseDateTime64BestEffortOrNull(syslog_after) - time_shift AS res64_null,
parseDateTime64BestEffortOrZero(syslog_after) - time_shift AS res64_zero,
parseDateTime64BestEffortUS(syslog_after) - time_shift AS res64_us,
parseDateTime64BestEffortUSOrNull(syslog_after) - time_shift AS res64_us_null,
parseDateTime64BestEffortUSOrZero(syslog_after) - time_shift AS res64_us_zero
FORMAT Vertical;