From 1e1df8e10185f1d768aad4e888533b665455501d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 16 Apr 2022 21:43:14 +0200 Subject: [PATCH 01/31] Replace remove-erase idiom by C++20 erase()/erase_if() - makes the code less verbose while being as efficient --- src/Common/PoolWithFailoverBase.h | 6 +--- src/Common/format.h | 29 +++++++++---------- src/Databases/MySQL/DatabaseMySQL.cpp | 5 +--- src/Dictionaries/PolygonDictionaryUtils.h | 4 +-- src/Interpreters/ActionsDAG.cpp | 3 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Processors/Merges/Algorithms/Graphite.cpp | 4 +-- src/Storages/MergeTree/MergeTreeData.cpp | 19 +++++------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 ++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 4 +-- .../System/StorageSystemZooKeeper.cpp | 4 +-- 11 files changed, 33 insertions(+), 53 deletions(-) diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 3cac939b385..42b5b3d0990 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -296,11 +296,7 @@ PoolWithFailoverBase::getMany( "All connection tries failed. Log: \n\n" + fail_messages + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); - try_results.erase( - std::remove_if( - try_results.begin(), try_results.end(), - [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }), - try_results.end()); + std::erase_if(try_results, [](const TryResult & r) { return r.entry.isNull() || !r.is_usable; }); /// Sort so that preferred items are near the beginning. std::stable_sort( diff --git a/src/Common/format.h b/src/Common/format.h index a9382f247ab..812bc818de1 100644 --- a/src/Common/format.h +++ b/src/Common/format.h @@ -59,23 +59,20 @@ namespace Format { size_t i = 0; bool should_delete = true; - str.erase( - std::remove_if( - str.begin(), - str.end(), - [&i, &should_delete, &str](char) + std::erase_if( + str, + [&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] == '}'); - ++i; - if (is_double_brace && should_delete) - { - should_delete = false; - return true; - } - should_delete = true; - return false; - }), - str.end()); + should_delete = false; + return true; + } + should_delete = true; + return false; + }); }; index_positions.emplace_back(); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 5f4027a26b3..56e8ffed759 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -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); /// Unset settings - storage_children.erase( - std::remove_if(storage_children.begin(), storage_children.end(), - [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), - storage_children.end()); + std::erase_if(storage_children, [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }); ast_storage->settings = nullptr; } auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, diff --git a/src/Dictionaries/PolygonDictionaryUtils.h b/src/Dictionaries/PolygonDictionaryUtils.h index 9d6d6ae0501..66a223f4ecc 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.h +++ b/src/Dictionaries/PolygonDictionaryUtils.h @@ -219,10 +219,10 @@ private: auto current_box = Box(Point(current_min_x, current_min_y), Point(current_max_x, current_max_y)); Polygon 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]); - }), possible_ids.end()); + }); int covered = 0; #ifndef __clang_analyzer__ /// Triggers a warning in boost geometry. auto it = std::find_if(possible_ids.begin(), possible_ids.end(), [&](const auto id) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index f06ac229e94..962d7b8e165 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -435,8 +435,7 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs, bool allow_consta } nodes.remove_if([&](const Node & node) { return visited_nodes.count(&node) == 0; }); - auto it = std::remove_if(inputs.begin(), inputs.end(), [&](const Node * node) { return visited_nodes.count(node) == 0; }); - inputs.erase(it, inputs.end()); + std::erase_if(inputs, [&](const Node * node) { return visited_nodes.count(node) == 0; }); } static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * node, ColumnsWithTypeAndName arguments) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 9af6b61a0c1..15e790f56d7 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -221,7 +221,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r 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()); } diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index c0f595fa539..f77bb790332 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -274,9 +274,9 @@ std::string buildTaggedRegex(std::string regexp_str) std::vector tags; splitInto<';'>(tags, regexp_str); - /* remove empthy elements */ + /* remove empty elements */ 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.size() == 1) /* only name */ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 29b3083c38f..1f4a9a97bd9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3640,10 +3640,10 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & parts = getVisibleDataPartsVectorInPartition(local_context, partition_id); 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(); - }), parts.end()); + }); if (parts.empty()) { @@ -3688,7 +3688,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String if (parts.empty()) 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()) { @@ -3698,7 +3698,7 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String } } return false; - }), parts.end()); + }); if (parts.empty()) { @@ -4184,8 +4184,7 @@ void MergeTreeData::filterVisibleDataParts(DataPartsVector & maybe_visible_parts 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); - maybe_visible_parts.erase(new_end_it, maybe_visible_parts.end()); + std::erase_if(maybe_visible_parts, need_remove_pred); [[maybe_unused]] size_t visible_size = maybe_visible_parts.size(); @@ -6454,15 +6453,11 @@ ReservationPtr MergeTreeData::balancedReservation( } // Remove irrelevant parts. - covered_parts.erase( - std::remove_if( - covered_parts.begin(), - covered_parts.end(), + std::erase_if(covered_parts, [min_bytes_to_rebalance_partition_over_jbod](const auto & part) { 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` for (const auto & part : covered_parts) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 31d52cfa8ff..256c902d4bc 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -176,11 +176,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( return active_parts_set.getContainingPart(part->info) != part->name; }; - auto new_end_it = std::remove_if(active_parts.begin(), active_parts.end(), remove_pred); - active_parts.erase(new_end_it, active_parts.end()); + std::erase_if(active_parts, remove_pred); - new_end_it = std::remove_if(outdated_parts.begin(), outdated_parts.end(), remove_pred); - outdated_parts.erase(new_end_it, outdated_parts.end()); + std::erase_if(outdated_parts, remove_pred); std::merge(active_parts.begin(), active_parts.end(), outdated_parts.begin(), outdated_parts.end(), diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 35168fc3f49..21dedc44217 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -611,9 +611,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper /// Multiple log entries that must be copied to the queue. - log_entries.erase( - std::remove_if(log_entries.begin(), log_entries.end(), [&min_log_entry](const String & entry) { return entry < min_log_entry; }), - log_entries.end()); + std::erase_if(log_entries, [&min_log_entry](const String & entry) { return entry < min_log_entry; }); if (!log_entries.empty()) { diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 879951df162..4ba6c00ad9d 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -279,10 +279,10 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c if (!prefix.empty()) { // 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; - }), nodes.end()); + }); } std::vector> futures; From 2275af699e49c505c000003992e3c5ab2e6fa925 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 18 Apr 2022 09:36:24 +0200 Subject: [PATCH 02/31] Empty commit (to trigger tests) From 9a882cc8a655a909bc011f504dfaefcd49d62521 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Apr 2022 20:27:05 +0300 Subject: [PATCH 03/31] Regression test for CHECKSUM_DOESNT_MATCH error because of per-column TTL bug This is a regression test for possible CHECKSUM_DOESNT_MATCH due to per-column TTL bug, that had been fixed in [1]. [1]: https://github.com/ClickHouse/ClickHouse/pull/35820 v2: mark 02265_column_ttl as long Signed-off-by: Azat Khuzhin --- .../0_stateless/02265_column_ttl.reference | 0 .../queries/0_stateless/02265_column_ttl.sql | 40 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/02265_column_ttl.reference create mode 100644 tests/queries/0_stateless/02265_column_ttl.sql diff --git a/tests/queries/0_stateless/02265_column_ttl.reference b/tests/queries/0_stateless/02265_column_ttl.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02265_column_ttl.sql b/tests/queries/0_stateless/02265_column_ttl.sql new file mode 100644 index 00000000000..3ab175329bc --- /dev/null +++ b/tests/queries/0_stateless/02265_column_ttl.sql @@ -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; From 39aadf0975e9953f026664ba2b774e96ba7002af Mon Sep 17 00:00:00 2001 From: Memo Date: Fri, 22 Apr 2022 10:49:59 +0800 Subject: [PATCH 04/31] replaced toStartOfFiveMinute to toStartOfFiveMinutes --- .../functions/date-time-functions.md | 2 +- .../functions/date-time-functions.md | 2 +- .../functions/date-time-functions.md | 2 +- .../functions/date-time-functions.md | 2 +- src/Common/DateLUTImpl.h | 2 +- src/Common/tests/gtest_DateLUTImpl.cpp | 6 +++--- src/Functions/DateTimeTransforms.h | 8 ++++---- src/Functions/registerFunctionsDateTime.cpp | 4 ++-- src/Functions/toStartOfFiveMinute.cpp | 6 +++--- tests/fuzz/all.dict | 2 +- tests/fuzz/dictionaries/functions.dict | 2 +- tests/performance/date_time_long.xml | 2 +- .../0_stateless/00189_time_zones_long.reference | 2 +- .../0_stateless/00189_time_zones_long.sql | 14 +++++++------- tests/queries/0_stateless/00534_filimonov.data | 2 +- .../00921_datetime64_compatibility_long.python | 2 +- ...00921_datetime64_compatibility_long.reference | 2 +- .../0_stateless/01921_datatype_date32.reference | 2 +- .../0_stateless/01921_datatype_date32.sql | 4 ++-- .../01958_partial_hour_timezone.reference | 2 +- .../0_stateless/01958_partial_hour_timezone.sql | 2 +- .../02096_date_time_1970_saturation.sql | 4 ++-- .../requirements/requirements.md | 6 +++--- .../requirements/requirements.py | 16 ++++++++-------- .../tests/date_time_functions.py | 8 ++++---- utils/db-generator/query_db_generator.cpp | 2 +- 26 files changed, 54 insertions(+), 54 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index fc48c97bb61..b45a1fb3b47 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -373,7 +373,7 @@ Result: - [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. diff --git a/docs/ja/sql-reference/functions/date-time-functions.md b/docs/ja/sql-reference/functions/date-time-functions.md index 565b10eaece..cac2eaa3d44 100644 --- a/docs/ja/sql-reference/functions/date-time-functions.md +++ b/docs/ja/sql-reference/functions/date-time-functions.md @@ -151,7 +151,7 @@ Mode引数は、toWeek()のmode引数とまったく同じように動作しま 日付と時刻を分の開始まで切り捨てます。 -## トスタートオフィブミニュート {#tostartoffiveminute} +## トスタートオフィブミニュート {#tostartoffiveminutes} 日付と時刻を切り捨てます。 diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index b41defdc92d..0c4503073e1 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -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). -## toStartOfFiveMinute {#tostartoffiveminute} +## toStartOfFiveMinutes {#tostartoffiveminutes} Округляет дату-с-временем вниз до начала пятиминутного интервала. diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index 969f71011fd..96477fadb3b 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -227,7 +227,7 @@ SELECT toStartOfSecond(dt64, 'Asia/Istanbul'); - [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 服务器配置选项。 -## toStartOfFiveMinute {#tostartoffiveminute} +## toStartOfFiveMinutes {#tostartoffiveminutes} 将DateTime以五分钟为单位向前取整到最接近的时间点。 diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index bfeb3389753..fcf4714bd37 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -565,7 +565,7 @@ public: /// NOTE: Assuming timezone offset is a multiple of 15 minutes. 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 toStartOfTenMinutes(Time t) const { return toStartOfMinuteInterval(t, 10); } inline Time toStartOfHour(Time t) const { return roundDown(t, 3600); } diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 1eec83a6ec9..d522448d337 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -119,7 +119,7 @@ TEST(DateLUTTest, TimeValuesInMiddleOfRange) EXPECT_EQ(lut.toSecond(time), 11 /*unsigned*/); EXPECT_EQ(lut.toMinute(time), 20 /*unsigned*/); 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.toStartOfTenMinutes(time), 1568650800 /*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.toMinute(time), 0 /*unsigned*/); 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.toStartOfTenMinutes(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.toSecond(time), 53 /*unsigned*/); 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.toStartOfTenMinutes(time), 4294343400 /*time_t*/); EXPECT_EQ(lut.toStartOfHour(time), 4294342800 /*time_t*/); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 3a7baf45adc..6fabbb2af1d 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -467,17 +467,17 @@ struct ToStartOfNanosecondImpl using FactorTransform = ZeroTransform; }; -struct ToStartOfFiveMinuteImpl +struct ToStartOfFiveMinutesImpl { - static constexpr auto name = "toStartOfFiveMinute"; + static constexpr auto name = "toStartOfFiveMinutes"; static inline UInt32 execute(const DecimalUtils::DecimalComponents & 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) { - return time_zone.toStartOfFiveMinute(t); + return time_zone.toStartOfFiveMinutes(t); } static inline UInt32 execute(Int32, const DateLUTImpl &) { diff --git a/src/Functions/registerFunctionsDateTime.cpp b/src/Functions/registerFunctionsDateTime.cpp index c2d8acac6dd..5f733b07281 100644 --- a/src/Functions/registerFunctionsDateTime.cpp +++ b/src/Functions/registerFunctionsDateTime.cpp @@ -27,7 +27,7 @@ void registerFunctionToLastDayOfMonth(FunctionFactory &); void registerFunctionToStartOfQuarter(FunctionFactory &); void registerFunctionToStartOfYear(FunctionFactory &); void registerFunctionToStartOfMinute(FunctionFactory &); -void registerFunctionToStartOfFiveMinute(FunctionFactory &); +void registerFunctionToStartOfFiveMinutes(FunctionFactory &); void registerFunctionToStartOfTenMinutes(FunctionFactory &); void registerFunctionToStartOfFifteenMinutes(FunctionFactory &); void registerFunctionToStartOfHour(FunctionFactory &); @@ -109,7 +109,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionToStartOfMillisecond(factory); registerFunctionToStartOfSecond(factory); registerFunctionToStartOfMinute(factory); - registerFunctionToStartOfFiveMinute(factory); + registerFunctionToStartOfFiveMinutes(factory); registerFunctionToStartOfTenMinutes(factory); registerFunctionToStartOfFifteenMinutes(factory); registerFunctionToStartOfHour(factory); diff --git a/src/Functions/toStartOfFiveMinute.cpp b/src/Functions/toStartOfFiveMinute.cpp index c154b8f2bc7..4fff3aa1bef 100644 --- a/src/Functions/toStartOfFiveMinute.cpp +++ b/src/Functions/toStartOfFiveMinute.cpp @@ -6,11 +6,11 @@ namespace DB { -using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething; +using FunctionToStartOfFiveMinutes = FunctionDateOrDateTimeToSomething; -void registerFunctionToStartOfFiveMinute(FunctionFactory & factory) +void registerFunctionToStartOfFiveMinutes(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 1863cd20bdd..356428a0b86 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1311,7 +1311,7 @@ "toSecond" "toStartOfDay" "toStartOfFifteenMinutes" -"toStartOfFiveMinute" +"toStartOfFiveMinutes" "toStartOfHour" "toStartOfInterval" "toStartOfISOYear" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 3f393aa6846..1bdaed5ee1b 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -507,7 +507,7 @@ "toStartOfFifteenMinutes" "emptyArrayUInt8" "dictGetUInt8" -"toStartOfFiveMinute" +"toStartOfFiveMinutes" "cbrt" "toStartOfMinute" "dictGet" diff --git a/tests/performance/date_time_long.xml b/tests/performance/date_time_long.xml index f210c807b12..240481969a8 100644 --- a/tests/performance/date_time_long.xml +++ b/tests/performance/date_time_long.xml @@ -16,7 +16,7 @@ toISOYear toStartOfMinute - toStartOfFiveMinute + toStartOfFiveMinutes toStartOfFifteenMinutes toStartOfHour toStartOfDay diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index df42e8f1b6e..c55542f59a6 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -107,7 +107,7 @@ toStartOfMinute 2019-02-06 19:57:00 2019-02-07 04:57:00 2019-02-06 11:57:00 -toStartOfFiveMinute +toStartOfFiveMinutes 2019-02-06 22:55:00 2019-02-06 20:55:00 2019-02-06 19:55:00 diff --git a/tests/queries/0_stateless/00189_time_zones_long.sql b/tests/queries/0_stateless/00189_time_zones_long.sql index ecc5f62ed1d..790b645e4f9 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.sql +++ b/tests/queries/0_stateless/00189_time_zones_long.sql @@ -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), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); -/* toStartOfFiveMinute */ +/* toStartOfFiveMinutes */ -SELECT 'toStartOfFiveMinute'; -SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); -SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); -SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); -SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo'); -SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); +SELECT 'toStartOfFiveMinutes'; +SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); +SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); +SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); +SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo'); +SELECT toString(toStartOfFiveMinutes(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); /* toStartOfTenMinutes */ diff --git a/tests/queries/0_stateless/00534_filimonov.data b/tests/queries/0_stateless/00534_filimonov.data index 827b3ee4fbc..f07d79ed1bf 100644 --- a/tests/queries/0_stateless/00534_filimonov.data +++ b/tests/queries/0_stateless/00534_filimonov.data @@ -199,7 +199,7 @@ SELECT toMinute(NULL); SELECT toStartOfDay(NULL); SELECT toMonday(NULL); SELECT toUInt16OrZero(NULL); -SELECT toStartOfFiveMinute(NULL); +SELECT toStartOfFiveMinutes(NULL); SELECT halfMD5(NULL); SELECT toStartOfHour(NULL); SELECT toRelativeYearNum(NULL); diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.python b/tests/queries/0_stateless/00921_datetime64_compatibility_long.python index 03cc088fd87..fc8a229268b 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.python +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.python @@ -28,7 +28,7 @@ toStartOfWeek(N, 'Europe/Moscow') toStartOfDay(N, 'Europe/Moscow') toStartOfHour(N, 'Europe/Moscow') toStartOfMinute(N, 'Europe/Moscow') -toStartOfFiveMinute(N, 'Europe/Moscow') +toStartOfFiveMinutes(N, 'Europe/Moscow') toStartOfTenMinutes(N, 'Europe/Moscow') toStartOfFifteenMinutes(N, 'Europe/Moscow') toStartOfInterval(N, INTERVAL 1 year, 'Europe/Moscow') diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 230bfa0c117..6dd1b4b9322 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -98,7 +98,7 @@ Code: 43 "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 "DateTime('Europe/Moscow')","2019-09-16 19:20:00" "DateTime('Europe/Moscow')","2019-09-16 19:20:00" diff --git a/tests/queries/0_stateless/01921_datatype_date32.reference b/tests/queries/0_stateless/01921_datatype_date32.reference index 8beaefbeb38..fc76ba92dc4 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.reference +++ b/tests/queries/0_stateless/01921_datatype_date32.reference @@ -104,7 +104,7 @@ 2021-01-01 -------toStartOfSecond--------- -------toStartOfMinute--------- --------toStartOfFiveMinute--------- +-------toStartOfFiveMinutes--------- -------toStartOfTenMinutes--------- -------toStartOfFifteenMinutes--------- -------toStartOfHour--------- diff --git a/tests/queries/0_stateless/01921_datatype_date32.sql b/tests/queries/0_stateless/01921_datatype_date32.sql index e01bdfeee8d..3ab026faab9 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.sql +++ b/tests/queries/0_stateless/01921_datatype_date32.sql @@ -46,8 +46,8 @@ select '-------toStartOfSecond---------'; select toStartOfSecond(x1) from t1; -- { serverError 43 } select '-------toStartOfMinute---------'; select toStartOfMinute(x1) from t1; -- { serverError 43 } -select '-------toStartOfFiveMinute---------'; -select toStartOfFiveMinute(x1) from t1; -- { serverError 43 } +select '-------toStartOfFiveMinutes---------'; +select toStartOfFiveMinutes(x1) from t1; -- { serverError 43 } select '-------toStartOfTenMinutes---------'; select toStartOfTenMinutes(x1) from t1; -- { serverError 43 } select '-------toStartOfFifteenMinutes---------'; diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.reference b/tests/queries/0_stateless/01958_partial_hour_timezone.reference index a86391b491c..026e35d3b80 100644 --- a/tests/queries/0_stateless/01958_partial_hour_timezone.reference +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.reference @@ -5,7 +5,7 @@ timeZoneOffset(t): -2670 formatDateTime(t, '%F %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 -toStartOfFiveMinute(t): 1970-06-17 07:35:00 +toStartOfFiveMinutes(t): 1970-06-17 07:35:00 toStartOfFifteenMinutes(t): 1970-06-17 07:30:00 toStartOfTenMinutes(t): 1970-06-17 07:30:00 toStartOfHour(t): 1970-06-17 07:00:00 diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.sql b/tests/queries/0_stateless/01958_partial_hour_timezone.sql index 9bcb03ea4f2..26350e55620 100644 --- a/tests/queries/0_stateless/01958_partial_hour_timezone.sql +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.sql @@ -5,7 +5,7 @@ SELECT toUnixTimestamp(t), formatDateTime(t, '%F %T', 'Africa/Monrovia'), toString(t, 'Africa/Monrovia'), toStartOfMinute(t), - toStartOfFiveMinute(t), + toStartOfFiveMinutes(t), toStartOfFifteenMinutes(t), toStartOfTenMinutes(t), toStartOfHour(t), diff --git a/tests/queries/0_stateless/02096_date_time_1970_saturation.sql b/tests/queries/0_stateless/02096_date_time_1970_saturation.sql index e0c401443a7..8cf60c95f73 100644 --- a/tests/queries/0_stateless/02096_date_time_1970_saturation.sql +++ b/tests/queries/0_stateless/02096_date_time_1970_saturation.sql @@ -12,7 +12,7 @@ select toStartOfYear(toDate(0)); select toStartOfYear(toDateTime(0, 'Europe/Moscow')); select toTime(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 toStartOfFifteenMinutes(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 toTime(toDateTime(0, 'America/Los_Angeles'), '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 toStartOfFifteenMinutes(toDateTime(0, 'America/Los_Angeles')); select toStartOfHour(toDateTime(0, 'America/Los_Angeles')); diff --git a/tests/testflows/datetime64_extended_range/requirements/requirements.md b/tests/testflows/datetime64_extended_range/requirements/requirements.md index 659057ebfdc..1546d4a2a46 100644 --- a/tests/testflows/datetime64_extended_range/requirements/requirements.md +++ b/tests/testflows/datetime64_extended_range/requirements/requirements.md @@ -54,7 +54,7 @@ * 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.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.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes) * 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) 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 -[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]. ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes diff --git a/tests/testflows/datetime64_extended_range/requirements/requirements.py b/tests/testflows/datetime64_extended_range/requirements/requirements.py index 1bbaf3547d9..de25c0e9c26 100644 --- a/tests/testflows/datetime64_extended_range/requirements/requirements.py +++ b/tests/testflows/datetime64_extended_range/requirements/requirements.py @@ -714,15 +714,15 @@ RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfSecond = Req num="4.2.0.4.21", ) -RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinute = Requirement( - name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute", +RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinutes = Requirement( + name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes", version="1.0", priority=None, group=None, type=None, uid=None, 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" "\n" ), @@ -1944,7 +1944,7 @@ SRS_010_ClickHouse_DateTime64_Extended_Range = Specification( num="4.2.0.4.21", ), Heading( - name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinute", + name="RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFiveMinutes", level=5, 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_toStartOfMinute, 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_toStartOfFifteenMinutes, 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.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.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.24 [RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfFifteenMinutes](#rqsrs-010datetime64extendedrangedatesandtimesfunctionstostartoffifteenminutes) * 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) 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 -[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]. ###### RQ.SRS-010.DateTime64.ExtendedRange.DatesAndTimesFunctions.toStartOfTenMinutes diff --git a/tests/testflows/datetime64_extended_range/tests/date_time_functions.py b/tests/testflows/datetime64_extended_range/tests/date_time_functions.py index 53add63e8f2..9038d0a8ee7 100644 --- a/tests/testflows/datetime64_extended_range/tests/date_time_functions.py +++ b/tests/testflows/datetime64_extended_range/tests/date_time_functions.py @@ -576,13 +576,13 @@ def to_start_of_minutes_interval(self, interval, func): @TestScenario @Requirements( - RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinute( + RQ_SRS_010_DateTime64_ExtendedRange_DatesAndTimesFunctions_toStartOfFiveMinutes( "1.0" ) ) -def to_start_of_five_minute(self): - """Check the toStartOfFiveMinute with DateTime64 extended range.""" - to_start_of_minutes_interval(interval=5, func="toStartOfFiveMinute") +def to_start_of_five_minutes(self): + """Check the toStartOfFiveMinutes with DateTime64 extended range.""" + to_start_of_minutes_interval(interval=5, func="toStartOfFiveMinutes") @TestScenario diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index 6455bc045d6..a60ebd0ebcf 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -249,7 +249,7 @@ std::map 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}, {"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}, - {"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}, {"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}, From 32721b001fa57f572d58c6ff5ec2625fd3b69cdd Mon Sep 17 00:00:00 2001 From: Memo Date: Fri, 22 Apr 2022 11:18:07 +0800 Subject: [PATCH 05/31] add alias --- src/Functions/toStartOfFiveMinute.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/toStartOfFiveMinute.cpp b/src/Functions/toStartOfFiveMinute.cpp index 4fff3aa1bef..9abb8968a6b 100644 --- a/src/Functions/toStartOfFiveMinute.cpp +++ b/src/Functions/toStartOfFiveMinute.cpp @@ -11,6 +11,7 @@ using FunctionToStartOfFiveMinutes = FunctionDateOrDateTimeToSomething(); + factory.registerAlias("ToStartOfFiveMinute", FunctionToStartOfFiveMinutes::name); } } From 25f4d76da3e720a4cc79531d4003372d111055cc Mon Sep 17 00:00:00 2001 From: Memo Date: Fri, 22 Apr 2022 11:24:44 +0800 Subject: [PATCH 06/31] change name --- .../{toStartOfFiveMinute.cpp => toStartOfFiveMinutes.cpp} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename src/Functions/{toStartOfFiveMinute.cpp => toStartOfFiveMinutes.cpp} (100%) diff --git a/src/Functions/toStartOfFiveMinute.cpp b/src/Functions/toStartOfFiveMinutes.cpp similarity index 100% rename from src/Functions/toStartOfFiveMinute.cpp rename to src/Functions/toStartOfFiveMinutes.cpp From 856412ea6e770eaffe659b6ec98fa177a2563dd0 Mon Sep 17 00:00:00 2001 From: Memo Date: Fri, 22 Apr 2022 11:27:24 +0800 Subject: [PATCH 07/31] fix wrong alias --- src/Functions/toStartOfFiveMinutes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/toStartOfFiveMinutes.cpp b/src/Functions/toStartOfFiveMinutes.cpp index 9abb8968a6b..b311c69d8c6 100644 --- a/src/Functions/toStartOfFiveMinutes.cpp +++ b/src/Functions/toStartOfFiveMinutes.cpp @@ -11,7 +11,7 @@ using FunctionToStartOfFiveMinutes = FunctionDateOrDateTimeToSomething(); - factory.registerAlias("ToStartOfFiveMinute", FunctionToStartOfFiveMinutes::name); + factory.registerAlias("toStartOfFiveMinute", FunctionToStartOfFiveMinutes::name); } } From 357dd7354ca6987d669e9b551180cd96887cfe23 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 22 Apr 2022 11:39:36 +0000 Subject: [PATCH 08/31] Fix no-parallel tag --- tests/queries/0_stateless/01161_all_system_tables.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index a5ed2ea7e6d..aa55194c004 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash - # Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 48dddc53b9a7b918753d68533e8e1136e44032bd Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 22 Apr 2022 17:32:53 +0300 Subject: [PATCH 09/31] Update 01161_all_system_tables.sh --- tests/queries/0_stateless/01161_all_system_tables.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index aa55194c004..9988c1f3625 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -1,5 +1,8 @@ #!/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) # shellcheck source=../shell_config.sh From e49323ca9c6710aa87e7f7c6888c7263dec724a6 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Sat, 23 Apr 2022 14:23:33 +0300 Subject: [PATCH 10/31] Update 01161_all_system_tables.sh --- tests/queries/0_stateless/01161_all_system_tables.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index 9988c1f3625..e542a048953 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -14,9 +14,10 @@ LIMIT=10000 function run_selects() { + # NOTE sistem.dist1 is from 01129_dict_get_join_lose_constness, let's ignore it thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables - WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' + WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' and name != 'dict1' AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") for t in "${tables_arr[@]}" From fa657d1d7fe58fff218ca4c02f1298d24c4b3fc7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 Apr 2022 22:36:27 +0300 Subject: [PATCH 11/31] bash-completion: add completion for send_logs_level Signed-off-by: Azat Khuzhin --- .../completions/clickhouse-bootstrap | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/programs/bash-completion/completions/clickhouse-bootstrap b/programs/bash-completion/completions/clickhouse-bootstrap index 793d47501dd..98fcd68db16 100644 --- a/programs/bash-completion/completions/clickhouse-bootstrap +++ b/programs/bash-completion/completions/clickhouse-bootstrap @@ -15,6 +15,17 @@ shopt -s extglob export _CLICKHOUSE_COMPLETION_LOADED=1 +CLICKHOUSE_logs_level=( + none + fatal + error + warning + information + debug + trace + test +) + CLICKHOUSE_QueryProcessingStage=( complete fetch_columns @@ -113,6 +124,10 @@ function _complete_for_clickhouse_generic_bin_impl() COMPREPLY=( $(compgen -W "${CLICKHOUSE_QueryProcessingStage[*]}" -- "$cur") ) return 1 ;; + --send_logs_level) + COMPREPLY=( $(compgen -W "${CLICKHOUSE_logs_level[*]}" -- "$cur") ) + return 1 + ;; --format|--input-format|--output-format) COMPREPLY=( $(compgen -W "${CLICKHOUSE_Format[*]}" -- "$cur") ) return 1 From c9aac5dd026e7455f1b9bcc8d0b04e72e851be4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Apr 2022 22:38:59 +0200 Subject: [PATCH 12/31] Fix strange whitespace (or I do not know YAML) --- packages/clickhouse-client.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/packages/clickhouse-client.yaml b/packages/clickhouse-client.yaml index 2a1389b6625..5e53090b581 100644 --- a/packages/clickhouse-client.yaml +++ b/packages/clickhouse-client.yaml @@ -18,9 +18,9 @@ conflicts: maintainer: "ClickHouse Dev Team " description: | 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. - This package provides clickhouse-client , clickhouse-local and clickhouse-benchmark + This package provides clickhouse-client, clickhouse-local and clickhouse-benchmark. overrides: deb: From f27c4d5fd45f7a599f9f5e5358db0d10a78fe0a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Apr 2022 03:25:00 +0300 Subject: [PATCH 13/31] Update 01958_partial_hour_timezone.reference --- tests/queries/0_stateless/01958_partial_hour_timezone.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.reference b/tests/queries/0_stateless/01958_partial_hour_timezone.reference index 026e35d3b80..ab2f7221390 100644 --- a/tests/queries/0_stateless/01958_partial_hour_timezone.reference +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.reference @@ -5,7 +5,7 @@ timeZoneOffset(t): -2670 formatDateTime(t, '%F %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 -toStartOfFiveMinutes(t): 1970-06-17 07:35:00 +toStartOfFiveMinutes(t): 1970-06-17 07:35:00 toStartOfFifteenMinutes(t): 1970-06-17 07:30:00 toStartOfTenMinutes(t): 1970-06-17 07:30:00 toStartOfHour(t): 1970-06-17 07:00:00 From 1eded7c32b3dde47cc63aa59ff7b1efc3a1443bf Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Sun, 24 Apr 2022 20:02:47 -0600 Subject: [PATCH 14/31] Delete date-time-functions.md --- .../functions/date-time-functions.md | 450 ------------------ 1 file changed, 450 deletions(-) delete mode 100644 docs/ja/sql-reference/functions/date-time-functions.md diff --git a/docs/ja/sql-reference/functions/date-time-functions.md b/docs/ja/sql-reference/functions/date-time-functions.md deleted file mode 100644 index cac2eaa3d44..00000000000 --- a/docs/ja/sql-reference/functions/date-time-functions.md +++ /dev/null @@ -1,450 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 39 -toc_title: "\u65E5\u4ED8\u3068\u6642\u523B\u306E\u64CD\u4F5C" ---- - -# 日付と時刻を操作するための関数 {#functions-for-working-with-dates-and-times} - -タイムゾーンのサポート - -タイムゾーンに論理的に使用される日付と時刻を操作するためのすべての関数は、オプションのタイムゾーン引数を受け入れることができます。 例:アジア/エカテリンブルク。 この場合、ローカル(既定)のタイムゾーンではなく、指定されたタイムゾーンを使用します。 - -``` sql -SELECT - toDateTime('2016-06-15 23:00:00') AS time, - toDate(time) AS date_local, - toDate(time, 'Asia/Yekaterinburg') AS date_yekat, - toString(time, 'US/Samoa') AS time_samoa -``` - -``` text -┌────────────────time─┬─date_local─┬─date_yekat─┬─time_samoa──────────┐ -│ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-16 │ 2016-06-15 09:00:00 │ -└─────────────────────┴────────────┴────────────┴─────────────────────┘ -``` - -UTCと時間数が異なるタイムゾーンのみがサポートされます。 - -## トティムゾーン {#totimezone} - -時刻または日付と時刻を指定したタイムゾーンに変換します。 - -## toYear {#toyear} - -日付または日付と時刻を、年番号(AD)を含むUInt16番号に変換します。 - -## トクアーター {#toquarter} - -Dateまたはdate with timeを、四半期番号を含むUInt8番号に変換します。 - -## トモンス {#tomonth} - -Dateまたはdate with timeを、月番号(1-12)を含むUInt8番号に変換します。 - -## 今日の年 {#todayofyear} - -日付または日付と時刻を、年の日の番号を含むUInt16番号(1-366)に変換します。 - -## 今日の月 {#todayofmonth} - -日付または日付と時刻を、月の日の番号を含むUInt8番号(1-31)に変換します。 - -## 今日の週 {#todayofweek} - -Dateまたはdate with timeを、曜日の番号を含むUInt8番号に変換します(月曜日は1、日曜日は7)。 - -## toHour {#tohour} - -時刻を持つ日付を、24時間(0-23)の時間数を含むUInt8数値に変換します。 -This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). - -## トミヌテ {#tominute} - -時刻を持つ日付を、時間の分の数を含むUInt8番号(0-59)に変換します。 - -## toSecond {#tosecond} - -日付と時刻を、分(0-59)の秒数を含むUInt8数値に変換します。 -うるう秒は考慮されません。 - -## toUnixTimestamp {#to-unix-timestamp} - -DateTime引数の場合:値を内部の数値表現(Unixタイムスタンプ)に変換します。 -文字列引数の場合:タイムゾーンに従って文字列からdatetimeを解析し(オプションの第二引数、サーバーのタイムゾーンがデフォルトで使用されます)、対応するunixタ -Date引数の場合:動作は指定されていません。 - -**構文** - -``` sql -toUnixTimestamp(datetime) -toUnixTimestamp(str, [timezone]) -``` - -**戻り値** - -- Unixタイムスタンプを返す。 - -タイプ: `UInt32`. - -**例** - -クエリ: - -``` sql -SELECT toUnixTimestamp('2017-11-05 08:07:47', 'Asia/Tokyo') AS unix_timestamp -``` - -結果: - -``` text -┌─unix_timestamp─┐ -│ 1509836867 │ -└────────────────┘ -``` - -## トスタート {#tostartofyear} - -日付または日付を年の最初の日に切り捨てます。 -日付を返します。 - -## トスタートフィソイヤー {#tostartofisoyear} - -日付または日付をiso年の最初の日に切り捨てます。 -日付を返します。 - -## toStartOfQuarter {#tostartofquarter} - -日付または日付を四半期の最初の日に切り捨てます。 -四半期の最初の日は、1月、1月、1月、または1月のいずれかです。 -日付を返します。 - -## トスタートモンス {#tostartofmonth} - -日付または日付を月の最初の日に切り捨てます。 -日付を返します。 - -!!! attention "注意" - 不正な日付を解析する動作は、実装固有です。 ClickHouseはゼロの日付を返すか、例外をスローするか “natural” オーバーフロー - -## トモンデイ {#tomonday} - -日付または日付を時刻とともに最も近い月曜日に切り捨てます。 -日付を返します。 - -## トスタートフィーク(t\[,モード\]) {#tostartofweektmode} - -日付または時刻の日付を、最も近い日曜日または月曜日にモードで切り捨てます。 -日付を返します。 -Mode引数は、toWeek()のmode引数とまったく同じように動作します。 単一引数構文では、モード値0が使用されます。 - -## トスタートフデイ {#tostartofday} - -日付を時刻とともに日の始まりまで切り捨てます。 - -## トスタートフール {#tostartofhour} - -日付と時刻を時間の開始まで切り捨てます。 - -## トスタートフミニュート {#tostartofminute} - -日付と時刻を分の開始まで切り捨てます。 - -## トスタートオフィブミニュート {#tostartoffiveminutes} - -日付と時刻を切り捨てます。 - -## toStartOfTenMinutes {#tostartoftenminutes} - -日付と時刻を切り捨てて、十分間隔の開始まで切り捨てます。 - -## トスタートフィフテンミニュート {#tostartoffifteenminutes} - -日付を時刻とともに切り捨てて、十分間隔の開始まで切り捨てます。 - -## トスタートオフインターバル(time_or_data,区間x単位\[,time_zone\]) {#tostartofintervaltime-or-data-interval-x-unit-time-zone} - -これは、名前付きの他の関数の一般化です `toStartOf*`. 例えば, -`toStartOfInterval(t, INTERVAL 1 year)` と同じを返します `toStartOfYear(t)`, -`toStartOfInterval(t, INTERVAL 1 month)` と同じを返します `toStartOfMonth(t)`, -`toStartOfInterval(t, INTERVAL 1 day)` と同じを返します `toStartOfDay(t)`, -`toStartOfInterval(t, INTERVAL 15 minute)` と同じを返します `toStartOfFifteenMinutes(t)` 等。 - -## トータイム {#totime} - -時刻を保持しながら、時刻を持つ日付を特定の固定日付に変換します。 - -## toRelativeYearNum {#torelativeyearnum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる年の番号に変換します。 - -## toRelativeQuarterNum {#torelativequarternum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる四半期の数値に変換します。 - -## トレラティブモンスヌム {#torelativemonthnum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる月の番号に変換します。 - -## toRelativeWeekNum {#torelativeweeknum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる週の数に変換します。 - -## toRelativeDayNum {#torelativedaynum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる日付の数値に変換します。 - -## toRelativeHourNum {#torelativehournum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる時間の数に変換します。 - -## トレラティブミノテン {#torelativeminutenum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる分の数に変換します。 - -## toRelativeSecondNum {#torelativesecondnum} - -時刻または日付を持つ日付を、過去の特定の固定点から始まる秒数に変換します。 - -## toISOYear {#toisoyear} - -Dateまたはdate with timeをISO年番号を含むUInt16番号に変換します。 - -## toISOWeek {#toisoweek} - -日付または日付と時刻をISO週番号を含むUInt8番号に変換します。 - -## トウィーク(日付\[,モード\]) {#toweekdatemode} - -この関数は、dateまたはdatetimeの週番号を返します。 二引数形式のtoWeek()を使用すると、週が日曜日か月曜日か、戻り値が0から53または1から53の範囲であるかどうかを指定できます。 引数modeを省略すると、デフォルトのモードは0になります。 -`toISOWeek()`と等価な互換性関数です `toWeek(date,3)`. -次の表は、mode引数の動作方法を示しています。 - -| モード | 週の最初の日 | 範囲 | Week 1 is the first week … | -|--------|--------------|------|----------------------------| -| 0 | 日曜日 | 0-53 | 今年の日曜日と | -| 1 | 月曜日 | 0-53 | 今年は4日以上 | -| 2 | 日曜日 | 1-53 | 今年の日曜日と | -| 3 | 月曜日 | 1-53 | 今年は4日以上 | -| 4 | 日曜日 | 0-53 | 今年は4日以上 | -| 5 | 月曜日 | 0-53 | 今年の月曜日と | -| 6 | 日曜日 | 1-53 | 今年は4日以上 | -| 7 | 月曜日 | 1-53 | 今年の月曜日と | -| 8 | 日曜日 | 1-53 | 1を含む | -| 9 | 月曜日 | 1-53 | 1を含む | - -の意味を持つモード値の場合 “with 4 or more days this year,” 週はISO8601:1988に従って番号が付けられます: - -- 1月を含む週が新年に4日以上ある場合、それは1週である。 - -- それ以外の場合は、前年の最後の週であり、次の週は1週です。 - -の意味を持つモード値の場合 “contains January 1”、1月を含む週は1週である。 たとえそれが一日だけ含まれていても、その週が含まれている新年の日数は問題ではありません。 - -``` sql -toWeek(date, [, mode][, Timezone]) -``` - -**パラメータ** - -- `date` – Date or DateTime. -- `mode` – Optional parameter, Range of values is \[0,9\], default is 0. -- `Timezone` – Optional parameter, it behaves like any other conversion function. - -**例** - -``` sql -SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS week1, toWeek(date,9) AS week9; -``` - -``` text -┌───────date─┬─week0─┬─week1─┬─week9─┐ -│ 2016-12-27 │ 52 │ 52 │ 1 │ -└────────────┴───────┴───────┴───────┘ -``` - -## toYearWeek(日付\[,モード\]) {#toyearweekdatemode} - -日付の年と週を返します。 結果の年は、年の最初と最後の週のdate引数の年とは異なる場合があります。 - -Mode引数は、toWeek()のmode引数とまったく同じように動作します。 単一引数構文では、モード値0が使用されます。 - -`toISOYear()`と等価な互換性関数です `intDiv(toYearWeek(date,3),100)`. - -**例** - -``` sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; -``` - -``` text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ -``` - -## さて {#now} - -ゼロの引数を受け取り、要求の実行のいずれかの瞬間に現在の時刻を返します。 -この関数は、要求が完了するまでに時間がかかった場合でも、定数を返します。 - -## 今日 {#today} - -ゼロの引数を受け取り、要求の実行のいずれかの瞬間に現在の日付を返します。 -と同じ ‘toDate(now())’. - -## 昨日 {#yesterday} - -ゼロの引数を受け取り、要求の実行のいずれかの瞬間に昨日の日付を返します。 -と同じ ‘today() - 1’. - -## タイムスロット {#timeslot} - -時間を半分の時間に丸めます。 -この機能はYandexに固有のものです。Metricaは、トラッキングタグが単一のユーザーの連続したページビューを表示する場合、セッションを二つのセッションに分割するための最小時間であるため、こ つまり、タプル(タグID、ユーザー ID、およびタイムスロット)を使用して、対応するセッションに含まれるページビューを検索できます。 - -## トイヤイム {#toyyyymm} - -日付または日付と時刻を、年と月の番号(YYYY\*100+MM)を含むUInt32番号に変換します。 - -## トイヤイム {#toyyyymmdd} - -日付または日付と時刻を、年と月の番号(YYYY\*10000+MM\*100+DD)を含むUInt32番号に変換します。 - -## トイヤイム {#toyyyymmddhhmmss} - -日付または日付と時刻を、年と月の番号(YYYY\*10000000000+MM\*100000000+DD\*1000000+hh\*10000+mm\*100+ss)を含むUInt64番号に変換します。 - -## addYears,addMonths,addWeeks,addDays,addHours,addMinutes,addSeconds,addQuarters {#addyears-addmonths-addweeks-adddays-addhours-addminutes-addseconds-addquarters} - -関数は、日付/日付時刻に日付/日付時刻の間隔を追加し、日付/日付時刻を返します。 例えば: - -``` sql -WITH - toDate('2018-01-01') AS date, - toDateTime('2018-01-01 00:00:00') AS date_time -SELECT - addYears(date, 1) AS add_years_with_date, - addYears(date_time, 1) AS add_years_with_date_time -``` - -``` text -┌─add_years_with_date─┬─add_years_with_date_time─┐ -│ 2019-01-01 │ 2019-01-01 00:00:00 │ -└─────────────────────┴──────────────────────────┘ -``` - -## subtruttyears、subtrutmonths、subtrutweeks、subtrutdays、subtrutthours、subtrutminutes、subtrutseconds、subtrutquarters {#subtractyears-subtractmonths-subtractweeks-subtractdays-subtracthours-subtractminutes-subtractseconds-subtractquarters} - -関数Date/DateTime間隔をDate/DateTimeに減算し、Date/DateTimeを返します。 例えば: - -``` sql -WITH - toDate('2019-01-01') AS date, - toDateTime('2019-01-01 00:00:00') AS date_time -SELECT - subtractYears(date, 1) AS subtract_years_with_date, - subtractYears(date_time, 1) AS subtract_years_with_date_time -``` - -``` text -┌─subtract_years_with_date─┬─subtract_years_with_date_time─┐ -│ 2018-01-01 │ 2018-01-01 00:00:00 │ -└──────────────────────────┴───────────────────────────────┘ -``` - -## dateDiff {#datediff} - -Date値またはDateTime値の差を返します。 - -**構文** - -``` sql -dateDiff('unit', startdate, enddate, [timezone]) -``` - -**パラメータ** - -- `unit` — Time unit, in which the returned value is expressed. [文字列](../syntax.md#syntax-string-literal). - - Supported values: - - | unit | - | ---- | - |second | - |minute | - |hour | - |day | - |week | - |month | - |quarter | - |year | - -- `startdate` — The first time value to compare. [日付](../../sql-reference/data-types/date.md) または [DateTime](../../sql-reference/data-types/datetime.md). - -- `enddate` — The second time value to compare. [日付](../../sql-reference/data-types/date.md) または [DateTime](../../sql-reference/data-types/datetime.md). - -- `timezone` — Optional parameter. If specified, it is applied to both `startdate` と `enddate`. 指定されていない場合、 `startdate` と `enddate` 使用されます。 それらが同じでない場合、結果は未指定です。 - -**戻り値** - -の違い `startdate` と `enddate` で表される。 `unit`. - -タイプ: `int`. - -**例** - -クエリ: - -``` sql -SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00')); -``` - -結果: - -``` text -┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐ -│ 25 │ -└────────────────────────────────────────────────────────────────────────────────────────┘ -``` - -## タイムスロット(StartTime,Duration,\[,Size\]) {#timeslotsstarttime-duration-size} - -で始まる時間間隔の場合 ‘StartTime’ そして続けるために ‘Duration’ これは、この区間の点で構成される時間内の瞬間の配列を返します。 ‘Size’ 秒で。 ‘Size’ 定数UInt32で、既定では1800に設定されます。 -例えば, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`. -これは、対応するセッションでページビューを検索するために必要です。 - -## formatDateTime(時刻,Format\[,タイムゾーン\]) {#formatdatetime} - -Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. - -サポートされている形式の修飾子: -(“Example” 列に時間の書式設定の結果が表示されます `2018-01-02 22:33:44`) - -| 修飾子 | 説明 | 例 | -|--------|-------------------------------------------------|------------| -| %C | 年を100で割り、整数に切り捨てます(00-99) | 20 | -| %d | 月の日、ゼロパッド(01-31) | 02 | -| %D | %M/%d/%yに相当する短いMM/DD/YYの日付 | 01/02/18 | -| %e | 月の日,スペース埋め(1-31) | 2 | -| %F | %Y-%m-%dに相当します | 2018-01-02 | -| %H | 24時間形式(00-23時) | 22 | -| %I | 12時間形式(01-12) | 10 | -| %j | 年の日(001-366) | 002 | -| %m | 十進数としての月(01-12) | 01 | -| %M | 分(00-59) | 33 | -| %n | 改行文字(") | | -| %p | AMまたはPMの指定 | PM | -| %R | 24時間HH:MM時間、%H:%Mに相当 | 22:33 | -| %S | 第二(00-59) | 44 | -| %t | 横タブ文字(') | | -| %T | ISO8601時間形式(HH:MM:SS)、%H:%M:%Sに相当します | 22:33:44 | -| %u | ISO8601平日as番号、月曜日as1(1-7) | 2 | -| %V | ISO8601週番号(01-53) | 01 | -| %w | 曜日を十進数とし、日曜日を0(0-6)とします) | 2 | -| %y | 年,最後の二桁(00-99) | 18 | -| %Y | 年 | 2018 | -| %% | %記号 | % | - -[元の記事](https://clickhouse.com/docs/en/query_language/functions/date_time_functions/) From 6df54611a9da2fde3f76bb04798248d89a7af0b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Apr 2022 15:01:56 +0300 Subject: [PATCH 15/31] Log which taskstats provider is in use Signed-off-by: Azat Khuzhin v2: add missing __builtin_unreachable() for gcc built [1] [1]: https://s3.amazonaws.com/clickhouse-builds/36351/99e72a960325cc1fb3cd3d6a1b56cf2b1cc2d733/binary_gcc/build_log.log (cherry picked from commit 2be022cb56b4d8f65f02393c4c6069f409ace4e8) --- programs/server/Server.cpp | 7 ++++++- src/Common/ThreadProfileEvents.cpp | 14 ++++++++++++++ src/Common/ThreadProfileEvents.h | 18 ++++++++++-------- 3 files changed, 30 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 786fff9b0e4..085018fbb87 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1507,7 +1507,8 @@ int Server::main(const std::vector & /*args*/) } #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," " neither clickhouse-server process has CAP_NET_ADMIN capability." @@ -1518,6 +1519,10 @@ int Server::main(const std::vector & /*args*/) " It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.", executable_path); } + else + { + LOG_INFO(log, "Tasks stats provider: {}", TasksStatsCounters::metricsProviderString(tasks_stats_provider)); + } if (!hasLinuxCapability(CAP_SYS_NICE)) { diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index e28e5d3e85e..1a7b4d88322 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -67,6 +67,20 @@ namespace ProfileEvents 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() { return findBestAvailableProvider() != MetricsProvider::None; diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index befd759c71b..0f2ad7dfd74 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -176,7 +176,17 @@ extern PerfEventsCounters current_thread_counters; class TasksStatsCounters { public: + enum class MetricsProvider + { + None, + Procfs, + Netlink, + }; + + static const char * metricsProviderString(MetricsProvider provider); static bool checkIfAvailable(); + static MetricsProvider findBestAvailableProvider(); + static std::unique_ptr create(UInt64 tid); void reset(); @@ -186,16 +196,8 @@ private: ::taskstats stats; //-V730_NOINIT std::function<::taskstats()> stats_getter; - enum class MetricsProvider - { - None, - Procfs, - Netlink - }; - explicit TasksStatsCounters(UInt64 tid, MetricsProvider provider); - static MetricsProvider findBestAvailableProvider(); static void incrementProfileEvents(const ::taskstats & prev, const ::taskstats & curr, ProfileEvents::Counters & profile_events); }; From d161b798a893315454a3cf834c5d33a4b63b15e5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Apr 2022 13:50:06 +0300 Subject: [PATCH 16/31] Strict taskstats parser This will unhide possible issues (since sometimes statistics is inaccurate). Also before this patch TASKSTATS_TYPE_AGGR_TGID had been handled, while it not correct to take into account, but it was not requested anyway. Signed-off-by: Azat Khuzhin (cherry picked from commit bfa65b651a7e7696b13315a85d16688c1e957b99) --- src/Common/TaskStatsInfoGetter.cpp | 36 ++++++++++++++---------------- 1 file changed, 17 insertions(+), 19 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index f2ef0f6d2aa..a74b4c2b89b 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -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)); - for (const NetlinkMessage::Attribute * attr = &answer.payload.attribute; - attr < answer.end(); - attr = attr->next()) - { - 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(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; - } - } - } - } + const NetlinkMessage::Attribute * attr = &answer.payload.attribute; + if (attr->header.nla_type != TASKSTATS_TYPE_AGGR_PID) + throw Exception("Expected TASKSTATS_TYPE_AGGR_PID", ErrorCodes::NETLINK_ERROR); - 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(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); } From aaa222821cbfce468643330c45417fbac8b1d3cf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Apr 2022 22:59:11 +0300 Subject: [PATCH 17/31] tests: simplify 01268_procfs_metrics Signed-off-by: Azat Khuzhin --- .../01268_procfs_metrics.reference | 4 +- .../0_stateless/01268_procfs_metrics.sh | 46 ++----------------- 2 files changed, 5 insertions(+), 45 deletions(-) diff --git a/tests/queries/0_stateless/01268_procfs_metrics.reference b/tests/queries/0_stateless/01268_procfs_metrics.reference index e9ec4d01307..7b46bb2d8ec 100644 --- a/tests/queries/0_stateless/01268_procfs_metrics.reference +++ b/tests/queries/0_stateless/01268_procfs_metrics.reference @@ -1,2 +1,2 @@ -1 -Test OK +OSCPUVirtualTimeMicroseconds +OSReadChars diff --git a/tests/queries/0_stateless/01268_procfs_metrics.sh b/tests/queries/0_stateless/01268_procfs_metrics.sh index c1697edd632..75a8d72ae4c 100755 --- a/tests/queries/0_stateless/01268_procfs_metrics.sh +++ b/tests/queries/0_stateless/01268_procfs_metrics.sh @@ -8,46 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function read_numbers_func() -{ - $CLICKHOUSE_CLIENT -q " - SELECT * FROM numbers(600000000) FORMAT Null SETTINGS max_threads = 1 - "; -} - - -function show_processes_func() -{ - while true; do - 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" +$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('${BASH_SOURCE[0]}', 'LineAsString') FORMAT Null" |& { + grep -F -o -e OSCPUVirtualTimeMicroseconds -e OSReadChars +} | sort | uniq From 35613e3010ca18e035f0cabf9cda7331352b284f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Apr 2022 08:19:55 +0300 Subject: [PATCH 18/31] tests: adopt 01268_procfs_metrics for difference between proc and taskstruct Signed-off-by: Azat Khuzhin --- .../0_stateless/01268_procfs_metrics.reference | 2 +- tests/queries/0_stateless/01268_procfs_metrics.sh | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01268_procfs_metrics.reference b/tests/queries/0_stateless/01268_procfs_metrics.reference index 7b46bb2d8ec..7feb2a6cd61 100644 --- a/tests/queries/0_stateless/01268_procfs_metrics.reference +++ b/tests/queries/0_stateless/01268_procfs_metrics.reference @@ -1,2 +1,2 @@ -OSCPUVirtualTimeMicroseconds OSReadChars +OSCPUVirtualTimeMicroseconds diff --git a/tests/queries/0_stateless/01268_procfs_metrics.sh b/tests/queries/0_stateless/01268_procfs_metrics.sh index 75a8d72ae4c..959d922d801 100755 --- a/tests/queries/0_stateless/01268_procfs_metrics.sh +++ b/tests/queries/0_stateless/01268_procfs_metrics.sh @@ -8,6 +8,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('${BASH_SOURCE[0]}', 'LineAsString') FORMAT Null" |& { - grep -F -o -e OSCPUVirtualTimeMicroseconds -e OSReadChars -} | sort | uniq +# NOTE: netlink taskstruct interface uses rounding to 1KB [1], so we cannot use ${BASH_SOURCE[0]} +# +# [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101 +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 +# 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 +exit 0 From 278101fc0cb3ad97c65431c948a5eca99f1e68c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Apr 2022 08:47:45 +0300 Subject: [PATCH 19/31] Add a comment for taskstruct netlink interface rounding to KiB Signed-off-by: Azat Khuzhin --- src/Common/TaskStatsInfoGetter.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Common/TaskStatsInfoGetter.h b/src/Common/TaskStatsInfoGetter.h index 08e35757212..66655d7ad0d 100644 --- a/src/Common/TaskStatsInfoGetter.h +++ b/src/Common/TaskStatsInfoGetter.h @@ -10,6 +10,11 @@ namespace DB { /// 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 { public: From 7246101f61055bf379f0d6aaf205bea16ec661c4 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 25 Apr 2022 13:51:33 +0300 Subject: [PATCH 20/31] better fix for tests --- .../queries/0_stateless/01129_dict_get_join_lose_constness.sql | 2 ++ tests/queries/0_stateless/01161_all_system_tables.sh | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql b/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql index 09659145977..4f7197dc5ce 100644 --- a/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql +++ b/tests/queries/0_stateless/01129_dict_get_join_lose_constness.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP DICTIONARY IF EXISTS system.dict1; CREATE DICTIONARY IF NOT EXISTS system.dict1 diff --git a/tests/queries/0_stateless/01161_all_system_tables.sh b/tests/queries/0_stateless/01161_all_system_tables.sh index e542a048953..9988c1f3625 100755 --- a/tests/queries/0_stateless/01161_all_system_tables.sh +++ b/tests/queries/0_stateless/01161_all_system_tables.sh @@ -14,10 +14,9 @@ LIMIT=10000 function run_selects() { - # NOTE sistem.dist1 is from 01129_dict_get_join_lose_constness, let's ignore it thread_num=$1 readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables - WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' and name != 'dict1' + WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper' and name!='merge_tree_metadata_cache' AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num") for t in "${tables_arr[@]}" From 824335ca3394550384f817d19987c66eb97284c4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 25 Apr 2022 14:42:08 +0200 Subject: [PATCH 21/31] Filter out sychronized events globbaly --- .../cancel_and_rerun_workflow_lambda/app.py | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 27aff07fc0b..9fe11880f3d 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -108,7 +108,8 @@ def _exec_get_with_retry(url): 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( WorkflowDescription( run_id=workflow["id"], + head_sha=workflow["head_sha"], status=workflow["status"], rerun_url=workflow["rerun_url"], cancel_url=workflow["cancel_url"], @@ -170,11 +172,9 @@ def get_workflows_description_for_pull_request( return workflow_descriptions -def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: - pull_request_event = event_data["pull_request"] +def get_workflow_description_fallback(pull_request_event) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] 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") # Fallback for a case of an already deleted branch and no workflows received 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 ] - 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 = [ WorkflowDescription( run_id=wf["id"], + head_sha=wf["head_sha"], status=wf["status"], rerun_url=wf["rerun_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}") return WorkflowDescription( run_id=workflow["id"], + head_sha=workflow["head_sha"], status=workflow["status"], rerun_url=workflow["rerun_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") workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = ( - workflow_descriptions or get_workflow_description_fallback(event_data) + workflow_descriptions or get_workflow_description_fallback(pull_request) ) urls_to_cancel = [] for workflow_description in workflow_descriptions: @@ -296,13 +292,14 @@ def main(event): print("PR is synchronized, going to stop old actions") workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = ( - workflow_descriptions or get_workflow_description_fallback(event_data) + workflow_descriptions or get_workflow_description_fallback(pull_request) ) urls_to_cancel = [] for workflow_description in workflow_descriptions: if ( workflow_description.status != "completed" and workflow_description.conclusion != "cancelled" + and workflow_description.head_sha != pull_request["head"]["sha"] ): urls_to_cancel.append(workflow_description.cancel_url) 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") workflow_descriptions = get_workflows_description_for_pull_request(pull_request) workflow_descriptions = ( - workflow_descriptions or get_workflow_description_fallback(event_data) + workflow_descriptions or get_workflow_description_fallback(pull_request) ) if not workflow_descriptions: print("Not found any workflows") From 8440fec3dba4814dd8f2c5edebfd4d43e0fd4927 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 31 Mar 2022 12:54:25 +0300 Subject: [PATCH 22/31] Fix coding alignment in MutateTask Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 73b703aa9ca..e958fc49d00 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -74,7 +74,7 @@ static void splitMutationCommands( mutated_columns.emplace(column_name); 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) { From 615d35e233f94cdbf1443aa542930fbd9064e2bf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 31 Mar 2022 12:43:43 +0300 Subject: [PATCH 23/31] Remove dead function MergeTreeDataMergerMutator::splitMutationCommands() Signed-off-by: Azat Khuzhin --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 98 ------------------- .../MergeTree/MergeTreeDataMergerMutator.h | 10 -- 2 files changed, 108 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 23e4e91a74d..4b904ac298f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -640,104 +640,6 @@ size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData:: return static_cast(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 MergeTreeDataMergerMutator::getColumnsForNewDataPart( MergeTreeData::DataPartPtr source_part, diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 9a60e4c6078..dcbdda14704 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -146,16 +146,6 @@ private: friend class MutateTask; 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 getColumnsForNewDataPart( MergeTreeData::DataPartPtr source_part, From f4df88a9383b87377faa3e3450bb85ebbce6a4db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 31 Mar 2022 13:21:01 +0300 Subject: [PATCH 24/31] Add a comment for ColumnsDescription::subcolumns Signed-off-by: Azat Khuzhin --- src/Storages/ColumnsDescription.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index d3d6f7f2ff5..4e1d60db2f2 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -217,6 +217,13 @@ public: private: ColumnsContainer columns; + /// Subcolumns is not nested columns. + /// + /// Example of subcolumns: + /// - .size0 for Array + /// - .null for Nullable + /// + /// While nested columns is just foo.bar SubcolumnsContainter subcolumns; void modifyColumnOrder(const String & column_name, const String & after_column, bool first); From 8b544e26d34929618d54e87dbf30e0ea28957ad3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 31 Mar 2022 12:46:21 +0300 Subject: [PATCH 25/31] Move some functions from MergeTreeDataMergerMutator to MutateTask Signed-off-by: Azat Khuzhin --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 118 ----------------- .../MergeTree/MergeTreeDataMergerMutator.h | 11 -- src/Storages/MergeTree/MutateTask.cpp | 122 +++++++++++++++++- 3 files changed, 120 insertions(+), 131 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4b904ac298f..cd0ca27d131 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -640,122 +640,4 @@ size_t MergeTreeDataMergerMutator::estimateNeededDiskSpace(const MergeTreeData:: return static_cast(res * DISK_USAGE_COEFFICIENT_TO_RESERVE); } -std::pair -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; -} - - } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index dcbdda14704..f99f3d2e70d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -146,17 +146,6 @@ private: friend class MutateTask; friend class MergeTask; - /// Get the columns list of the resulting part in the same order as storage_columns. - static std::pair 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 : /** 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. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e958fc49d00..c71d2a89e19 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -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 +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. static MergeTreeIndices getIndicesForNewDataPart( @@ -1337,7 +1455,7 @@ bool MutateTask::prepare() /// It shouldn't be changed by mutation. 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->getSerializationInfos(), ctx->commands_for_part); @@ -1357,7 +1475,7 @@ bool MutateTask::prepare() ctx->execute_ttl_type = ExecuteTTLType::NONE; 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 /// TODO We can materialize compact part without copying data From 01438e106026240624d2f8a6a69e9a8ac31263b8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Apr 2022 18:26:47 +0300 Subject: [PATCH 26/31] tests: properly wait for queries in 01502_long_log_tinylog_deadlock_race Signed-off-by: Azat Khuzhin --- .../01502_long_log_tinylog_deadlock_race.sh | 78 +++++++++---------- 1 file changed, 36 insertions(+), 42 deletions(-) diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh index 1087a7ed96b..a3dadf48c38 100755 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -10,46 +10,40 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -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)' - sleep 0.0$RANDOM - done +function thread_create() +{ + $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 } -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)' - sleep 0.0$RANDOM - done +function thread_drop() +{ + $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 } -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)' - sleep 0.0$RANDOM - done +function thread_rename() +{ + $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 } -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)' - sleep 0.0$RANDOM - done +function thread_select() +{ + $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 } -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)' - sleep 0.0$RANDOM - done +function thread_insert() +{ + $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 } -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)' - sleep 0.0$RANDOM - done +function thread_insert_select() +{ + $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 } export -f thread_create @@ -65,18 +59,18 @@ export -f thread_insert_select function test_with_engine { echo "Testing $1" - timeout 10 bash -c "thread_create t1 $1" & - timeout 10 bash -c "thread_create t2 $1" & - timeout 10 bash -c 'thread_drop t1' & - timeout 10 bash -c 'thread_drop t2' & - timeout 10 bash -c 'thread_rename t1 t2' & - timeout 10 bash -c 'thread_rename t2 t1' & - timeout 10 bash -c 'thread_select t1' & - timeout 10 bash -c 'thread_select t2' & - timeout 10 bash -c 'thread_insert t1 5' & - timeout 10 bash -c 'thread_insert t2 10' & - timeout 10 bash -c 'thread_insert_select t1 t2' & - timeout 10 bash -c 'thread_insert_select t2 t1' & + clickhouse_client_loop_timeout 10 thread_create t1 $1 & + clickhouse_client_loop_timeout 10 thread_create t2 $1 & + clickhouse_client_loop_timeout 10 thread_drop t1 & + clickhouse_client_loop_timeout 10 thread_drop t2 & + clickhouse_client_loop_timeout 10 thread_rename t1 t2 & + clickhouse_client_loop_timeout 10 thread_rename t2 t1 & + clickhouse_client_loop_timeout 10 thread_select t1 & + clickhouse_client_loop_timeout 10 thread_select t2 & + clickhouse_client_loop_timeout 10 thread_insert t1 5 & + clickhouse_client_loop_timeout 10 thread_insert t2 10 & + clickhouse_client_loop_timeout 10 thread_insert_select t1 t2 & + clickhouse_client_loop_timeout 10 thread_insert_select t2 t1 & wait echo "Done $1" From 88f05ac14ab60ac5353c06964c8470de21a5f627 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 Apr 2022 20:08:05 +0200 Subject: [PATCH 27/31] Fix black --- tests/integration/test_storage_postgresql_replica/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py index 741cd2bd521..5df06befbcf 100644 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ b/tests/integration/test_storage_postgresql_replica/test.py @@ -698,7 +698,7 @@ def test_abrupt_connection_loss_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(): pytest.skip("Temporary disabled (FIXME)") From 3a92c11f1e754bdda93f0ba1bde2e56dacc41b6f Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 25 Apr 2022 14:09:51 -0400 Subject: [PATCH 28/31] CodeQL - Run Daily Analysis Signed-off-by: Julio Jimenez --- .github/workflows/codeql.yml | 33 +-------------------------------- 1 file changed, 1 insertion(+), 32 deletions(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 934f43123c7..73acc65e5c3 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -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" "on": - # push: - # branches: [ master ] - # pull_request: - # # The branches below must be a subset of the branches above - # branches: [ master ] schedule: - - cron: '0 */6 * * *' + - cron: '0 0 * * *' workflow_dispatch: env: @@ -38,8 +22,6 @@ jobs: fail-fast: false matrix: 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: - name: Checkout repository @@ -47,23 +29,10 @@ jobs: with: submodules: 'true' - # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL uses: github/codeql-action/init@v2 with: 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 run: | From 921ac31224292955da9a23bc40ef8bfb5b6128f8 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 25 Apr 2022 17:29:31 -0400 Subject: [PATCH 29/31] git, cmake, and python are already installed in ubuntu-latest runner Signed-off-by: Julio Jimenez --- .github/workflows/codeql.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml index 73acc65e5c3..ba6626a1b81 100644 --- a/.github/workflows/codeql.yml +++ b/.github/workflows/codeql.yml @@ -36,7 +36,7 @@ jobs: - name: Build 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)" mkdir build cd build From da3d7f2835e752008f41a61e02fa9f2b8c84e9df Mon Sep 17 00:00:00 2001 From: XenoAmess Date: Tue, 26 Apr 2022 05:51:56 +0800 Subject: [PATCH 30/31] fix boolean documents. --- docs/en/sql-reference/data-types/boolean.md | 4 +++- docs/zh/sql-reference/data-types/boolean.md | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/boolean.md b/docs/en/sql-reference/data-types/boolean.md index a20e30777af..a21b1c41a21 100644 --- a/docs/en/sql-reference/data-types/boolean.md +++ b/docs/en/sql-reference/data-types/boolean.md @@ -5,6 +5,8 @@ sidebar_label: Boolean # 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/) diff --git a/docs/zh/sql-reference/data-types/boolean.md b/docs/zh/sql-reference/data-types/boolean.md index 99aacad23a9..bd0a1ee7312 100644 --- a/docs/zh/sql-reference/data-types/boolean.md +++ b/docs/zh/sql-reference/data-types/boolean.md @@ -1,3 +1,5 @@ # 布尔值 {#boolean-values} -没有单独的类型来存储布尔值。可以使用 UInt8 类型,取值限制为 0 或 1。 +从 https://github.com/ClickHouse/ClickHouse/commit/4076ae77b46794e73594a9f400200088ed1e7a6e 之后,有单独的类型来存储布尔值。 + +在此之前的版本,没有单独的类型来存储布尔值。可以使用 UInt8 类型,取值限制为 0 或 1。 From 02662dfd13f47f83978475a9c43409da437c2539 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Apr 2022 08:23:01 +0300 Subject: [PATCH 31/31] Update ColumnsDescription.h --- src/Storages/ColumnsDescription.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 4e1d60db2f2..670aeaa293d 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -217,13 +217,14 @@ public: private: ColumnsContainer columns; - /// Subcolumns is not nested columns. + + /// Subcolumns are not nested columns. /// /// Example of subcolumns: /// - .size0 for Array /// - .null for Nullable /// - /// While nested columns is just foo.bar + /// While nested columns have form like foo.bar SubcolumnsContainter subcolumns; void modifyColumnOrder(const String & column_name, const String & after_column, bool first);