Merge branch 'master' into sanity-checks-message-fix

This commit is contained in:
mergify[bot] 2022-04-26 05:45:11 +00:00 committed by GitHub
commit 74aa65e36e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
63 changed files with 442 additions and 538 deletions

View File

@ -1,24 +1,8 @@
# For most projects, this workflow file will not need changing; you simply need
# to commit it to your repository.
#
# You may wish to alter this file to override the set of languages analyzed,
# or to provide custom queries or build logic.
#
# ******** NOTE ********
# We have attempted to detect the languages in your repository. Please check
# the `language` matrix defined below to confirm you have the correct set of
# supported CodeQL languages.
#
name: "CodeQL" name: "CodeQL"
"on": "on":
# push:
# branches: [ master ]
# pull_request:
# # The branches below must be a subset of the branches above
# branches: [ master ]
schedule: schedule:
- cron: '0 */6 * * *' - cron: '0 0 * * *'
workflow_dispatch: workflow_dispatch:
env: env:
@ -38,8 +22,6 @@ jobs:
fail-fast: false fail-fast: false
matrix: matrix:
language: ['cpp'] language: ['cpp']
# CodeQL supports [ 'cpp', 'csharp', 'go', 'java', 'javascript', 'python', 'ruby' ]
# Learn more about CodeQL language support at https://git.io/codeql-language-support
steps: steps:
- name: Checkout repository - name: Checkout repository
@ -47,27 +29,14 @@ jobs:
with: with:
submodules: 'true' submodules: 'true'
# Initializes the CodeQL tools for scanning.
- name: Initialize CodeQL - name: Initialize CodeQL
uses: github/codeql-action/init@v2 uses: github/codeql-action/init@v2
with: with:
languages: ${{ matrix.language }} languages: ${{ matrix.language }}
# If you wish to specify custom queries, you can do so here or in a config file.
# By default, queries listed here will override any specified in a config file.
# Prefix the list here with "+" to use these queries and those in the config file.
# queries: ./path/to/local/query, your-org/your-repo/queries@main
# Autobuild attempts to build any compiled languages (C/C++, C#, or Java).
# If this step fails, then you should remove it and run the build manually (see below)
# - name: Autobuild
# uses: github/codeql-action/autobuild@v2
# Command-line programs to run using the OS shell.
# 📚 https://git.io/JvXDl
- name: Build - name: Build
run: | run: |
sudo apt-get install -yq git cmake python ninja-build sudo apt-get install -yq ninja-build
sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
mkdir build mkdir build
cd build cd build

View File

@ -61,7 +61,7 @@ else ()
endif () endif ()
if (ARCH_PPC64LE) if (ARCH_PPC64LE)
set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS") set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -mcpu=power8 -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS")
endif () endif ()
set (TEST_FLAG "-msse4.2") set (TEST_FLAG "-msse4.2")

View File

@ -5,6 +5,8 @@ sidebar_label: Boolean
# Boolean Values {#boolean-values} # Boolean Values {#boolean-values}
There is no separate type for boolean values. Use UInt8 type, restricted to the values 0 or 1. Since https://github.com/ClickHouse/ClickHouse/commit/4076ae77b46794e73594a9f400200088ed1e7a6e , there be a separate type for boolean values.
For versions before that, there is no separate type for boolean values. Use UInt8 type, restricted to the values 0 or 1.
[Original article](https://clickhouse.com/docs/en/data_types/boolean/) <!--hide--> [Original article](https://clickhouse.com/docs/en/data_types/boolean/) <!--hide-->

View File

@ -375,7 +375,7 @@ Result:
- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter. - [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter.
## toStartOfFiveMinute {#tostartoffiveminute} ## toStartOfFiveMinutes {#tostartoffiveminutes}
Rounds down a date with time to the start of the five-minute interval. Rounds down a date with time to the start of the five-minute interval.

View File

@ -384,7 +384,7 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d
- Часовая зона сервера, конфигурационный параметр [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). - Часовая зона сервера, конфигурационный параметр [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone).
## toStartOfFiveMinute {#tostartoffiveminute} ## toStartOfFiveMinutes {#tostartoffiveminutes}
Округляет дату-с-временем вниз до начала пятиминутного интервала. Округляет дату-с-временем вниз до начала пятиминутного интервала.

View File

@ -1,3 +1,5 @@
# 布尔值 {#boolean-values} # 布尔值 {#boolean-values}
没有单独的类型来存储布尔值。可以使用 UInt8 类型,取值限制为 0 或 1。 从 https://github.com/ClickHouse/ClickHouse/commit/4076ae77b46794e73594a9f400200088ed1e7a6e 之后,有单独的类型来存储布尔值。
在此之前的版本,没有单独的类型来存储布尔值。可以使用 UInt8 类型,取值限制为 0 或 1。

View File

@ -227,7 +227,7 @@ SELECT toStartOfSecond(dt64, 'Asia/Istanbul');
- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 服务器配置选项。 - [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 服务器配置选项。
## toStartOfFiveMinute {#tostartoffiveminute} ## toStartOfFiveMinutes {#tostartoffiveminutes}
将DateTime以五分钟为单位向前取整到最接近的时间点。 将DateTime以五分钟为单位向前取整到最接近的时间点。

View File

@ -18,9 +18,9 @@ conflicts:
maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>" maintainer: "ClickHouse Dev Team <packages+linux@clickhouse.com>"
description: | description: |
Client binary for ClickHouse Client binary for ClickHouse
ClickHouse is a column-oriented database management system ClickHouse is a column-oriented database management system.
that allows generating analytical data reports in real time. that allows generating analytical data reports in real time.
This package provides clickhouse-client , clickhouse-local and clickhouse-benchmark This package provides clickhouse-client, clickhouse-local and clickhouse-benchmark.
overrides: overrides:
deb: deb:

View File

@ -15,6 +15,17 @@ shopt -s extglob
export _CLICKHOUSE_COMPLETION_LOADED=1 export _CLICKHOUSE_COMPLETION_LOADED=1
CLICKHOUSE_logs_level=(
none
fatal
error
warning
information
debug
trace
test
)
CLICKHOUSE_QueryProcessingStage=( CLICKHOUSE_QueryProcessingStage=(
complete complete
fetch_columns fetch_columns
@ -113,6 +124,10 @@ function _complete_for_clickhouse_generic_bin_impl()
COMPREPLY=( $(compgen -W "${CLICKHOUSE_QueryProcessingStage[*]}" -- "$cur") ) COMPREPLY=( $(compgen -W "${CLICKHOUSE_QueryProcessingStage[*]}" -- "$cur") )
return 1 return 1
;; ;;
--send_logs_level)
COMPREPLY=( $(compgen -W "${CLICKHOUSE_logs_level[*]}" -- "$cur") )
return 1
;;
--format|--input-format|--output-format) --format|--input-format|--output-format)
COMPREPLY=( $(compgen -W "${CLICKHOUSE_Format[*]}" -- "$cur") ) COMPREPLY=( $(compgen -W "${CLICKHOUSE_Format[*]}" -- "$cur") )
return 1 return 1

View File

@ -1508,7 +1508,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
} }
#if defined(OS_LINUX) #if defined(OS_LINUX)
if (!TasksStatsCounters::checkIfAvailable()) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider();
if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None)
{ {
LOG_INFO(log, "It looks like this system does not have procfs mounted at /proc location," LOG_INFO(log, "It looks like this system does not have procfs mounted at /proc location,"
" neither clickhouse-server process has CAP_NET_ADMIN capability." " neither clickhouse-server process has CAP_NET_ADMIN capability."
@ -1519,6 +1520,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.", " It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.",
executable_path); executable_path);
} }
else
{
LOG_INFO(log, "Tasks stats provider: {}", TasksStatsCounters::metricsProviderString(tasks_stats_provider));
}
if (!hasLinuxCapability(CAP_SYS_NICE)) if (!hasLinuxCapability(CAP_SYS_NICE))
{ {

View File

@ -1234,6 +1234,7 @@ void ClientBase::sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDes
} }
void ClientBase::sendDataFromPipe(Pipe&& pipe, ASTPtr parsed_query, bool have_more_data) void ClientBase::sendDataFromPipe(Pipe&& pipe, ASTPtr parsed_query, bool have_more_data)
try
{ {
QueryPipeline pipeline(std::move(pipe)); QueryPipeline pipeline(std::move(pipe));
PullingAsyncPipelineExecutor executor(pipeline); PullingAsyncPipelineExecutor executor(pipeline);
@ -1266,6 +1267,12 @@ void ClientBase::sendDataFromPipe(Pipe&& pipe, ASTPtr parsed_query, bool have_mo
if (!have_more_data) if (!have_more_data)
connection->sendData({}, "", false); connection->sendData({}, "", false);
} }
catch (...)
{
connection->sendCancel();
receiveEndOfQuery();
throw;
}
void ClientBase::sendDataFromStdin(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query) void ClientBase::sendDataFromStdin(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query)
{ {
@ -1406,7 +1413,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
apply_query_settings(*with_output->settings_ast); apply_query_settings(*with_output->settings_ast);
if (!connection->checkConnected()) if (!connection->checkConnected())
{
auto poco_logs_level = Poco::Logger::parseLevel(config().getString("send_logs_level", "none"));
/// Print under WARNING also because it is used by clickhouse-test.
if (poco_logs_level >= Poco::Message::PRIO_WARNING)
{
fmt::print(stderr, "Connection lost. Reconnecting.\n");
}
connect(); connect();
}
ASTPtr input_function; ASTPtr input_function;
if (insert && insert->select) if (insert && insert->select)

View File

@ -377,9 +377,10 @@ bool Connection::ping()
{ {
// LOG_TRACE(log_wrapper.get(), "Ping"); // LOG_TRACE(log_wrapper.get(), "Ping");
TimeoutSetter timeout_setter(*socket, sync_request_timeout, true);
try try
{ {
TimeoutSetter timeout_setter(*socket, sync_request_timeout, true);
UInt64 pong = 0; UInt64 pong = 0;
writeVarUInt(Protocol::Client::Ping, *out); writeVarUInt(Protocol::Client::Ping, *out);
out->next(); out->next();
@ -405,6 +406,10 @@ bool Connection::ping()
} }
catch (const Poco::Exception & e) catch (const Poco::Exception & e)
{ {
/// Explicitly disconnect since ping() can receive EndOfStream,
/// and in this case this ping() will return false,
/// while next ping() may return true.
disconnect();
LOG_TRACE(log_wrapper.get(), fmt::runtime(e.displayText())); LOG_TRACE(log_wrapper.get(), fmt::runtime(e.displayText()));
return false; return false;
} }

View File

@ -565,7 +565,7 @@ public:
/// NOTE: Assuming timezone offset is a multiple of 15 minutes. /// NOTE: Assuming timezone offset is a multiple of 15 minutes.
inline Time toStartOfMinute(Time t) const { return toStartOfMinuteInterval(t, 1); } inline Time toStartOfMinute(Time t) const { return toStartOfMinuteInterval(t, 1); }
inline Time toStartOfFiveMinute(Time t) const { return toStartOfMinuteInterval(t, 5); } inline Time toStartOfFiveMinutes(Time t) const { return toStartOfMinuteInterval(t, 5); }
inline Time toStartOfFifteenMinutes(Time t) const { return toStartOfMinuteInterval(t, 15); } inline Time toStartOfFifteenMinutes(Time t) const { return toStartOfMinuteInterval(t, 15); }
inline Time toStartOfTenMinutes(Time t) const { return toStartOfMinuteInterval(t, 10); } inline Time toStartOfTenMinutes(Time t) const { return toStartOfMinuteInterval(t, 10); }
inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); } inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); }

View File

@ -296,11 +296,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
"All connection tries failed. Log: \n\n" + fail_messages + "\n", "All connection tries failed. Log: \n\n" + fail_messages + "\n",
DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED);
try_results.erase( std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; });
std::remove_if(
try_results.begin(), try_results.end(),
[](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }),
try_results.end());
/// Sort so that preferred items are near the beginning. /// Sort so that preferred items are near the beginning.
std::stable_sort( std::stable_sort(

View File

@ -265,26 +265,24 @@ void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const
{ {
NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid)); NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid));
for (const NetlinkMessage::Attribute * attr = &answer.payload.attribute; const NetlinkMessage::Attribute * attr = &answer.payload.attribute;
attr < answer.end(); if (attr->header.nla_type != TASKSTATS_TYPE_AGGR_PID)
attr = attr->next()) throw Exception("Expected TASKSTATS_TYPE_AGGR_PID", ErrorCodes::NETLINK_ERROR);
{
if (attr->header.nla_type == TASKSTATS_TYPE_AGGR_TGID || attr->header.nla_type == TASKSTATS_TYPE_AGGR_PID)
{
for (const NetlinkMessage::Attribute * nested_attr = reinterpret_cast<const NetlinkMessage::Attribute *>(attr->payload);
nested_attr < attr->next();
nested_attr = nested_attr->next())
{
if (nested_attr->header.nla_type == TASKSTATS_TYPE_STATS)
{
out_stats = unalignedLoad<::taskstats>(nested_attr->payload);
return;
}
}
}
}
throw Exception("There is no TASKSTATS_TYPE_STATS attribute in the Netlink response", ErrorCodes::NETLINK_ERROR); /// TASKSTATS_TYPE_AGGR_PID
const NetlinkMessage::Attribute * nested_attr = reinterpret_cast<const NetlinkMessage::Attribute *>(attr->payload);
if (nested_attr->header.nla_type != TASKSTATS_TYPE_PID)
throw Exception("Expected TASKSTATS_TYPE_PID", ErrorCodes::NETLINK_ERROR);
if (nested_attr == nested_attr->next())
throw Exception("No TASKSTATS_TYPE_STATS packet after TASKSTATS_TYPE_PID", ErrorCodes::NETLINK_ERROR);
nested_attr = nested_attr->next();
if (nested_attr->header.nla_type != TASKSTATS_TYPE_STATS)
throw Exception("Expected TASKSTATS_TYPE_STATS", ErrorCodes::NETLINK_ERROR);
out_stats = unalignedLoad<::taskstats>(nested_attr->payload);
if (attr->next() != answer.end())
throw Exception("Unexpected end of response", ErrorCodes::NETLINK_ERROR);
} }

View File

@ -10,6 +10,11 @@ namespace DB
{ {
/// Get taskstat info from OS kernel via Netlink protocol. /// Get taskstat info from OS kernel via Netlink protocol.
///
/// NOTE: unlike procfs interface, netlink interface, rounds some values to KiBs [1].
///
/// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101
///
class TaskStatsInfoGetter : private boost::noncopyable class TaskStatsInfoGetter : private boost::noncopyable
{ {
public: public:

View File

@ -67,6 +67,20 @@ namespace ProfileEvents
namespace DB namespace DB
{ {
const char * TasksStatsCounters::metricsProviderString(MetricsProvider provider)
{
switch (provider)
{
case MetricsProvider::None:
return "none";
case MetricsProvider::Procfs:
return "procfs";
case MetricsProvider::Netlink:
return "netlink";
}
__builtin_unreachable();
}
bool TasksStatsCounters::checkIfAvailable() bool TasksStatsCounters::checkIfAvailable()
{ {
return findBestAvailableProvider() != MetricsProvider::None; return findBestAvailableProvider() != MetricsProvider::None;

View File

@ -176,7 +176,17 @@ extern PerfEventsCounters current_thread_counters;
class TasksStatsCounters class TasksStatsCounters
{ {
public: public:
enum class MetricsProvider
{
None,
Procfs,
Netlink,
};
static const char * metricsProviderString(MetricsProvider provider);
static bool checkIfAvailable(); static bool checkIfAvailable();
static MetricsProvider findBestAvailableProvider();
static std::unique_ptr<TasksStatsCounters> create(UInt64 tid); static std::unique_ptr<TasksStatsCounters> create(UInt64 tid);
void reset(); void reset();
@ -186,16 +196,8 @@ private:
::taskstats stats; //-V730_NOINIT ::taskstats stats; //-V730_NOINIT
std::function<::taskstats()> stats_getter; std::function<::taskstats()> stats_getter;
enum class MetricsProvider
{
None,
Procfs,
Netlink
};
explicit TasksStatsCounters(UInt64 tid, MetricsProvider provider); explicit TasksStatsCounters(UInt64 tid, MetricsProvider provider);
static MetricsProvider findBestAvailableProvider();
static void incrementProfileEvents(const ::taskstats & prev, const ::taskstats & curr, ProfileEvents::Counters & profile_events); static void incrementProfileEvents(const ::taskstats & prev, const ::taskstats & curr, ProfileEvents::Counters & profile_events);
}; };

View File

@ -59,23 +59,20 @@ namespace Format
{ {
size_t i = 0; size_t i = 0;
bool should_delete = true; bool should_delete = true;
str.erase( std::erase_if(
std::remove_if( str,
str.begin(), [&i, &should_delete, &str](char)
str.end(), {
[&i, &should_delete, &str](char) bool is_double_brace = (str[i] == '{' && str[i + 1] == '{') || (str[i] == '}' && str[i + 1] == '}');
++i;
if (is_double_brace && should_delete)
{ {
bool is_double_brace = (str[i] == '{' && str[i + 1] == '{') || (str[i] == '}' && str[i + 1] == '}'); should_delete = false;
++i; return true;
if (is_double_brace && should_delete) }
{ should_delete = true;
should_delete = false; return false;
return true; });
}
should_delete = true;
return false;
}),
str.end());
}; };
index_positions.emplace_back(); index_positions.emplace_back();

View File

@ -119,7 +119,7 @@ TEST(DateLUTTest, TimeValuesInMiddleOfRange)
EXPECT_EQ(lut.toSecond(time), 11 /*unsigned*/); EXPECT_EQ(lut.toSecond(time), 11 /*unsigned*/);
EXPECT_EQ(lut.toMinute(time), 20 /*unsigned*/); EXPECT_EQ(lut.toMinute(time), 20 /*unsigned*/);
EXPECT_EQ(lut.toStartOfMinute(time), 1568650800 /*time_t*/); EXPECT_EQ(lut.toStartOfMinute(time), 1568650800 /*time_t*/);
EXPECT_EQ(lut.toStartOfFiveMinute(time), 1568650800 /*time_t*/); EXPECT_EQ(lut.toStartOfFiveMinutes(time), 1568650800 /*time_t*/);
EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 1568650500 /*time_t*/); EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 1568650500 /*time_t*/);
EXPECT_EQ(lut.toStartOfTenMinutes(time), 1568650800 /*time_t*/); EXPECT_EQ(lut.toStartOfTenMinutes(time), 1568650800 /*time_t*/);
EXPECT_EQ(lut.toStartOfHour(time), 1568649600 /*time_t*/); EXPECT_EQ(lut.toStartOfHour(time), 1568649600 /*time_t*/);
@ -181,7 +181,7 @@ TEST(DateLUTTest, TimeValuesAtLeftBoderOfRange)
EXPECT_EQ(lut.toSecond(time), 0 /*unsigned*/); EXPECT_EQ(lut.toSecond(time), 0 /*unsigned*/);
EXPECT_EQ(lut.toMinute(time), 0 /*unsigned*/); EXPECT_EQ(lut.toMinute(time), 0 /*unsigned*/);
EXPECT_EQ(lut.toStartOfMinute(time), 0 /*time_t*/); EXPECT_EQ(lut.toStartOfMinute(time), 0 /*time_t*/);
EXPECT_EQ(lut.toStartOfFiveMinute(time), 0 /*time_t*/); EXPECT_EQ(lut.toStartOfFiveMinutes(time), 0 /*time_t*/);
EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 0 /*time_t*/); EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 0 /*time_t*/);
EXPECT_EQ(lut.toStartOfTenMinutes(time), 0 /*time_t*/); EXPECT_EQ(lut.toStartOfTenMinutes(time), 0 /*time_t*/);
EXPECT_EQ(lut.toStartOfHour(time), 0 /*time_t*/); EXPECT_EQ(lut.toStartOfHour(time), 0 /*time_t*/);
@ -244,7 +244,7 @@ TEST(DateLUTTest, TimeValuesAtRightBoderOfRangeOfOldLUT)
EXPECT_EQ(lut.toMinute(time), 17 /*unsigned*/); EXPECT_EQ(lut.toMinute(time), 17 /*unsigned*/);
EXPECT_EQ(lut.toSecond(time), 53 /*unsigned*/); EXPECT_EQ(lut.toSecond(time), 53 /*unsigned*/);
EXPECT_EQ(lut.toStartOfMinute(time), 4294343820 /*time_t*/); EXPECT_EQ(lut.toStartOfMinute(time), 4294343820 /*time_t*/);
EXPECT_EQ(lut.toStartOfFiveMinute(time), 4294343700 /*time_t*/); EXPECT_EQ(lut.toStartOfFiveMinutes(time), 4294343700 /*time_t*/);
EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 4294343700 /*time_t*/); EXPECT_EQ(lut.toStartOfFifteenMinutes(time), 4294343700 /*time_t*/);
EXPECT_EQ(lut.toStartOfTenMinutes(time), 4294343400 /*time_t*/); EXPECT_EQ(lut.toStartOfTenMinutes(time), 4294343400 /*time_t*/);
EXPECT_EQ(lut.toStartOfHour(time), 4294342800 /*time_t*/); EXPECT_EQ(lut.toStartOfHour(time), 4294342800 /*time_t*/);

View File

@ -152,10 +152,7 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context
storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name);
/// Unset settings /// Unset settings
storage_children.erase( std::erase_if(storage_children, [&](const ASTPtr & element) { return element.get() == ast_storage->settings; });
std::remove_if(storage_children.begin(), storage_children.end(),
[&](const ASTPtr & element) { return element.get() == ast_storage->settings; }),
storage_children.end());
ast_storage->settings = nullptr; ast_storage->settings = nullptr;
} }
auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true,

View File

@ -219,10 +219,10 @@ private:
auto current_box = Box(Point(current_min_x, current_min_y), Point(current_max_x, current_max_y)); auto current_box = Box(Point(current_min_x, current_min_y), Point(current_max_x, current_max_y));
Polygon tmp_poly; Polygon tmp_poly;
bg::convert(current_box, tmp_poly); bg::convert(current_box, tmp_poly);
possible_ids.erase(std::remove_if(possible_ids.begin(), possible_ids.end(), [&](const auto id) std::erase_if(possible_ids, [&](const auto id)
{ {
return !bg::intersects(current_box, polygons[id]); return !bg::intersects(current_box, polygons[id]);
}), possible_ids.end()); });
int covered = 0; int covered = 0;
#ifndef __clang_analyzer__ /// Triggers a warning in boost geometry. #ifndef __clang_analyzer__ /// Triggers a warning in boost geometry.
auto it = std::find_if(possible_ids.begin(), possible_ids.end(), [&](const auto id) auto it = std::find_if(possible_ids.begin(), possible_ids.end(), [&](const auto id)

View File

@ -467,17 +467,17 @@ struct ToStartOfNanosecondImpl
using FactorTransform = ZeroTransform; using FactorTransform = ZeroTransform;
}; };
struct ToStartOfFiveMinuteImpl struct ToStartOfFiveMinutesImpl
{ {
static constexpr auto name = "toStartOfFiveMinute"; static constexpr auto name = "toStartOfFiveMinutes";
static inline UInt32 execute(const DecimalUtils::DecimalComponents<DateTime64> & t, const DateLUTImpl & time_zone) static inline UInt32 execute(const DecimalUtils::DecimalComponents<DateTime64> & t, const DateLUTImpl & time_zone)
{ {
return time_zone.toStartOfFiveMinute(t.whole); return time_zone.toStartOfFiveMinutes(t.whole);
} }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt32 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
return time_zone.toStartOfFiveMinute(t); return time_zone.toStartOfFiveMinutes(t);
} }
static inline UInt32 execute(Int32, const DateLUTImpl &) static inline UInt32 execute(Int32, const DateLUTImpl &)
{ {

View File

@ -27,7 +27,7 @@ void registerFunctionToLastDayOfMonth(FunctionFactory &);
void registerFunctionToStartOfQuarter(FunctionFactory &); void registerFunctionToStartOfQuarter(FunctionFactory &);
void registerFunctionToStartOfYear(FunctionFactory &); void registerFunctionToStartOfYear(FunctionFactory &);
void registerFunctionToStartOfMinute(FunctionFactory &); void registerFunctionToStartOfMinute(FunctionFactory &);
void registerFunctionToStartOfFiveMinute(FunctionFactory &); void registerFunctionToStartOfFiveMinutes(FunctionFactory &);
void registerFunctionToStartOfTenMinutes(FunctionFactory &); void registerFunctionToStartOfTenMinutes(FunctionFactory &);
void registerFunctionToStartOfFifteenMinutes(FunctionFactory &); void registerFunctionToStartOfFifteenMinutes(FunctionFactory &);
void registerFunctionToStartOfHour(FunctionFactory &); void registerFunctionToStartOfHour(FunctionFactory &);
@ -109,7 +109,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
registerFunctionToStartOfMillisecond(factory); registerFunctionToStartOfMillisecond(factory);
registerFunctionToStartOfSecond(factory); registerFunctionToStartOfSecond(factory);
registerFunctionToStartOfMinute(factory); registerFunctionToStartOfMinute(factory);
registerFunctionToStartOfFiveMinute(factory); registerFunctionToStartOfFiveMinutes(factory);
registerFunctionToStartOfTenMinutes(factory); registerFunctionToStartOfTenMinutes(factory);
registerFunctionToStartOfFifteenMinutes(factory); registerFunctionToStartOfFifteenMinutes(factory);
registerFunctionToStartOfHour(factory); registerFunctionToStartOfHour(factory);

View File

@ -1,18 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
namespace DB
{
using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinuteImpl>;
void registerFunctionToStartOfFiveMinute(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfFiveMinute>();
}
}

View File

@ -0,0 +1,19 @@
#include <Functions/FunctionFactory.h>
#include <Functions/DateTimeTransforms.h>
#include <Functions/FunctionDateOrDateTimeToSomething.h>
namespace DB
{
using FunctionToStartOfFiveMinutes = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfFiveMinutesImpl>;
void registerFunctionToStartOfFiveMinutes(FunctionFactory & factory)
{
factory.registerFunction<FunctionToStartOfFiveMinutes>();
factory.registerAlias("toStartOfFiveMinute", FunctionToStartOfFiveMinutes::name);
}
}

View File

@ -435,8 +435,7 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs, bool allow_consta
} }
nodes.remove_if([&](const Node & node) { return !visited_nodes.contains(&node); }); nodes.remove_if([&](const Node & node) { return !visited_nodes.contains(&node); });
auto it = std::remove_if(inputs.begin(), inputs.end(), [&](const Node * node) { return !visited_nodes.contains(node); }); std::erase_if(inputs, [&](const Node * node) { return !visited_nodes.contains(node); });
inputs.erase(it, inputs.end());
} }
static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * node, ColumnsWithTypeAndName arguments) static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * node, ColumnsWithTypeAndName arguments)

View File

@ -221,7 +221,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r
static void filterAndSortQueueNodes(Strings & all_nodes) static void filterAndSortQueueNodes(Strings & all_nodes)
{ {
all_nodes.erase(std::remove_if(all_nodes.begin(), all_nodes.end(), [] (const String & s) { return !startsWith(s, "query-"); }), all_nodes.end()); std::erase_if(all_nodes, [] (const String & s) { return !startsWith(s, "query-"); });
::sort(all_nodes.begin(), all_nodes.end()); ::sort(all_nodes.begin(), all_nodes.end());
} }

View File

@ -274,9 +274,9 @@ std::string buildTaggedRegex(std::string regexp_str)
std::vector<std::string> tags; std::vector<std::string> tags;
splitInto<';'>(tags, regexp_str); splitInto<';'>(tags, regexp_str);
/* remove empthy elements */ /* remove empty elements */
using namespace std::string_literals; using namespace std::string_literals;
tags.erase(std::remove(tags.begin(), tags.end(), ""s), tags.end()); std::erase(tags, ""s);
if (tags[0].find('=') == tags[0].npos) if (tags[0].find('=') == tags[0].npos)
{ {
if (tags.size() == 1) /* only name */ if (tags.size() == 1) /* only name */

View File

@ -217,6 +217,14 @@ public:
private: private:
ColumnsContainer columns; ColumnsContainer columns;
/// Subcolumns are not nested columns.
///
/// Example of subcolumns:
/// - .size0 for Array
/// - .null for Nullable
///
/// While nested columns have form like foo.bar
SubcolumnsContainter subcolumns; SubcolumnsContainter subcolumns;
void modifyColumnOrder(const String & column_name, const String & after_column, bool first); void modifyColumnOrder(const String & column_name, const String & after_column, bool first);

View File

@ -3676,10 +3676,10 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String &
parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); parts = getVisibleDataPartsVectorInPartition(local_context, partition_id);
auto disk = getStoragePolicy()->getDiskByName(name); auto disk = getStoragePolicy()->getDiskByName(name);
parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) std::erase_if(parts, [&](auto part_ptr)
{ {
return part_ptr->volume->getDisk()->getName() == disk->getName(); return part_ptr->volume->getDisk()->getName() == disk->getName();
}), parts.end()); });
if (parts.empty()) if (parts.empty())
{ {
@ -3724,7 +3724,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
if (parts.empty()) if (parts.empty())
throw Exception("Nothing to move (сheck that the partition exists).", ErrorCodes::NO_SUCH_DATA_PART); throw Exception("Nothing to move (сheck that the partition exists).", ErrorCodes::NO_SUCH_DATA_PART);
parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) std::erase_if(parts, [&](auto part_ptr)
{ {
for (const auto & disk : volume->getDisks()) for (const auto & disk : volume->getDisks())
{ {
@ -3734,7 +3734,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String
} }
} }
return false; return false;
}), parts.end()); });
if (parts.empty()) if (parts.empty())
{ {
@ -4220,8 +4220,7 @@ void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts
return !part->version.isVisible(snapshot_version, current_tid); return !part->version.isVisible(snapshot_version, current_tid);
}; };
auto new_end_it = std::remove_if(maybe_visible_parts.begin(), maybe_visible_parts.end(), need_remove_pred); std::erase_if(maybe_visible_parts, need_remove_pred);
maybe_visible_parts.erase(new_end_it, maybe_visible_parts.end());
[[maybe_unused]] size_t visible_size = maybe_visible_parts.size(); [[maybe_unused]] size_t visible_size = maybe_visible_parts.size();
@ -6515,15 +6514,11 @@ ReservationPtr MergeTreeData::balancedReservation(
} }
// Remove irrelevant parts. // Remove irrelevant parts.
covered_parts.erase( std::erase_if(covered_parts,
std::remove_if(
covered_parts.begin(),
covered_parts.end(),
[min_bytes_to_rebalance_partition_over_jbod](const auto & part) [min_bytes_to_rebalance_partition_over_jbod](const auto & part)
{ {
return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod); return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod);
}), });
covered_parts.end());
// Include current submerging big parts which are not yet in `currently_submerging_big_parts` // Include current submerging big parts which are not yet in `currently_submerging_big_parts`
for (const auto & part : covered_parts) for (const auto & part : covered_parts)

View File

@ -176,11 +176,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge(
return active_parts_set.getContainingPart(part->info) != part->name; return active_parts_set.getContainingPart(part->info) != part->name;
}; };
auto new_end_it = std::remove_if(active_parts.begin(), active_parts.end(), remove_pred); std::erase_if(active_parts, remove_pred);
active_parts.erase(new_end_it, active_parts.end());
new_end_it = std::remove_if(outdated_parts.begin(), outdated_parts.end(), remove_pred); std::erase_if(outdated_parts, remove_pred);
outdated_parts.erase(new_end_it, outdated_parts.end());
std::merge(active_parts.begin(), active_parts.end(), std::merge(active_parts.begin(), active_parts.end(),
outdated_parts.begin(), outdated_parts.end(), outdated_parts.begin(), outdated_parts.end(),
@ -640,220 +638,4 @@ size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData::
return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE); return static_cast<size_t>(res * DISK_USAGE_COEFFICIENT_TO_RESERVE);
} }
void MergeTreeDataMergerMutator::splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames)
{
ColumnsDescription part_columns(part->getColumns());
if (!isWidePart(part))
{
NameSet mutated_columns;
for (const auto & command : commands)
{
if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|| command.type == MutationCommand::Type::MATERIALIZE_COLUMN
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|| command.type == MutationCommand::Type::MATERIALIZE_TTL
|| command.type == MutationCommand::Type::DELETE
|| command.type == MutationCommand::Type::UPDATE)
{
for_interpreter.push_back(command);
for (const auto & [column_name, expr] : command.column_to_update_expression)
mutated_columns.emplace(column_name);
if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN)
mutated_columns.emplace(command.column_name);
}
else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION)
{
for_file_renames.push_back(command);
}
else if (part_columns.has(command.column_name))
{
if (command.type == MutationCommand::Type::DROP_COLUMN)
{
mutated_columns.emplace(command.column_name);
}
else if (command.type == MutationCommand::Type::RENAME_COLUMN)
{
for_interpreter.push_back(
{
.type = MutationCommand::Type::READ_COLUMN,
.column_name = command.rename_to,
});
mutated_columns.emplace(command.column_name);
part_columns.rename(command.column_name, command.rename_to);
}
}
}
/// If it's compact part, then we don't need to actually remove files
/// from disk we just don't read dropped columns
for (const auto & column : part->getColumns())
{
if (!mutated_columns.contains(column.name))
for_interpreter.emplace_back(
MutationCommand{.type = MutationCommand::Type::READ_COLUMN, .column_name = column.name, .data_type = column.type});
}
}
else
{
for (const auto & command : commands)
{
if (command.type == MutationCommand::Type::MATERIALIZE_INDEX
|| command.type == MutationCommand::Type::MATERIALIZE_COLUMN
|| command.type == MutationCommand::Type::MATERIALIZE_PROJECTION
|| command.type == MutationCommand::Type::MATERIALIZE_TTL
|| command.type == MutationCommand::Type::DELETE
|| command.type == MutationCommand::Type::UPDATE)
{
for_interpreter.push_back(command);
}
else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION)
{
for_file_renames.push_back(command);
}
/// If we don't have this column in source part, than we don't need
/// to materialize it
else if (part_columns.has(command.column_name))
{
if (command.type == MutationCommand::Type::READ_COLUMN)
{
for_interpreter.push_back(command);
}
else if (command.type == MutationCommand::Type::RENAME_COLUMN)
{
part_columns.rename(command.column_name, command.rename_to);
for_file_renames.push_back(command);
}
else
{
for_file_renames.push_back(command);
}
}
}
}
}
std::pair<NamesAndTypesList, SerializationInfoByName>
MergeTreeDataMergerMutator::getColumnsForNewDataPart(
MergeTreeData::DataPartPtr source_part,
const Block & updated_header,
NamesAndTypesList storage_columns,
const SerializationInfoByName & serialization_infos,
const MutationCommands & commands_for_removes)
{
NameSet removed_columns;
NameToNameMap renamed_columns_to_from;
NameToNameMap renamed_columns_from_to;
ColumnsDescription part_columns(source_part->getColumns());
/// All commands are validated in AlterCommand so we don't care about order
for (const auto & command : commands_for_removes)
{
/// If we don't have this column in source part, than we don't need to materialize it
if (!part_columns.has(command.column_name))
continue;
if (command.type == MutationCommand::DROP_COLUMN)
removed_columns.insert(command.column_name);
if (command.type == MutationCommand::RENAME_COLUMN)
{
renamed_columns_to_from.emplace(command.rename_to, command.column_name);
renamed_columns_from_to.emplace(command.column_name, command.rename_to);
}
}
SerializationInfoByName new_serialization_infos;
for (const auto & [name, info] : serialization_infos)
{
if (removed_columns.contains(name))
continue;
auto it = renamed_columns_from_to.find(name);
if (it != renamed_columns_from_to.end())
new_serialization_infos.emplace(it->second, info);
else
new_serialization_infos.emplace(name, info);
}
/// In compact parts we read all columns, because they all stored in a
/// single file
if (!isWidePart(source_part))
return {updated_header.getNamesAndTypesList(), new_serialization_infos};
Names source_column_names = source_part->getColumns().getNames();
NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end());
for (auto it = storage_columns.begin(); it != storage_columns.end();)
{
if (updated_header.has(it->name))
{
auto updated_type = updated_header.getByName(it->name).type;
if (updated_type != it->type)
it->type = updated_type;
++it;
}
else
{
if (!source_columns_name_set.contains(it->name))
{
/// Source part doesn't have column but some other column
/// was renamed to it's name.
auto renamed_it = renamed_columns_to_from.find(it->name);
if (renamed_it != renamed_columns_to_from.end()
&& source_columns_name_set.contains(renamed_it->second))
++it;
else
it = storage_columns.erase(it);
}
else
{
/// Check that this column was renamed to some other name
bool was_renamed = renamed_columns_from_to.contains(it->name);
bool was_removed = removed_columns.contains(it->name);
/// If we want to rename this column to some other name, than it
/// should it's previous version should be dropped or removed
if (renamed_columns_to_from.contains(it->name) && !was_renamed && !was_removed)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Incorrect mutation commands, trying to rename column {} to {}, but part {} already has column {}", renamed_columns_to_from[it->name], it->name, source_part->name, it->name);
/// Column was renamed and no other column renamed to it's name
/// or column is dropped.
if (!renamed_columns_to_from.contains(it->name) && (was_renamed || was_removed))
it = storage_columns.erase(it);
else
++it;
}
}
}
return {storage_columns, new_serialization_infos};
}
ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies)
{
if (!metadata_snapshot->hasAnyTTL())
return ExecuteTTLType::NONE;
bool has_ttl_expression = false;
for (const auto & dependency : dependencies)
{
if (dependency.kind == ColumnDependency::TTL_EXPRESSION)
has_ttl_expression = true;
if (dependency.kind == ColumnDependency::TTL_TARGET)
return ExecuteTTLType::NORMAL;
}
return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE;
}
} }

View File

@ -146,27 +146,6 @@ private:
friend class MutateTask; friend class MutateTask;
friend class MergeTask; friend class MergeTask;
/** Split mutation commands into two parts:
* First part should be executed by mutations interpreter.
* Other is just simple drop/renames, so they can be executed without interpreter.
*/
static void splitMutationCommands(
MergeTreeData::DataPartPtr part,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames);
/// Get the columns list of the resulting part in the same order as storage_columns.
static std::pair<NamesAndTypesList, SerializationInfoByName> getColumnsForNewDataPart(
MergeTreeData::DataPartPtr source_part,
const Block & updated_header,
NamesAndTypesList storage_columns,
const SerializationInfoByName & serialization_infos,
const MutationCommands & commands_for_removes);
static ExecuteTTLType shouldExecuteTTL(
const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies);
public : public :
/** Is used to cancel all merges and mutations. On cancel() call all currently running actions will throw exception soon. /** Is used to cancel all merges and mutations. On cancel() call all currently running actions will throw exception soon.
* All new attempts to start a merge or mutation will throw an exception until all 'LockHolder' objects will be destroyed. * All new attempts to start a merge or mutation will throw an exception until all 'LockHolder' objects will be destroyed.

View File

@ -74,7 +74,7 @@ static void splitMutationCommands(
mutated_columns.emplace(column_name); mutated_columns.emplace(column_name);
if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN) if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN)
mutated_columns.emplace(command.column_name); mutated_columns.emplace(command.column_name);
} }
else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION)
{ {
@ -146,6 +146,124 @@ static void splitMutationCommands(
} }
} }
/// Get the columns list of the resulting part in the same order as storage_columns.
static std::pair<NamesAndTypesList, SerializationInfoByName>
getColumnsForNewDataPart(
MergeTreeData::DataPartPtr source_part,
const Block & updated_header,
NamesAndTypesList storage_columns,
const SerializationInfoByName & serialization_infos,
const MutationCommands & commands_for_removes)
{
NameSet removed_columns;
NameToNameMap renamed_columns_to_from;
NameToNameMap renamed_columns_from_to;
ColumnsDescription part_columns(source_part->getColumns());
/// All commands are validated in AlterCommand so we don't care about order
for (const auto & command : commands_for_removes)
{
/// If we don't have this column in source part, than we don't need to materialize it
if (!part_columns.has(command.column_name))
continue;
if (command.type == MutationCommand::DROP_COLUMN)
removed_columns.insert(command.column_name);
if (command.type == MutationCommand::RENAME_COLUMN)
{
renamed_columns_to_from.emplace(command.rename_to, command.column_name);
renamed_columns_from_to.emplace(command.column_name, command.rename_to);
}
}
SerializationInfoByName new_serialization_infos;
for (const auto & [name, info] : serialization_infos)
{
if (removed_columns.contains(name))
continue;
auto it = renamed_columns_from_to.find(name);
if (it != renamed_columns_from_to.end())
new_serialization_infos.emplace(it->second, info);
else
new_serialization_infos.emplace(name, info);
}
/// In compact parts we read all columns, because they all stored in a
/// single file
if (!isWidePart(source_part))
return {updated_header.getNamesAndTypesList(), new_serialization_infos};
Names source_column_names = source_part->getColumns().getNames();
NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end());
for (auto it = storage_columns.begin(); it != storage_columns.end();)
{
if (updated_header.has(it->name))
{
auto updated_type = updated_header.getByName(it->name).type;
if (updated_type != it->type)
it->type = updated_type;
++it;
}
else
{
if (!source_columns_name_set.contains(it->name))
{
/// Source part doesn't have column but some other column
/// was renamed to it's name.
auto renamed_it = renamed_columns_to_from.find(it->name);
if (renamed_it != renamed_columns_to_from.end()
&& source_columns_name_set.contains(renamed_it->second))
++it;
else
it = storage_columns.erase(it);
}
else
{
/// Check that this column was renamed to some other name
bool was_renamed = renamed_columns_from_to.contains(it->name);
bool was_removed = removed_columns.contains(it->name);
/// If we want to rename this column to some other name, than it
/// should it's previous version should be dropped or removed
if (renamed_columns_to_from.contains(it->name) && !was_renamed && !was_removed)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Incorrect mutation commands, trying to rename column {} to {}, but part {} already has column {}", renamed_columns_to_from[it->name], it->name, source_part->name, it->name);
/// Column was renamed and no other column renamed to it's name
/// or column is dropped.
if (!renamed_columns_to_from.contains(it->name) && (was_renamed || was_removed))
it = storage_columns.erase(it);
else
++it;
}
}
}
return {storage_columns, new_serialization_infos};
}
static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snapshot, const ColumnDependencies & dependencies)
{
if (!metadata_snapshot->hasAnyTTL())
return ExecuteTTLType::NONE;
bool has_ttl_expression = false;
for (const auto & dependency : dependencies)
{
if (dependency.kind == ColumnDependency::TTL_EXPRESSION)
has_ttl_expression = true;
if (dependency.kind == ColumnDependency::TTL_TARGET)
return ExecuteTTLType::NORMAL;
}
return has_ttl_expression ? ExecuteTTLType::RECALCULATE : ExecuteTTLType::NONE;
}
/// Get skip indices, that should exists in the resulting data part. /// Get skip indices, that should exists in the resulting data part.
static MergeTreeIndices getIndicesForNewDataPart( static MergeTreeIndices getIndicesForNewDataPart(
@ -1337,7 +1455,7 @@ bool MutateTask::prepare()
/// It shouldn't be changed by mutation. /// It shouldn't be changed by mutation.
ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info; ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info;
auto [new_columns, new_infos] = MergeTreeDataMergerMutator::getColumnsForNewDataPart( auto [new_columns, new_infos] = MutationHelpers::getColumnsForNewDataPart(
ctx->source_part, ctx->updated_header, ctx->storage_columns, ctx->source_part, ctx->updated_header, ctx->storage_columns,
ctx->source_part->getSerializationInfos(), ctx->commands_for_part); ctx->source_part->getSerializationInfos(), ctx->commands_for_part);
@ -1357,7 +1475,7 @@ bool MutateTask::prepare()
ctx->execute_ttl_type = ExecuteTTLType::NONE; ctx->execute_ttl_type = ExecuteTTLType::NONE;
if (ctx->mutating_pipeline.initialized()) if (ctx->mutating_pipeline.initialized())
ctx->execute_ttl_type = MergeTreeDataMergerMutator::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies()); ctx->execute_ttl_type = MutationHelpers::shouldExecuteTTL(ctx->metadata_snapshot, ctx->interpreter->getColumnDependencies());
/// All columns from part are changed and may be some more that were missing before in part /// All columns from part are changed and may be some more that were missing before in part
/// TODO We can materialize compact part without copying data /// TODO We can materialize compact part without copying data

View File

@ -611,9 +611,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
/// Multiple log entries that must be copied to the queue. /// Multiple log entries that must be copied to the queue.
log_entries.erase( std::erase_if(log_entries, [&min_log_entry](const String & entry) { return entry < min_log_entry; });
std::remove_if(log_entries.begin(), log_entries.end(), [&min_log_entry](const String & entry) { return entry < min_log_entry; }),
log_entries.end());
if (!log_entries.empty()) if (!log_entries.empty())
{ {

View File

@ -279,10 +279,10 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c
if (!prefix.empty()) if (!prefix.empty())
{ {
// Remove nodes that do not match specified prefix // Remove nodes that do not match specified prefix
nodes.erase(std::remove_if(nodes.begin(), nodes.end(), [&prefix, &path_part] (const String & node) std::erase_if(nodes, [&prefix, &path_part] (const String & node)
{ {
return (path_part + '/' + node).substr(0, prefix.size()) != prefix; return (path_part + '/' + node).substr(0, prefix.size()) != prefix;
}), nodes.end()); });
} }
std::vector<std::future<Coordination::GetResponse>> futures; std::vector<std::future<Coordination::GetResponse>> futures;

View File

@ -108,7 +108,8 @@ def _exec_get_with_retry(url):
WorkflowDescription = namedtuple( WorkflowDescription = namedtuple(
"WorkflowDescription", ["run_id", "status", "rerun_url", "cancel_url", "conclusion"] "WorkflowDescription",
["run_id", "head_sha", "status", "rerun_url", "cancel_url", "conclusion"],
) )
@ -160,6 +161,7 @@ def get_workflows_description_for_pull_request(
workflow_descriptions.append( workflow_descriptions.append(
WorkflowDescription( WorkflowDescription(
run_id=workflow["id"], run_id=workflow["id"],
head_sha=workflow["head_sha"],
status=workflow["status"], status=workflow["status"],
rerun_url=workflow["rerun_url"], rerun_url=workflow["rerun_url"],
cancel_url=workflow["cancel_url"], cancel_url=workflow["cancel_url"],
@ -170,11 +172,9 @@ def get_workflows_description_for_pull_request(
return workflow_descriptions return workflow_descriptions
def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: def get_workflow_description_fallback(pull_request_event) -> List[WorkflowDescription]:
pull_request_event = event_data["pull_request"]
head_repo = pull_request_event["head"]["repo"]["full_name"] head_repo = pull_request_event["head"]["repo"]["full_name"]
head_branch = pull_request_event["head"]["ref"] head_branch = pull_request_event["head"]["ref"]
head_sha = pull_request_event["head"]["sha"]
print("Get last 500 workflows from API to search related there") print("Get last 500 workflows from API to search related there")
# Fallback for a case of an already deleted branch and no workflows received # Fallback for a case of an already deleted branch and no workflows received
request_url = f"{API_URL}/actions/runs?per_page=100" request_url = f"{API_URL}/actions/runs?per_page=100"
@ -213,16 +213,11 @@ def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]:
} }
for wf in workflows_data for wf in workflows_data
] ]
if event_data["action"] == "synchronize":
print(f"Leave only workflows with SHA but {head_sha} for updated PR")
# Cancel all events with SHA different than current
workflows_data = list(
filter(lambda x: x["head_sha"] != head_sha, workflows_data)
)
workflow_descriptions = [ workflow_descriptions = [
WorkflowDescription( WorkflowDescription(
run_id=wf["id"], run_id=wf["id"],
head_sha=wf["head_sha"],
status=wf["status"], status=wf["status"],
rerun_url=wf["rerun_url"], rerun_url=wf["rerun_url"],
cancel_url=wf["cancel_url"], cancel_url=wf["cancel_url"],
@ -238,6 +233,7 @@ def get_workflow_description(workflow_id) -> WorkflowDescription:
workflow = _exec_get_with_retry(API_URL + f"/actions/runs/{workflow_id}") workflow = _exec_get_with_retry(API_URL + f"/actions/runs/{workflow_id}")
return WorkflowDescription( return WorkflowDescription(
run_id=workflow["id"], run_id=workflow["id"],
head_sha=workflow["head_sha"],
status=workflow["status"], status=workflow["status"],
rerun_url=workflow["rerun_url"], rerun_url=workflow["rerun_url"],
cancel_url=workflow["cancel_url"], cancel_url=workflow["cancel_url"],
@ -281,7 +277,7 @@ def main(event):
print("PR merged/closed or manually labeled 'do not test' will kill workflows") print("PR merged/closed or manually labeled 'do not test' will kill workflows")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(pull_request)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(event_data) workflow_descriptions or get_workflow_description_fallback(pull_request)
) )
urls_to_cancel = [] urls_to_cancel = []
for workflow_description in workflow_descriptions: for workflow_description in workflow_descriptions:
@ -296,13 +292,14 @@ def main(event):
print("PR is synchronized, going to stop old actions") print("PR is synchronized, going to stop old actions")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(pull_request)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(event_data) workflow_descriptions or get_workflow_description_fallback(pull_request)
) )
urls_to_cancel = [] urls_to_cancel = []
for workflow_description in workflow_descriptions: for workflow_description in workflow_descriptions:
if ( if (
workflow_description.status != "completed" workflow_description.status != "completed"
and workflow_description.conclusion != "cancelled" and workflow_description.conclusion != "cancelled"
and workflow_description.head_sha != pull_request["head"]["sha"]
): ):
urls_to_cancel.append(workflow_description.cancel_url) urls_to_cancel.append(workflow_description.cancel_url)
print(f"Found {len(urls_to_cancel)} workflows to cancel") print(f"Found {len(urls_to_cancel)} workflows to cancel")
@ -311,7 +308,7 @@ def main(event):
print("PR marked with can be tested label, rerun workflow") print("PR marked with can be tested label, rerun workflow")
workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = get_workflows_description_for_pull_request(pull_request)
workflow_descriptions = ( workflow_descriptions = (
workflow_descriptions or get_workflow_description_fallback(event_data) workflow_descriptions or get_workflow_description_fallback(pull_request)
) )
if not workflow_descriptions: if not workflow_descriptions:
print("Not found any workflows") print("Not found any workflows")

View File

@ -1311,7 +1311,7 @@
"toSecond" "toSecond"
"toStartOfDay" "toStartOfDay"
"toStartOfFifteenMinutes" "toStartOfFifteenMinutes"
"toStartOfFiveMinute" "toStartOfFiveMinutes"
"toStartOfHour" "toStartOfHour"
"toStartOfInterval" "toStartOfInterval"
"toStartOfISOYear" "toStartOfISOYear"

View File

@ -507,7 +507,7 @@
"toStartOfFifteenMinutes" "toStartOfFifteenMinutes"
"emptyArrayUInt8" "emptyArrayUInt8"
"dictGetUInt8" "dictGetUInt8"
"toStartOfFiveMinute" "toStartOfFiveMinutes"
"cbrt" "cbrt"
"toStartOfMinute" "toStartOfMinute"
"dictGet" "dictGet"

View File

@ -698,7 +698,7 @@ def test_abrupt_connection_loss_while_heavy_replication(started_cluster):
def test_abrupt_server_restart_while_heavy_replication(started_cluster): def test_abrupt_server_restart_while_heavy_replication(started_cluster):
# FIXME (kssenii) temporary disabled # FIXME (kssenii) temporary disabled
if instance.is_built_with_address_sanitizer(): if instance.is_built_with_address_sanitizer():
pytest.skip("Temporary disabled (FIXME)") pytest.skip("Temporary disabled (FIXME)")

View File

@ -16,7 +16,7 @@
<value>toISOYear</value> <value>toISOYear</value>
<value>toStartOfMinute</value> <value>toStartOfMinute</value>
<value>toStartOfFiveMinute</value> <value>toStartOfFiveMinutes</value>
<value>toStartOfFifteenMinutes</value> <value>toStartOfFifteenMinutes</value>
<value>toStartOfHour</value> <value>toStartOfHour</value>
<value>toStartOfDay</value> <value>toStartOfDay</value>

View File

@ -107,7 +107,7 @@ toStartOfMinute
2019-02-06 19:57:00 2019-02-06 19:57:00
2019-02-07 04:57:00 2019-02-07 04:57:00
2019-02-06 11:57:00 2019-02-06 11:57:00
toStartOfFiveMinute toStartOfFiveMinutes
2019-02-06 22:55:00 2019-02-06 22:55:00
2019-02-06 20:55:00 2019-02-06 20:55:00
2019-02-06 19:55:00 2019-02-06 19:55:00

View File

@ -162,14 +162,14 @@ SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Europe/London'), 'Europ
SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo');
SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
/* toStartOfFiveMinute */ /* toStartOfFiveMinutes */
SELECT 'toStartOfFiveMinute'; SELECT 'toStartOfFiveMinutes';
SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow');
SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris');
SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/London'), 'Europe/London');
SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo');
SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn');
/* toStartOfTenMinutes */ /* toStartOfTenMinutes */

View File

@ -199,7 +199,7 @@ SELECT toMinute(NULL);
SELECT toStartOfDay(NULL); SELECT toStartOfDay(NULL);
SELECT toMonday(NULL); SELECT toMonday(NULL);
SELECT toUInt16OrZero(NULL); SELECT toUInt16OrZero(NULL);
SELECT toStartOfFiveMinute(NULL); SELECT toStartOfFiveMinutes(NULL);
SELECT halfMD5(NULL); SELECT halfMD5(NULL);
SELECT toStartOfHour(NULL); SELECT toStartOfHour(NULL);
SELECT toRelativeYearNum(NULL); SELECT toRelativeYearNum(NULL);

View File

@ -28,7 +28,7 @@ toStartOfWeek(N, 'Europe/Moscow')
toStartOfDay(N, 'Europe/Moscow') toStartOfDay(N, 'Europe/Moscow')
toStartOfHour(N, 'Europe/Moscow') toStartOfHour(N, 'Europe/Moscow')
toStartOfMinute(N, 'Europe/Moscow') toStartOfMinute(N, 'Europe/Moscow')
toStartOfFiveMinute(N, 'Europe/Moscow') toStartOfFiveMinutes(N, 'Europe/Moscow')
toStartOfTenMinutes(N, 'Europe/Moscow') toStartOfTenMinutes(N, 'Europe/Moscow')
toStartOfFifteenMinutes(N, 'Europe/Moscow') toStartOfFifteenMinutes(N, 'Europe/Moscow')
toStartOfInterval(N, INTERVAL 1 year, 'Europe/Moscow') toStartOfInterval(N, INTERVAL 1 year, 'Europe/Moscow')

View File

@ -98,7 +98,7 @@ Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:00" "DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00" "DateTime('Europe/Moscow')","2019-09-16 19:20:00"
------------------------------------------ ------------------------------------------
SELECT toStartOfFiveMinute(N, \'Europe/Moscow\') SELECT toStartOfFiveMinutes(N, \'Europe/Moscow\')
Code: 43 Code: 43
"DateTime('Europe/Moscow')","2019-09-16 19:20:00" "DateTime('Europe/Moscow')","2019-09-16 19:20:00"
"DateTime('Europe/Moscow')","2019-09-16 19:20:00" "DateTime('Europe/Moscow')","2019-09-16 19:20:00"

View File

@ -1,3 +1,5 @@
-- Tags: no-parallel
DROP DICTIONARY IF EXISTS system.dict1; DROP DICTIONARY IF EXISTS system.dict1;
CREATE DICTIONARY IF NOT EXISTS system.dict1 CREATE DICTIONARY IF NOT EXISTS system.dict1

View File

@ -1,6 +1,8 @@
#!/usr/bin/env bash #!/usr/bin/env bash
# Tags: no-parallel # Server may ignore some exceptions, but it still print exceptions to logs and (at least in CI) sends Error and Warning log messages to client
# making test fail because of non-empty stderr. Ignore such log messages.
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh

View File

@ -4,7 +4,7 @@ drop table if exists mt2;
create table mt1 (n Int64) engine=MergeTree order by n; create table mt1 (n Int64) engine=MergeTree order by n;
create table mt2 (n Int64) engine=MergeTree order by n; create table mt2 (n Int64) engine=MergeTree order by n;
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- no transaction
rollback; -- { serverError INVALID_TRANSACTION } rollback; -- { serverError INVALID_TRANSACTION }
begin transaction; begin transaction;
@ -31,7 +31,7 @@ select 'on exception before start', arraySort(groupArray(n)) from (select n from
-- rollback on exception before start -- rollback on exception before start
select functionThatDoesNotExist(); -- { serverError 46 } select functionThatDoesNotExist(); -- { serverError 46 }
-- cannot commit after exception -- cannot commit after exception
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- after 46
begin transaction; -- { serverError INVALID_TRANSACTION } begin transaction; -- { serverError INVALID_TRANSACTION }
rollback; rollback;
@ -42,7 +42,7 @@ select 'on exception while processing', arraySort(groupArray(n)) from (select n
-- rollback on exception while processing -- rollback on exception while processing
select throwIf(100 < number) from numbers(1000); -- { serverError 395 } select throwIf(100 < number) from numbers(1000); -- { serverError 395 }
-- cannot commit after exception -- cannot commit after exception
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- after 395
insert into mt1 values (5); -- { serverError INVALID_TRANSACTION } insert into mt1 values (5); -- { serverError INVALID_TRANSACTION }
insert into mt2 values (50); -- { serverError INVALID_TRANSACTION } insert into mt2 values (50); -- { serverError INVALID_TRANSACTION }
select 1; -- { serverError INVALID_TRANSACTION } select 1; -- { serverError INVALID_TRANSACTION }
@ -52,10 +52,9 @@ begin transaction;
insert into mt1 values (6); insert into mt1 values (6);
insert into mt2 values (60); insert into mt2 values (60);
select 'on session close', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2); select 'on session close', arraySort(groupArray(n)) from (select n from mt1 union all select * from mt2);
-- trigger reconnection by error on client, check rollback on session close
insert into mt1 values ([1]); -- { clientError 43 } insert into mt1 values ([1]); -- { clientError 43 }
commit; -- { serverError INVALID_TRANSACTION } -- INSERT failures does not produce client reconnect anymore, so rollback can be done
rollback; -- { serverError INVALID_TRANSACTION } rollback;
begin transaction; begin transaction;
insert into mt1 values (7); insert into mt1 values (7);
@ -82,19 +81,19 @@ rollback;
begin transaction; begin transaction;
create table m (n int) engine=Memory; -- { serverError 48 } create table m (n int) engine=Memory; -- { serverError 48 }
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- after 48
rollback; rollback;
create table m (n int) engine=Memory; create table m (n int) engine=Memory;
begin transaction; begin transaction;
insert into m values (1); -- { serverError 48 } insert into m values (1); -- { serverError 48 }
select * from m; -- { serverError INVALID_TRANSACTION } select * from m; -- { serverError INVALID_TRANSACTION }
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- after 48
rollback; rollback;
begin transaction; begin transaction;
select * from m; -- { serverError 48 } select * from m; -- { serverError 48 }
commit; -- { serverError INVALID_TRANSACTION } commit; -- { serverError INVALID_TRANSACTION } -- after 48
rollback; rollback;
drop table m; drop table m;

View File

@ -1,2 +1,2 @@
1 OSReadChars
Test OK OSCPUVirtualTimeMicroseconds

View File

@ -8,46 +8,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh # shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh . "$CURDIR"/../shell_config.sh
function read_numbers_func() # NOTE: netlink taskstruct interface uses rounding to 1KB [1], so we cannot use ${BASH_SOURCE[0]}
{ #
$CLICKHOUSE_CLIENT -q " # [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101
SELECT * FROM numbers(600000000) FORMAT Null SETTINGS max_threads = 1 tmp_path=$(mktemp "$CURDIR/01268_procfs_metrics.XXXXXX")
"; trap 'rm -f $tmp_path' EXIT
} truncate -s1025 "$tmp_path"
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars
function show_processes_func() # NOTE: that OSCPUVirtualTimeMicroseconds is in microseconds, so 1e6 is not enough.
{ $CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM numbers(10e6) FORMAT Null" |& grep -m1 -F -o -e OSCPUVirtualTimeMicroseconds
while true; do exit 0
sleep 0.1;
# These two system metrics for the generating query above are guaranteed to be nonzero when ProcFS is mounted at /proc
$CLICKHOUSE_CLIENT -q "
SELECT count() > 0 FROM system.processes\
WHERE ProfileEvents['OSCPUVirtualTimeMicroseconds'] > 0 AND ProfileEvents['OSReadChars'] > 0 \
SETTINGS max_threads = 1
" | grep '1' && break;
done
}
export -f read_numbers_func;
export -f show_processes_func;
TIMEOUT=3
timeout $TIMEOUT bash -c read_numbers_func &
timeout $TIMEOUT bash -c show_processes_func &
wait
# otherwise it can be alive after test
query_alive=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query ILIKE 'SELECT * FROM numbers(600000000)%'")
while [[ $query_alive != 0 ]]
do
$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query ilike '%SELECT * FROM numbers(600000000)%'" 2> /dev/null 1> /dev/null
sleep 0.5
query_alive=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.processes WHERE query ILIKE 'SELECT * FROM numbers(600000000)%'")
done
echo "Test OK"

View File

@ -10,46 +10,40 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh . "$CURDIR"/../shell_config.sh
function thread_create { function thread_create()
while true; do {
$CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
function thread_drop { function thread_drop()
while true; do {
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
function thread_rename { function thread_rename()
while true; do {
$CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
function thread_select { function thread_select()
while true; do {
$CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
function thread_insert { function thread_insert()
while true; do {
$CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
function thread_insert_select { function thread_insert_select()
while true; do {
$CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)'
sleep 0.0$RANDOM sleep 0.0$RANDOM
done
} }
export -f thread_create export -f thread_create
@ -65,18 +59,18 @@ export -f thread_insert_select
function test_with_engine { function test_with_engine {
echo "Testing $1" echo "Testing $1"
timeout 10 bash -c "thread_create t1 $1" & clickhouse_client_loop_timeout 10 thread_create t1 $1 &
timeout 10 bash -c "thread_create t2 $1" & clickhouse_client_loop_timeout 10 thread_create t2 $1 &
timeout 10 bash -c 'thread_drop t1' & clickhouse_client_loop_timeout 10 thread_drop t1 &
timeout 10 bash -c 'thread_drop t2' & clickhouse_client_loop_timeout 10 thread_drop t2 &
timeout 10 bash -c 'thread_rename t1 t2' & clickhouse_client_loop_timeout 10 thread_rename t1 t2 &
timeout 10 bash -c 'thread_rename t2 t1' & clickhouse_client_loop_timeout 10 thread_rename t2 t1 &
timeout 10 bash -c 'thread_select t1' & clickhouse_client_loop_timeout 10 thread_select t1 &
timeout 10 bash -c 'thread_select t2' & clickhouse_client_loop_timeout 10 thread_select t2 &
timeout 10 bash -c 'thread_insert t1 5' & clickhouse_client_loop_timeout 10 thread_insert t1 5 &
timeout 10 bash -c 'thread_insert t2 10' & clickhouse_client_loop_timeout 10 thread_insert t2 10 &
timeout 10 bash -c 'thread_insert_select t1 t2' & clickhouse_client_loop_timeout 10 thread_insert_select t1 t2 &
timeout 10 bash -c 'thread_insert_select t2 t1' & clickhouse_client_loop_timeout 10 thread_insert_select t2 t1 &
wait wait
echo "Done $1" echo "Done $1"

View File

@ -104,7 +104,7 @@
2021-01-01 2021-01-01
-------toStartOfSecond--------- -------toStartOfSecond---------
-------toStartOfMinute--------- -------toStartOfMinute---------
-------toStartOfFiveMinute--------- -------toStartOfFiveMinutes---------
-------toStartOfTenMinutes--------- -------toStartOfTenMinutes---------
-------toStartOfFifteenMinutes--------- -------toStartOfFifteenMinutes---------
-------toStartOfHour--------- -------toStartOfHour---------

View File

@ -46,8 +46,8 @@ select '-------toStartOfSecond---------';
select toStartOfSecond(x1) from t1; -- { serverError 43 } select toStartOfSecond(x1) from t1; -- { serverError 43 }
select '-------toStartOfMinute---------'; select '-------toStartOfMinute---------';
select toStartOfMinute(x1) from t1; -- { serverError 43 } select toStartOfMinute(x1) from t1; -- { serverError 43 }
select '-------toStartOfFiveMinute---------'; select '-------toStartOfFiveMinutes---------';
select toStartOfFiveMinute(x1) from t1; -- { serverError 43 } select toStartOfFiveMinutes(x1) from t1; -- { serverError 43 }
select '-------toStartOfTenMinutes---------'; select '-------toStartOfTenMinutes---------';
select toStartOfTenMinutes(x1) from t1; -- { serverError 43 } select toStartOfTenMinutes(x1) from t1; -- { serverError 43 }
select '-------toStartOfFifteenMinutes---------'; select '-------toStartOfFifteenMinutes---------';

View File

@ -5,7 +5,7 @@ timeZoneOffset(t): -2670
formatDateTime(t, '%F %T', 'Africa/Monrovia'): 1970-06-17 07:39:21 formatDateTime(t, '%F %T', 'Africa/Monrovia'): 1970-06-17 07:39:21
toString(t, 'Africa/Monrovia'): 1970-06-17 07:39:21 toString(t, 'Africa/Monrovia'): 1970-06-17 07:39:21
toStartOfMinute(t): 1970-06-17 07:39:00 toStartOfMinute(t): 1970-06-17 07:39:00
toStartOfFiveMinute(t): 1970-06-17 07:35:00 toStartOfFiveMinutes(t): 1970-06-17 07:35:00
toStartOfFifteenMinutes(t): 1970-06-17 07:30:00 toStartOfFifteenMinutes(t): 1970-06-17 07:30:00
toStartOfTenMinutes(t): 1970-06-17 07:30:00 toStartOfTenMinutes(t): 1970-06-17 07:30:00
toStartOfHour(t): 1970-06-17 07:00:00 toStartOfHour(t): 1970-06-17 07:00:00

View File

@ -5,7 +5,7 @@ SELECT toUnixTimestamp(t),
formatDateTime(t, '%F %T', 'Africa/Monrovia'), formatDateTime(t, '%F %T', 'Africa/Monrovia'),
toString(t, 'Africa/Monrovia'), toString(t, 'Africa/Monrovia'),
toStartOfMinute(t), toStartOfMinute(t),
toStartOfFiveMinute(t), toStartOfFiveMinutes(t),
toStartOfFifteenMinutes(t), toStartOfFifteenMinutes(t),
toStartOfTenMinutes(t), toStartOfTenMinutes(t),
toStartOfHour(t), toStartOfHour(t),

View File

@ -12,7 +12,7 @@ select toStartOfYear(toDate(0));
select toStartOfYear(toDateTime(0, 'Europe/Moscow')); select toStartOfYear(toDateTime(0, 'Europe/Moscow'));
select toTime(toDateTime(0, 'Europe/Moscow')); select toTime(toDateTime(0, 'Europe/Moscow'));
select toStartOfMinute(toDateTime(0, 'Europe/Moscow')); select toStartOfMinute(toDateTime(0, 'Europe/Moscow'));
select toStartOfFiveMinute(toDateTime(0, 'Europe/Moscow')); select toStartOfFiveMinutes(toDateTime(0, 'Europe/Moscow'));
select toStartOfTenMinutes(toDateTime(0, 'Europe/Moscow')); select toStartOfTenMinutes(toDateTime(0, 'Europe/Moscow'));
select toStartOfFifteenMinutes(toDateTime(0, 'Europe/Moscow')); select toStartOfFifteenMinutes(toDateTime(0, 'Europe/Moscow'));
select toStartOfHour(toDateTime(0, 'Europe/Moscow')); select toStartOfHour(toDateTime(0, 'Europe/Moscow'));
@ -25,7 +25,7 @@ select toStartOfQuarter(toDateTime(0, 'America/Los_Angeles'));
select toStartOfYear(toDateTime(0, 'America/Los_Angeles')); select toStartOfYear(toDateTime(0, 'America/Los_Angeles'));
select toTime(toDateTime(0, 'America/Los_Angeles'), 'America/Los_Angeles'); select toTime(toDateTime(0, 'America/Los_Angeles'), 'America/Los_Angeles');
select toStartOfMinute(toDateTime(0, 'America/Los_Angeles')); select toStartOfMinute(toDateTime(0, 'America/Los_Angeles'));
select toStartOfFiveMinute(toDateTime(0, 'America/Los_Angeles')); select toStartOfFiveMinutes(toDateTime(0, 'America/Los_Angeles'));
select toStartOfTenMinutes(toDateTime(0, 'America/Los_Angeles')); select toStartOfTenMinutes(toDateTime(0, 'America/Los_Angeles'));
select toStartOfFifteenMinutes(toDateTime(0, 'America/Los_Angeles')); select toStartOfFifteenMinutes(toDateTime(0, 'America/Los_Angeles'));
select toStartOfHour(toDateTime(0, 'America/Los_Angeles')); select toStartOfHour(toDateTime(0, 'America/Los_Angeles'));

View File

@ -0,0 +1,40 @@
-- Tags: replica, long
-- Regression test for possible CHECKSUM_DOESNT_MATCH due to per-column TTL bug.
-- That had been fixed in https://github.com/ClickHouse/ClickHouse/pull/35820
drop table if exists ttl_02265;
drop table if exists ttl_02265_r2;
-- The bug is appears only for Wide part.
create table ttl_02265 (date Date, key Int, value String TTL date + interval 1 month) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/ttl_02265', 'r1') order by key partition by date settings min_bytes_for_wide_part=0;
create table ttl_02265_r2 (date Date, key Int, value String TTL date + interval 1 month) engine=ReplicatedMergeTree('/clickhouse/tables/{database}/ttl_02265', 'r2') order by key partition by date settings min_bytes_for_wide_part=0;
-- after, 20100101_0_0_0 will have ttl.txt and value.bin
insert into ttl_02265 values ('2010-01-01', 2010, 'foo');
-- after, 20100101_0_0_1 will not have neither ttl.txt nor value.bin
optimize table ttl_02265 final;
-- after, 20100101_0_0_2 will not have ttl.txt, but will have value.bin
optimize table ttl_02265 final;
system sync replica ttl_02265;
-- after detach/attach it will not have TTL in-memory, and will not have ttl.txt
detach table ttl_02265;
attach table ttl_02265;
-- So now the state for 20100101_0_0_2 is as follow:
--
-- table | in_memory_ttl | ttl.txt | value.bin/mrk2
-- ttl_02265 | N | N | N
-- ttl_02265_r2 | Y | N | N
--
-- And hence on the replica that does not have TTL in-memory (this replica),
-- it will try to apply TTL, and the column will be dropped,
-- but on another replica the column won't be dropped since it has in-memory TTL and will not apply TTL.
-- and eventually this will lead to the following error:
--
-- MergeFromLogEntryTask: Code: 40. DB::Exception: Part 20100101_0_0_3 from r2 has different columns hash. (CHECKSUM_DOESNT_MATCH) (version 22.4.1.1). Data after merge is not byte-identical to data on another replicas. There could be several reasons: 1. Using newer version of compression library after server update. 2. Using another compression method. 3. Non-deterministic compression algorithm (highly unlikely). 4. Non-deterministic merge algorithm due to logical error in code. 5. Data corruption in memory due to bug in code. 6. Data corruption in memory due to hardware issue. 7. Manual modification of source data after server startup. 8. Manual modification of checksums stored in ZooKeeper. 9. Part format related settings like 'enable_mixed_granularity_parts' are different on different replicas. We will download merged part from replica to force byte-identical result.
--
optimize table ttl_02265 final;
system flush logs;
select * from system.part_log where database = currentDatabase() and table like 'ttl_02265%' and error != 0;

View File

@ -54,7 +54,7 @@
* 4.2.0.4.19 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfHour](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofhour) * 4.2.0.4.19 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfHour](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofhour)
* 4.2.0.4.20 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofminute) * 4.2.0.4.20 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofminute)
* 4.2.0.4.21 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfSecond](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofsecond) * 4.2.0.4.21 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfSecond](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofsecond)
* 4.2.0.4.22 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffiveminute) * 4.2.0.4.22 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffiveminutes)
* 4.2.0.4.23 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoftenminutes) * 4.2.0.4.23 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoftenminutes)
* 4.2.0.4.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes) * 4.2.0.4.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes)
* 4.2.0.4.25 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfInterval](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofinterval) * 4.2.0.4.25 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfInterval](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofinterval)
@ -417,10 +417,10 @@ version: 1.0
[ClickHouse] SHALL support correct operation of the [toStartOfSecond](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartofsecond) [ClickHouse] SHALL support correct operation of the [toStartOfSecond](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartofsecond)
function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range]. function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].
###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes
version: 1.0 version: 1.0
[ClickHouse] SHALL support correct operation of the [toStartOfFiveMinute](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminute) [ClickHouse] SHALL support correct operation of the [toStartOfFiveMinutes](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminutes)
function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range]. function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].
###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes

View File

@ -714,15 +714,15 @@ RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfSecond = Req
num="4.2.0.4.21", num="4.2.0.4.21",
) )
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinute = Requirement( RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinutes = Requirement(
name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute", name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes",
version="1.0", version="1.0",
priority=None, priority=None,
group=None, group=None,
type=None, type=None,
uid=None, uid=None,
description=( description=(
"[ClickHouse] SHALL support correct operation of the [toStartOfFiveMinute](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminute)\n" "[ClickHouse] SHALL support correct operation of the [toStartOfFiveMinutes](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminutes)\n"
"function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].\n" "function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].\n"
"\n" "\n"
), ),
@ -1944,7 +1944,7 @@ SRS_010_ClickHouse_DateTime64_Extended_Range = Specification(
num="4.2.0.4.21", num="4.2.0.4.21",
), ),
Heading( Heading(
name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute", name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes",
level=5, level=5,
num="4.2.0.4.22", num="4.2.0.4.22",
), ),
@ -2282,7 +2282,7 @@ SRS_010_ClickHouse_DateTime64_Extended_Range = Specification(
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfHour, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfHour,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfMinute, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfMinute,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfSecond, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfSecond,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinute, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinutes,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfTenMinutes, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfTenMinutes,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFifteenMinutes, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFifteenMinutes,
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfInterval, RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfInterval,
@ -2399,7 +2399,7 @@ SRS_010_ClickHouse_DateTime64_Extended_Range = Specification(
* 4.2.0.4.19 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfHour](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofhour) * 4.2.0.4.19 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfHour](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofhour)
* 4.2.0.4.20 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofminute) * 4.2.0.4.20 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofminute)
* 4.2.0.4.21 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfSecond](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofsecond) * 4.2.0.4.21 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfSecond](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofsecond)
* 4.2.0.4.22 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffiveminute) * 4.2.0.4.22 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffiveminutes)
* 4.2.0.4.23 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoftenminutes) * 4.2.0.4.23 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoftenminutes)
* 4.2.0.4.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes) * 4.2.0.4.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes)
* 4.2.0.4.25 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfInterval](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofinterval) * 4.2.0.4.25 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfInterval](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartofinterval)
@ -2762,10 +2762,10 @@ version: 1.0
[ClickHouse] SHALL support correct operation of the [toStartOfSecond](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartofsecond) [ClickHouse] SHALL support correct operation of the [toStartOfSecond](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartofsecond)
function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range]. function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].
###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes
version: 1.0 version: 1.0
[ClickHouse] SHALL support correct operation of the [toStartOfFiveMinute](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminute) [ClickHouse] SHALL support correct operation of the [toStartOfFiveMinutes](https://clickhouse.com/docs/en/sql-reference/functions/date-time-functions/#tostartoffiveminutes)
function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range]. function used with the [DateTime64] data type when it stores dates within the [normal date range] and the [extended date range].
###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes

View File

@ -576,13 +576,13 @@ def to_start_of_minutes_interval(self, interval, func):
@TestScenario @TestScenario
@Requirements( @Requirements(
RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinute( RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinutes(
"1.0" "1.0"
) )
) )
def to_start_of_five_minute(self): def to_start_of_five_minutes(self):
"""Check the toStartOfFiveMinute with DateTime64 extended range.""" """Check the toStartOfFiveMinutes with DateTime64 extended range."""
to_start_of_minutes_interval(interval=5, func="toStartOfFiveMinute") to_start_of_minutes_interval(interval=5, func="toStartOfFiveMinutes")
@TestScenario @TestScenario

View File

@ -249,7 +249,7 @@ std::map<std::string, ColumnType> func_to_param_type = {
{"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt}, {"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt},
{"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt}, {"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt},
{"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt}, {"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt},
{"tostartoffiveminute", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt}, {"tostartoffiveminutes", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt},
{"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt}, {"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt},
{"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt}, {"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt},
{"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt}, {"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt},