From f3f604ace491e35b251d8be928c7110d83978d9f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 27 Jun 2023 16:45:34 +0000 Subject: [PATCH 01/79] added table with pk size --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 17 +++++++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/System/StorageSystemParts.cpp | 3 +++ 3 files changed, 21 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c9930e61e98..55db22d6105 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1,5 +1,6 @@ #include "IMergeTreeDataPart.h" #include "Storages/MergeTree/IDataPartStorage.h" +#include "base/types.h" #include #include @@ -1800,6 +1801,22 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, storage.log); } +UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + if (parent_part) + metadata_snapshot = metadata_snapshot->projections.get(name).metadata; + const auto & pk = metadata_snapshot->getPrimaryKey(); + if (!pk.column_names.empty()) + { + String file = "primary" + getIndexExtension(false); + if (checksums.files.contains("primary" + getIndexExtension(true))) + file = "primary" + getIndexExtension(true); + return getFileSizeOrZero(file); + } + return 0; +} + void IMergeTreeDataPart::checkConsistencyBase() const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a36634d2cf9..b3c70c99d2e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -323,6 +323,7 @@ public: UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; UInt64 getMarksCount() const; + UInt64 getIndexSizeFromFile() const; UInt64 getBytesOnDisk() const { return bytes_on_disk; } void setBytesOnDisk(UInt64 bytes_on_disk_) { bytes_on_disk = bytes_on_disk_; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 86ecb336b51..e1e8ba1aa00 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -57,6 +57,7 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"bytes_on_disk", std::make_shared()}, {"data_compressed_bytes", std::make_shared()}, {"data_uncompressed_bytes", std::make_shared()}, + {"primary_key_size", std::make_shared()}, {"marks_bytes", std::make_shared()}, {"secondary_indices_compressed_bytes", std::make_shared()}, {"secondary_indices_uncompressed_bytes", std::make_shared()}, @@ -168,6 +169,8 @@ void StorageSystemParts::processNextStorage( columns[res_index++]->insert(columns_size.data_compressed); if (columns_mask[src_index++]) columns[res_index++]->insert(columns_size.data_uncompressed); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->getIndexSizeFromFile()); if (columns_mask[src_index++]) columns[res_index++]->insert(columns_size.marks); if (columns_mask[src_index++]) From c59ddf0c668c0a345c88df98b249b79cd58a8fcb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 27 Jun 2023 17:27:28 +0000 Subject: [PATCH 02/79] Resolved style check --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 55db22d6105..eb35fe178c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1801,7 +1801,7 @@ MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & di return getDataPartStorage().clonePart(path_to_clone, getDataPartStorage().getPartDirectory(), disk, storage.log); } -UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const +UInt64 IMergeTreeDataPart::getIndexSizeFromFile() const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (parent_part) From a013ec1abaccea5599b17e69d7a923addff76e4c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 27 Jun 2023 17:42:19 +0000 Subject: [PATCH 03/79] added field to tests --- .../queries/0_stateless/02117_show_create_table_system.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 09cc62dac00..f2c85a4d0ba 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -466,6 +466,7 @@ CREATE TABLE system.parts `bytes_on_disk` UInt64, `data_compressed_bytes` UInt64, `data_uncompressed_bytes` UInt64, + `primary_key_size` UInt64, `marks_bytes` UInt64, `secondary_indices_compressed_bytes` UInt64, `secondary_indices_uncompressed_bytes` UInt64, From 60ce9773e0310992fd900e2cbc7a0f0f2f858c10 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 29 Jun 2023 10:42:33 +0000 Subject: [PATCH 04/79] Added docs for primary_key_size --- docs/en/operations/system-tables/parts.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index e61c6ed2ba4..861104ff236 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -39,6 +39,8 @@ Columns: - `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of uncompressed data in the data part. All the auxiliary files (for example, files with marks) are not included. +- `primary_key_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The amount of memory (in bytes) used by primary key values in the primary.idx/cidx file on disk. + - `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks. - `secondary_indices_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of compressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included. From 60c6cc17fca0cf98ace414f6c81e780859439a1c Mon Sep 17 00:00:00 2001 From: yariks5s <114298166+yariks5s@users.noreply.github.com> Date: Tue, 4 Jul 2023 11:39:39 +0200 Subject: [PATCH 05/79] Update include brackets Co-authored-by: Sergei Trifonov --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87c3567f7d5..1cf86f1ae4d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1,6 +1,6 @@ #include "IMergeTreeDataPart.h" -#include "Storages/MergeTree/IDataPartStorage.h" -#include "base/types.h" +#include +#include #include #include From 0e4cd16582c5427cebb52fece006d16ec5e337e2 Mon Sep 17 00:00:00 2001 From: yariks5s <114298166+yariks5s@users.noreply.github.com> Date: Tue, 4 Jul 2023 11:53:23 +0200 Subject: [PATCH 06/79] Update include brackets --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1cf86f1ae4d..021f624e783 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1,4 +1,4 @@ -#include "IMergeTreeDataPart.h" +#include #include #include From 0d0e59abc8672348f9a3d8386fa20454271ff618 Mon Sep 17 00:00:00 2001 From: yariks5s <114298166+yariks5s@users.noreply.github.com> Date: Tue, 4 Jul 2023 12:08:58 +0200 Subject: [PATCH 07/79] Update IMergeTreeDataPart.cpp --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 021f624e783..1cf86f1ae4d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1,4 +1,4 @@ -#include +#include "IMergeTreeDataPart.h" #include #include From 546f12dc85fdbbcf3396767917bd9dbbf8522c41 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 05:05:27 +0000 Subject: [PATCH 08/79] Fix inserts to MongoDB tables --- src/Storages/StorageMongoDB.cpp | 60 ++++++++++++++++++- .../integration/test_storage_mongodb/test.py | 6 ++ 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 3287e3272e3..45b8aceb058 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -19,6 +19,8 @@ #include #include +#include + namespace DB { @@ -127,9 +129,7 @@ public: for (const auto j : collections::range(0, num_cols)) { - WriteBufferFromOwnString ostr; - data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{}); - document->add(data_names[j], ostr.str()); + insertValueIntoMongoDB(*document, data_names[j], *data_types[j], *columns[j], i); } documents.push_back(std::move(document)); @@ -151,6 +151,60 @@ public: } private: + + void insertValueIntoMongoDB( + Poco::MongoDB::Document & document, + const std::string & name, + const IDataType & data_type, + const IColumn & column, + size_t idx) + { + WhichDataType which(data_type); + + if (which.isArray()) + { + const ColumnArray & column_array = assert_cast(column); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + + size_t offset = offsets[idx - 1]; + size_t next_offset = offsets[idx]; + + const IColumn & nested_column = column_array.getData(); + + const auto * array_type = assert_cast(&data_type); + const DataTypePtr & nested_type = array_type->getNestedType(); + + Poco::MongoDB::Array::Ptr array = new Poco::MongoDB::Array(); + for (size_t i = 0; i + offset < next_offset; ++i) + { + insertValueIntoMongoDB(*array, Poco::NumberFormatter::format(i), *nested_type, nested_column, i + offset); + } + + document.add(name, array); + return; + } + + /// MongoDB does not support UInt64 type, so just cast it to Int64 + if (which.isNativeUInt()) + document.add(name, static_cast(column.getUInt(idx))); + else if (which.isNativeInt()) + document.add(name, static_cast(column.getInt(idx))); + else if (which.isFloat32()) + document.add(name, static_cast(column.getFloat32(idx))); + else if (which.isFloat64()) + document.add(name, static_cast(column.getFloat64(idx))); + else if (which.isDate()) + document.add(name, Poco::Timestamp(DateLUT::instance().fromDayNum(DayNum(column.getUInt(idx))) * 1000000)); + else if (which.isDateTime()) + document.add(name, Poco::Timestamp(column.getUInt(idx) * 1000000)); + else + { + WriteBufferFromOwnString ostr; + data_type.getDefaultSerialization()->serializeText(column, idx, ostr, FormatSettings{}); + document.add(name, ostr.str()); + } + } + String collection_name; String db_name; StorageMetadataPtr metadata_snapshot; diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 6ce71fb91fa..0abaa7a8214 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -244,6 +244,12 @@ def test_arrays(started_cluster): == "[]\n" ) + # Test INSERT SELECT + node.query("INSERT INTO arrays_mongo_table SELECT * FROM arrays_mongo_table") + + assert node.query("SELECT COUNT() FROM arrays_mongo_table") == "200\n" + assert node.query("SELECT COUNT(DISTINCT *) FROM arrays_mongo_table") == "100\n" + node.query("DROP TABLE arrays_mongo_table") arrays_mongo_table.drop() From d86ceef663cd0d3fcd8532ae63539e85bc4b210b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:14:48 +0800 Subject: [PATCH 09/79] Implement log file names rendering --- src/Loggers/Loggers.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 0c3a7bd615d..1e169190ca4 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -34,6 +34,16 @@ static std::string createDirectory(const std::string & file) return path; } +static std::string renderFileNameTemplate(time_t now, const std::string & file_path) +{ + fs::path path{file_path}; + std::tm buf; + localtime_r(&now, &buf); + std::stringstream ss; + ss << std::put_time(&buf, file_path.c_str()); + return path.replace_filename(ss.str()); +} + #ifndef WITHOUT_TEXT_LOG void Loggers::setTextLog(std::shared_ptr log, int max_priority) { @@ -68,9 +78,12 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// The maximum (the most verbose) of those will be used as default for Poco loggers int max_log_level = 0; - const auto log_path = config.getString("logger.log", ""); - if (!log_path.empty()) + time_t now = std::time({}); + + const auto log_path_prop = config.getString("logger.log", ""); + if (!log_path_prop.empty()) { + const auto log_path = renderFileNameTemplate(now, log_path_prop); createDirectory(log_path); std::string ext; @@ -109,9 +122,10 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split->addChannel(log, "log"); } - const auto errorlog_path = config.getString("logger.errorlog", ""); - if (!errorlog_path.empty()) + const auto errorlog_path_prop = config.getString("logger.errorlog", ""); + if (!errorlog_path_prop.empty()) { + const auto errorlog_path = renderFileNameTemplate(now, errorlog_path_prop); createDirectory(errorlog_path); // NOTE: we don't use notice & critical in the code, so in practice error log collects fatal & error & warning. From 479efaa79acd23e72fb06413fd84d4b7091bd019 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:18 +0800 Subject: [PATCH 10/79] Add clickhouse_log_file and clickhouse_error_log_file args to add_instance() --- tests/integration/helpers/cluster.py | 35 +++++++++++++++++----------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 21398790be3..5b583b865de 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -64,6 +64,13 @@ DEFAULT_ENV_NAME = ".env" SANITIZER_SIGN = "==================" +CLICKHOUSE_START_COMMAND = ( + "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" +) + +CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" + +CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" # to create docker-compose env file def _create_env_file(path, variables): @@ -1497,6 +1504,8 @@ class ClickHouseCluster: with_postgres=False, with_postgres_cluster=False, with_postgresql_java_client=False, + clickhouse_log_file=CLICKHOUSE_LOG_FILE, + clickhouse_error_log_file=CLICKHOUSE_ERROR_LOG_FILE, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, @@ -1563,6 +1572,13 @@ class ClickHouseCluster: "LLVM_PROFILE_FILE" ] = "/var/lib/clickhouse/server_%h_%p_%m.profraw" + clickhouse_start_command = CLICKHOUSE_START_COMMAND + if clickhouse_log_file: + clickhouse_start_command += " --log-file=" + clickhouse_log_file + if clickhouse_error_log_file: + clickhouse_start_command += " --errorlog-file=" + clickhouse_error_log_file + logging.debug(f"clickhouse_start_command: {clickhouse_start_command}") + instance = ClickHouseInstance( cluster=self, base_path=self.base_dir, @@ -1592,10 +1608,10 @@ class ClickHouseCluster: with_redis=with_redis, with_minio=with_minio, with_azurite=with_azurite, - with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, with_hive=with_hive, with_coredns=with_coredns, + with_cassandra=with_cassandra, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -1604,6 +1620,10 @@ class ClickHouseCluster: with_postgres=with_postgres, with_postgres_cluster=with_postgres_cluster, with_postgresql_java_client=with_postgresql_java_client, + clickhouse_start_command=clickhouse_start_command, + main_config_name=main_config_name, + users_config_name=users_config_name, + copy_common_configs=copy_common_configs, hostname=hostname, env_variables=env_variables, image=image, @@ -1612,9 +1632,6 @@ class ClickHouseCluster: ipv4_address=ipv4_address, ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, - main_config_name=main_config_name, - users_config_name=users_config_name, - copy_common_configs=copy_common_configs, external_dirs=external_dirs, tmpfs=tmpfs or [], config_root_name=config_root_name, @@ -3046,16 +3063,6 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -CLICKHOUSE_START_COMMAND = ( - "clickhouse server --config-file=/etc/clickhouse-server/{main_config_file}" - " --log-file=/var/log/clickhouse-server/clickhouse-server.log " - " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" -) - -CLICKHOUSE_STAY_ALIVE_COMMAND = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( - CLICKHOUSE_START_COMMAND -) - DOCKER_COMPOSE_TEMPLATE = """ version: '2.3' services: From fef71ab0b8759f7a659c4bb8c1be03a89df92f79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 6 Jul 2023 14:16:46 +0800 Subject: [PATCH 11/79] Add inegration test --- .../__init__.py | 58 +++++++++++++++++++ .../configs/config-file-template.xml | 6 ++ .../test.py | 0 3 files changed, 64 insertions(+) create mode 100644 tests/integration/test_render_log_file_name_templates/__init__.py create mode 100644 tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml create mode 100644 tests/integration/test_render_log_file_name_templates/test.py diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py new file mode 100644 index 00000000000..9fa87056d2c --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) diff --git a/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml new file mode 100644 index 00000000000..ba408eb9823 --- /dev/null +++ b/tests/integration/test_render_log_file_name_templates/configs/config-file-template.xml @@ -0,0 +1,6 @@ + + + /var/log/clickhouse-server/clickhouse-server-%Y-%m.log + /var/log/clickhouse-server/clickhouse-server-%Y-%m.err.log + + diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py new file mode 100644 index 00000000000..e69de29bb2d From b9fffacc653fb9175af03cbb8f53766b0272ddbc Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 6 Jul 2023 06:31:09 +0000 Subject: [PATCH 12/79] Fix build --- src/Storages/StorageMongoDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 45b8aceb058..21543541f36 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -186,9 +186,9 @@ private: /// MongoDB does not support UInt64 type, so just cast it to Int64 if (which.isNativeUInt()) - document.add(name, static_cast(column.getUInt(idx))); + document.add(name, static_cast(column.getUInt(idx))); else if (which.isNativeInt()) - document.add(name, static_cast(column.getInt(idx))); + document.add(name, static_cast(column.getInt(idx))); else if (which.isFloat32()) document.add(name, static_cast(column.getFloat32(idx))); else if (which.isFloat64()) From acd17c7974637714138a76fb83f73ec31946aa79 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:40:04 +0800 Subject: [PATCH 13/79] Make a deal with the "Style check" --- src/Loggers/Loggers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 1e169190ca4..4c85ea79a63 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -39,7 +39,7 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p fs::path path{file_path}; std::tm buf; localtime_r(&now, &buf); - std::stringstream ss; + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM ss << std::put_time(&buf, file_path.c_str()); return path.replace_filename(ss.str()); } From 7080d85d2de6c743cc5759fa2a50d1ada1d51068 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 10:54:36 +0800 Subject: [PATCH 14/79] Amend the documentation --- .../settings.md | 49 +++++++++++++++++- .../settings.md | 50 ++++++++++++++++++- 2 files changed, 95 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index bad7e388377..48361b0f157 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,13 +1201,58 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. +Both log and error log file names (only file names, not directories) support date and time format placeholders. + +**Placeholders** +Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. + +| Placeholder | Description | Example | +|-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Literal % | % | +| %n | New-line character | | +| %t | Horizontal tab character | | +| %Y | Year as a decimal number, e.g. 2017 | 2023 | +| %y | Last 2 digits of year as a decimal number (range [00,99]) | 23 | +| %C | First 2 digits of year as a decimal number (range [00,99]) | 20 | +| %G | Four-digit [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. Normally useful only with %V | 2023 | +| %g | Last 2 digits of [ISO 8601 week-based year](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), i.e. the year that contains the specified week. | 23 | +| %b | Abbreviated month name, e.g. Oct (locale dependent) | Jul | +| %h | Synonym of %b | Jul | +| %B | Full month name, e.g. October (locale dependent) | July | +| %m | Month as a decimal number (range [01,12]) | 07 | +| %U | Week of the year as a decimal number (Sunday is the first day of the week) (range [00,53]) | 27 | +| %W | Week of the year as a decimal number (Monday is the first day of the week) (range [00,53]) | 27 | +| %V | ISO 8601 week number (range [01,53]) | 27 | +| %j | Day of the year as a decimal number (range [001,366]) | 187 | +| %d | Day of the month as a zero-padded decimal number (range [01,31]). Single digit is preceded by zero. | 06 | +| %e | Day of the month as a space-padded decimal number (range [1,31]). Single digit is preceded by a space. |   6 | +| %a | Abbreviated weekday name, e.g. Fri (locale dependent) | Thu | +| %A | Full weekday name, e.g. Friday (locale dependent) | Thursday | +| %w | Weekday as a integer number with Sunday as 0 (range [0-6]) | 4 | +| %u | Weekday as a decimal number, where Monday is 1 (ISO 8601 format) (range [1-7]) | 4 | +| %H | Hour as a decimal number, 24 hour clock (range [00-23]) | 18 | +| %I | Hour as a decimal number, 12 hour clock (range [01,12]) | 06 | +| %M | Minute as a decimal number (range [00,59]) | 32 | +| %S | Second as a decimal number (range [00,60]) | 07 | +| %c | Standard date and time string, e.g. Sun Oct 17 04:41:13 2010 (locale dependent) | Thu Jul 6 18:32:07 2023 | +| %x | Localized date representation (locale dependent) | 07/06/23 | +| %X | Localized time representation, e.g. 18:40:20 or 6:40:20 PM (locale dependent) | 18:32:07 | +| %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 07/06/23 | +| %F | Short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2023-07-06 | +| %r | Localized 12-hour clock time (locale dependent) | 06:32:07 PM | +| %R | Equivalent to "%H:%M" | 18:32 | +| %T | Equivalent to "%H:%M:%S" (the ISO 8601 time format) | 18:32:07 | +| %p | Localized a.m. or p.m. designation (locale dependent) | PM | +| %z | Offset from UTC in the ISO 8601 format (e.g. -0430), or no characters if the time zone information is not available | +0800 | +| %Z | Locale-dependent time zone name or abbreviation, or no characters if the time zone information is not available | Z AWST | + **Example** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 true diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 5430469ea18..421df3fe3eb 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -575,14 +575,60 @@ ClickHouse поддерживает динамическое изменение - `errorlog` - Файл лога ошибок. - `size` - Размер файла. Действует для `log` и `errorlog`. Как только файл достиг размера `size`, ClickHouse архивирует и переименовывает его, а на его месте создает новый файл лога. - `count` - Количество заархивированных файлов логов, которые сохраняет ClickHouse. +- `stream_compress` – Сжимать `log` и `errorlog` с помощью алгоритма `lz4`. Чтобы активировать, узтановите значение `1` или `true`. + +Имена файлов `log` и `errorlog` (только имя файла, а не директорий) поддерживают спецификаторы шаблонов даты и времени. + +**Спецификаторы форматирования** +С помощью следующих спецификаторов, можно определить шаблон для формирования имени файла. Столбец “Пример” показывает возможные значения на момент времени `2023-07-06 18:32:07`. + +| Спецификатор | Описание | Пример | +|--------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| +| %% | Литерал % | % | +| %n | Символ новой строки | | +| %t | Символ горизонтальной табуляции | | +| %Y | Год как десятичное число, например, 2017 | 2023 | +| %y | Последние 2 цифры года в виде десятичного числа (диапазон [00,99]) | 23 | +| %C | Первые 2 цифры года в виде десятичного числа (диапазон [00,99]) | 20 | +| %G | Год по неделям согласно [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), то есть год, который содержит указанную неделю. Обычно используется вместе с %V. | 2023 | +| %g | Последние 2 цифры [года по неделям ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates), т.е. года, содержащего указанную неделю (диапазон [00,99]). | 23 | +| %b | Сокращённое название месяца, например Oct (зависит от локали) | Jul | +| %h | Синоним %b | Jul | +| %B | Полное название месяца, например, October (зависит от локали) | July | +| %m | Месяц в виде десятичного числа (диапазон [01,12]) | 07 | +| %U | Неделя года в виде десятичного числа (воскресенье - первый день недели) (диапазон [00,53]) | 27 | +| %W | Неделя года в виде десятичного числа (понедельник - первый день недели) (диапазон [00,53]) | 27 | +| %V | Неделя года ISO 8601 (диапазон [01,53]) | 27 | +| %j | День года в виде десятичного числа (диапазон [001,366]) | 187 | +| %d | День месяца в виде десятичного числа (диапазон [01,31]) Перед одиночной цифрой ставится ноль. | 06 | +| %e | День месяца в виде десятичного числа (диапазон [1,31]). Перед одиночной цифрой ставится пробел. |   6 | +| %a | Сокращённое название дня недели, например, Fri (зависит от локали) | Thu | +| %A | Полный день недели, например, Friday (зависит от локали) | Thursday | +| %w | День недели в виде десятичного числа, где воскресенье равно 0 (диапазон [0-6]) | 4 | +| %u | День недели в виде десятичного числа, где понедельник равен 1 (формат ISO 8601) (диапазон [1-7]) | 4 | +| %H | Час в виде десятичного числа, 24-часовой формат (диапазон [00-23]) | 18 | +| %I | Час в виде десятичного числа, 12-часовой формат (диапазон [01,12]) | 06 | +| %M | Минуты в виде десятичного числа (диапазон [00,59]) | 32 | +| %S | Секунды как десятичное число (диапазон [00,60]) | 07 | +| %c | Стандартная строка даты и времени, например, Sun Oct 17 04:41:13 2010 (зависит от локали) | Thu Jul 6 18:32:07 2023 | +| %x | Локализованное представление даты (зависит от локали) | 07/06/23 | +| %X | Локализованное представление времени, например, 18:40:20 или 6:40:20 PM (зависит от локали) | 18:32:07 | +| %D | Эквивалентно "%m/%d/%y" | 07/06/23 | +| %F | Эквивалентно "%Y-%m-%d" (формат даты ISO 8601) | 2023-07-06 | +| %r | Локализованное 12-часовое время (зависит от локали) | 06:32:07 PM | +| %R | Эквивалентно "%H:%M" | 18:32 | +| %T | Эквивалентно "%H:%M:%S" (формат времени ISO 8601) | 18:32:07 | +| %p | Локализованное обозначение a.m. или p.m. (зависит от локали) | PM | +| %z | Смещение от UTC в формате ISO 8601 (например, -0430), или без символов, если информация о часовом поясе недоступна | +0800 | +| %Z | Зависящее от локали название или аббревиатура часового пояса, если информация о часовом поясе доступна | Z AWST | **Пример** ``` xml trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.log + /var/log/clickhouse-server/clickhouse-server-%F-%T.err.log 1000M 10 From 63fbde41fee5fb8c0133dc5a576ed4e3caa5c3f2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:01:39 +0800 Subject: [PATCH 15/79] Reformat cluster.py (add empty line) --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5b583b865de..d4b1ee76712 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -72,6 +72,7 @@ CLICKHOUSE_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.log" CLICKHOUSE_ERROR_LOG_FILE = "/var/log/clickhouse-server/clickhouse-server.err.log" + # to create docker-compose env file def _create_env_file(path, variables): logging.debug(f"Env {variables} stored in {path}") From 3edee4174c040b079015ce6524c0d4c56926e348 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 11:34:03 +0800 Subject: [PATCH 16/79] Add AWST time zone abbreviation to the ignore list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..6ddca6db538 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -12,6 +12,7 @@ ARMv ASLR ASOF ASan +AWST Actian ActionsMenu ActiveRecord From 87ea1b6667ed9a79272e3b77c529369f2acc4e4e Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 13:01:30 +0800 Subject: [PATCH 17/79] Recover the integration test --- .../test.py | 58 +++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index e69de29bb2d..9fa87056d2c 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -0,0 +1,58 @@ +import pytest +import logging +from helpers.cluster import ClickHouseCluster +from datetime import datetime + + +log_dir = "/var/log/clickhouse-server/" +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster(): + cluster.add_instance( + "file-names-from-config", + main_configs=["configs/config-file-template.xml"], + clickhouse_log_file=None, + clickhouse_error_log_file=None, + ) + cluster.add_instance( + "file-names-from-params", + clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", + clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", + ) + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_check_file_names(started_cluster): + now = datetime.now() + log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" + ) + err_log_file = ( + log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" + ) + logging.debug(f"log_file {log_file} err_log_file {err_log_file}") + + for name, instance in started_cluster.instances.items(): + files = instance.exec_in_container( + ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True + ) + + logging.debug(f"check instance '{name}': {log_dir} contains: {files}") + + assert ( + instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) + == log_file + "\n" + ) + + assert ( + instance.exec_in_container( + ["bash", "-c", f"ls {err_log_file}"], nothrow=True + ) + == err_log_file + "\n" + ) From 95fedaedff3ad3e3cdb15d3cc2b06ab6d9ea1e9b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 15:16:10 +0800 Subject: [PATCH 18/79] Refine the integration test code --- .../__init__.py | 58 ------------------- .../test.py | 8 +-- 2 files changed, 2 insertions(+), 64 deletions(-) diff --git a/tests/integration/test_render_log_file_name_templates/__init__.py b/tests/integration/test_render_log_file_name_templates/__init__.py index 9fa87056d2c..e69de29bb2d 100644 --- a/tests/integration/test_render_log_file_name_templates/__init__.py +++ b/tests/integration/test_render_log_file_name_templates/__init__.py @@ -1,58 +0,0 @@ -import pytest -import logging -from helpers.cluster import ClickHouseCluster -from datetime import datetime - - -log_dir = "/var/log/clickhouse-server/" -cluster = ClickHouseCluster(__file__) - - -@pytest.fixture(scope="module") -def started_cluster(): - cluster.add_instance( - "file-names-from-config", - main_configs=["configs/config-file-template.xml"], - clickhouse_log_file=None, - clickhouse_error_log_file=None, - ) - cluster.add_instance( - "file-names-from-params", - clickhouse_log_file=log_dir + "clickhouse-server-%Y-%m.log", - clickhouse_error_log_file=log_dir + "clickhouse-server-%Y-%m.err.log", - ) - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_check_file_names(started_cluster): - now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) - logging.debug(f"log_file {log_file} err_log_file {err_log_file}") - - for name, instance in started_cluster.instances.items(): - files = instance.exec_in_container( - ["bash", "-c", f"ls -lh {log_dir}"], nothrow=True - ) - - logging.debug(f"check instance '{name}': {log_dir} contains: {files}") - - assert ( - instance.exec_in_container(["bash", "-c", f"ls {log_file}"], nothrow=True) - == log_file + "\n" - ) - - assert ( - instance.exec_in_container( - ["bash", "-c", f"ls {err_log_file}"], nothrow=True - ) - == err_log_file + "\n" - ) diff --git a/tests/integration/test_render_log_file_name_templates/test.py b/tests/integration/test_render_log_file_name_templates/test.py index 9fa87056d2c..58df32b823e 100644 --- a/tests/integration/test_render_log_file_name_templates/test.py +++ b/tests/integration/test_render_log_file_name_templates/test.py @@ -30,12 +30,8 @@ def started_cluster(): def test_check_file_names(started_cluster): now = datetime.now() - log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.log" - ) - err_log_file = ( - log_dir + f"clickhouse-server-{now.strftime('%Y')}-{now.strftime('%m')}.err.log" - ) + log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.log" + err_log_file = log_dir + f"clickhouse-server-{now.strftime('%Y-%m')}.err.log" logging.debug(f"log_file {log_file} err_log_file {err_log_file}") for name, instance in started_cluster.instances.items(): From 5b102ce7d44d678a674d29e4140a60950c69f537 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Fri, 7 Jul 2023 17:21:47 +0800 Subject: [PATCH 19/79] Amend English version of settings.md --- .../server-configuration-parameters/settings.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 48361b0f157..82dac74e647 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1201,12 +1201,12 @@ Keys: - `console` – Send `log` and `errorlog` to the console instead of file. To enable, set to `1` or `true`. - `stream_compress` – Compress `log` and `errorlog` with `lz4` stream compression. To enable, set to `1` or `true`. -Both log and error log file names (only file names, not directories) support date and time format placeholders. +Both log and error log file names (only file names, not directories) support date and time format specifiers. -**Placeholders** -Using the following placeholders, you can define a pattern for the resulting file name. “Example” column shows formatting result for `2023-07-06 18:32:07`. +**Format specifiers** +Using the following format specifiers, you can define a pattern for the resulting file name. “Example” column shows possible results for `2023-07-06 18:32:07`. -| Placeholder | Description | Example | +| Specifier | Description | Example | |-------------|---------------------------------------------------------------------------------------------------------------------|--------------------------| | %% | Literal % | % | | %n | New-line character | | From de0837fe21d3f7330fc58712c887b1cc570af05a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 03:55:25 +0200 Subject: [PATCH 20/79] Check that functional tests cleanup their tables --- tests/clickhouse-test | 15 ++++++++++++++- .../02788_fix_logical_error_in_sorting.sql | 6 ++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 95470f77987..46ec19b041d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1212,7 +1212,20 @@ class TestCase: seconds_left = max( args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - drop_database_query = "DROP DATABASE IF EXISTS " + database + + leftover_tables = clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ).decode().replace("\n", ", "); + + if 0 != len(leftover_tables): + raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + + drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: drop_database_query += " ON CLUSTER test_cluster_database_replicated" diff --git a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql index 1ff68191800..60905e2634f 100644 --- a/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql +++ b/tests/queries/0_stateless/02788_fix_logical_error_in_sorting.sql @@ -1,3 +1,6 @@ +DROP TABLE IF EXISTS session_events; +DROP TABLE IF EXISTS event_types; + CREATE TABLE session_events ( clientId UInt64, @@ -75,3 +78,6 @@ FROM WHERE runningDifference(timestamp) >= 500 ORDER BY timestamp ASC FORMAT Null; + +DROP TABLE session_events; +DROP TABLE event_types; From a61bc7cfa593a510f96c670ef987129a12dc9b40 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 9 Jul 2023 02:03:47 +0000 Subject: [PATCH 21/79] Automatic style fix --- tests/clickhouse-test | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46ec19b041d..1145dfa9358 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1213,17 +1213,23 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - leftover_tables = clickhouse_execute( - args, - f"SHOW TABLES FROM {database}", - timeout=seconds_left, - settings={ - "log_comment": args.testcase_basename, - }, - ).decode().replace("\n", ", "); + leftover_tables = ( + clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ) + .decode() + .replace("\n", ", ") + ) if 0 != len(leftover_tables): - raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + raise Exception( + f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." + ) drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: From 7311469c32f448a95e046c013f3ef34a2bd880b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 9 Jul 2023 04:17:15 +0200 Subject: [PATCH 22/79] Fix test_extreme_deduplication --- .../test_extreme_deduplication/test.py | 27 +++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 71f783d37c9..9ce3582a826 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -49,20 +49,25 @@ def test_deduplication_window_in_seconds(started_cluster): node.query("INSERT INTO simple VALUES (0, 1)") assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n") - # wait clean thread - time.sleep(2) + # Wait for the cleanup thread. + for i in range(100): + time.sleep(1) + + if ( + TSV.toMat( + node.query( + "SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/tables/0/simple/blocks'" + ) + )[0][0] + <= "1" + ): + break + else: + raise Exception("The blocks from Keeper were not removed in time") - assert ( - TSV.toMat( - node.query( - "SELECT count() FROM system.zookeeper WHERE path='/clickhouse/tables/0/simple/blocks'" - ) - )[0][0] - == "1" - ) node.query( "INSERT INTO simple VALUES (0, 0)" - ) # deduplication doesn't works here, the first hash node was deleted + ) # Deduplication doesn't work here as the first hash node was deleted assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3" node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") From 00cf66ab7a60025c3722044ee6bf10235e15333e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 26 Jun 2023 21:25:43 +0200 Subject: [PATCH 23/79] Add ability to disable native copy for BACKUP/RESTORE Native copy uses CopyObject for S3/GCS, but in GCS the CopyObject is buggy - it does not always works. Signed-off-by: Azat Khuzhin --- src/Backups/BackupFactory.h | 1 + src/Backups/BackupIO_S3.cpp | 10 +++-- src/Backups/BackupIO_S3.h | 6 ++- src/Backups/BackupSettings.cpp | 1 + src/Backups/BackupSettings.h | 3 ++ src/Backups/BackupsWorker.cpp | 2 + src/Backups/RestoreSettings.cpp | 1 + src/Backups/RestoreSettings.h | 3 ++ src/Backups/registerBackupEngineS3.cpp | 4 +- tests/config/config.d/storage_conf.xml | 8 ++++ .../02801_backup_native_copy.reference | 4 ++ .../0_stateless/02801_backup_native_copy.sh | 43 +++++++++++++++++++ 12 files changed, 78 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02801_backup_native_copy.reference create mode 100755 tests/queries/0_stateless/02801_backup_native_copy.sh diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 92a5e16533c..642f5cb07b9 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -35,6 +35,7 @@ public: std::shared_ptr backup_coordination; std::optional backup_uuid; bool deduplicate_files = true; + bool native_copy = true; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 9a2a457e13e..01e6bc78949 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,12 +101,13 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} + , native_copy(native_copy_) { request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } @@ -138,7 +139,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (destination_data_source_description.sameKind(data_source_description) + if (native_copy && destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { /// Use native copy, the more optimal way. @@ -177,12 +178,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} + , native_copy(native_copy_) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint @@ -194,7 +196,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + if (native_copy && source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index cca56bae6bc..d02e45370f9 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -32,13 +32,14 @@ private: const std::shared_ptr client; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; + const bool native_copy; }; class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; @@ -61,6 +62,7 @@ private: S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; const DataSourceDescription data_source_description; + const bool native_copy; }; } diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 882342467fe..8e9fe7956f9 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ + M(Bool, native_copy) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index 2c899687e6e..e21b70ee25f 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -38,6 +38,9 @@ struct BackupSettings /// Whether the BACKUP will omit similar files (within one backup only). bool deduplicate_files = true; + /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) + bool native_copy = true; + /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. size_t shard_num = 0; diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 53a076705c4..fddd4f34bb6 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -348,6 +348,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; + backup_create_params.native_copy = backup_settings.native_copy; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -647,6 +648,7 @@ void BackupsWorker::doRestore( backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; + backup_open_params.native_copy = restore_settings.native_copy; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index d12da704b2d..4dd75911a91 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -161,6 +161,7 @@ namespace M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ M(RestoreUDFCreationMode, create_function) \ + M(Bool, native_copy) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 3bce8698620..59d73c83d69 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -107,6 +107,9 @@ struct RestoreSettings /// How the RESTORE command will handle if a user-defined function which it's going to restore already exists. RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; + /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) + bool native_copy = true; + /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 8387b4627d5..ef8ced94590 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,12 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index deee71bd812..af04024d528 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -2,6 +2,14 @@ + + s3 + s3_common_disk/ + http://localhost:11111/test/common/ + clickhouse + clickhouse + 20000 + s3 s3_disk/ diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference new file mode 100644 index 00000000000..659df5e9b25 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -0,0 +1,4 @@ +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh new file mode 100755 index 00000000000..966d7ae9ce8 --- /dev/null +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag: no-fasttest - requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists data; + create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_common_disk'; + insert into data select * from numbers(10); +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" + +query_id=$(random_str 10) +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' +" From 5835e72fd6d5dd0225a0dda2f81887d6f61015fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:20:27 +0200 Subject: [PATCH 24/79] More generic approach to disable native copy Previous patch implements this only for BACKUP/RESTORE, but it can be useful for regular disks as well, so add allow_native_copy for disks. Note, that there is s3_allow_native_copy query setting, since it looks redundant, since it make sense only for S3 disks, and not on a per query basis. Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 34 +++++++++++++------ src/Backups/BackupIO_S3.h | 6 ++-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++++++-- src/IO/S3/copyS3File.cpp | 22 +++++++++++- src/IO/S3/copyS3File.h | 20 +++++++++-- src/Storages/StorageS3Settings.cpp | 2 ++ src/Storages/StorageS3Settings.h | 1 + 7 files changed, 80 insertions(+), 19 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 01e6bc78949..6531948c872 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,15 +101,16 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { + request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -139,11 +140,10 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (native_copy && destination_data_source_description.sameKind(data_source_description) + if (destination_data_source_description.sameKind(data_source_description) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying {} from S3 to disk {} using native copy", path_in_backup, destination_disk->getName()); + LOG_TRACE(log, "Copying {} from S3 to disk {}", path_in_backup, destination_disk->getName()); auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. @@ -152,7 +152,13 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); + auto create_read_buffer = [this, path_in_backup] + { + return readFile(path_in_backup); + }; + copyS3File( + create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -178,16 +184,16 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) , data_source_description{DataSourceType::S3, s3_uri.endpoint, false, false} - , native_copy(native_copy_) { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + request_settings.allow_native_copy = native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -196,15 +202,23 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// Use the native copy as a more optimal way to copy a file from S3 to S3 if it's possible. /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto source_data_source_description = src_disk->getDataSourceDescription(); - if (native_copy && source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in S3 bucket. /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - /// Use native copy, the more optimal way. - LOG_TRACE(log, "Copying file {} from disk {} to S3 using native copy", src_path, src_disk->getName()); + auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] + { + if (copy_encrypted) + return src_disk->readEncryptedFile(src_path, settings); + else + return src_disk->readFile(src_path, settings); + }; + + LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index d02e45370f9..16b2abfea3d 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -32,14 +32,13 @@ private: const std::shared_ptr client; S3Settings::RequestSettings request_settings; const DataSourceDescription data_source_description; - const bool native_copy; }; class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy_, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; @@ -62,7 +61,6 @@ private: S3Settings::RequestSettings request_settings; std::optional supports_batch_delete; const DataSourceDescription data_source_description; - const bool native_copy; }; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index e46ca3d0828..3c19af188dc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,7 +435,12 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -451,7 +456,12 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + auto create_read_buffer = [this, object_from] + { + return readObject(object_from); + }; + + copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 7886b84cd00..3f18d3b2145 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,7 +809,7 @@ void copyDataToS3File( } -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -826,6 +826,26 @@ void copyS3File( helper.performCopy(); } +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata, + ThreadPoolCallbackRunner schedule, + bool for_disk_s3) +{ + if (settings.allow_native_copy) + copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + else + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); +} + } #endif diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 618ef419a9b..d41f34c103c 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,9 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3File() is faster and spends less network traffic and memory. +/// however copyS3FileNative() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3File( +void copyS3FileNative( const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,6 +51,22 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); +/// Tries to copy file using native copy (copyS3FileNative()), if this is not +/// possible it will fallback to read-write copy (copyDataToS3File()) +void copyS3File( + const CreateReadBuffer & create_read_buffer, + const std::shared_ptr & s3_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + const S3Settings::RequestSettings & settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_s3 = false); + } #endif diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 89e6ee46b4d..0dc8d8d897b 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -182,6 +182,7 @@ S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); max_connections = collection.getOrDefault("max_connections", max_connections); list_object_keys_size = collection.getOrDefault("list_object_keys_size", list_object_keys_size); + allow_native_copy = collection.getOrDefault("allow_native_copy", allow_native_copy); throw_on_zero_files_match = collection.getOrDefault("throw_on_zero_files_match", throw_on_zero_files_match); } @@ -197,6 +198,7 @@ S3Settings::RequestSettings::RequestSettings( max_connections = config.getUInt64(key + "max_connections", settings.s3_max_connections); check_objects_after_upload = config.getBool(key + "check_objects_after_upload", settings.s3_check_objects_after_upload); list_object_keys_size = config.getUInt64(key + "list_object_keys_size", settings.s3_list_object_keys_size); + allow_native_copy = config.getBool(key + "allow_native_copy", allow_native_copy); throw_on_zero_files_match = config.getBool(key + "throw_on_zero_files_match", settings.s3_throw_on_zero_files_match); retry_attempts = config.getUInt64(key + "retry_attempts", settings.s3_retry_attempts); request_timeout_ms = config.getUInt64(key + "request_timeout_ms", settings.s3_request_timeout_ms); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 991e323acb6..581665a7dc5 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -71,6 +71,7 @@ struct S3Settings size_t retry_attempts = 10; size_t request_timeout_ms = 3000; size_t long_request_timeout_ms = 30000; // TODO: Take this from config like request_timeout_ms + bool allow_native_copy = true; bool throw_on_zero_files_match = false; From 5d63b8be0d317af7b2ee1fdfd7dc76daeeec3afd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 16:48:30 +0200 Subject: [PATCH 25/79] Add a test for allow_native_copy using clickhouse-disks (first ever) Signed-off-by: Azat Khuzhin --- .../02802_clickhouse_disks_s3_copy.reference | 4 +++ .../02802_clickhouse_disks_s3_copy.sh | 26 +++++++++++++++++++ .../02802_clickhouse_disks_s3_copy.xml | 21 +++++++++++++++ 3 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference create mode 100755 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh create mode 100644 tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference new file mode 100644 index 00000000000..96860a2f90a --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.reference @@ -0,0 +1,4 @@ +s3_plain_native_copy +Single operation copy has completed. +s3_plain_no_native_copy +Single part upload has completed. diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh new file mode 100755 index 00000000000..f879b7a5621 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires S3 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +config="${BASH_SOURCE[0]/.sh/.xml}" + +function run_test_for_disk() +{ + local disk=$1 && shift + + echo "$disk" + + clickhouse-disks -C "$config" --disk "$disk" write --input "$config" $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --log-level test --disk "$disk" copy $CLICKHOUSE_DATABASE/test $CLICKHOUSE_DATABASE/test.copy |& { + grep -o -e "Single part upload has completed." -e "Single operation copy has completed." + } + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy +} + +run_test_for_disk s3_plain_native_copy +run_test_for_disk s3_plain_no_native_copy diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml new file mode 100644 index 00000000000..d4235a70903 --- /dev/null +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml @@ -0,0 +1,21 @@ + + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + true + + + + s3_plain + http://localhost:11111/test/clickhouse-disks/ + clickhouse + clickhouse + false + + + + From 84c720b33e9ffe44c79658af57f5985b38b8a728 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:52:00 +0200 Subject: [PATCH 26/79] Beatify test_backup_restore_s3 (using per-query profile events) Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 148 +++++++++--------- 1 file changed, 75 insertions(+), 73 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 0285500d044..bb14fa4824b 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -2,6 +2,7 @@ from typing import Dict, Iterable import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +import uuid cluster = ClickHouseCluster(__file__) @@ -37,32 +38,31 @@ def new_backup_name(): return f"backup{backup_id_counter}" -def get_events(events_names: Iterable[str]) -> Dict[str, int]: - _events = TSV( +def get_events_for_query(query_id: str) -> Dict[str, int]: + events = TSV( node.query( - f"SELECT event, value FROM system.events WHERE event in {events_names} SETTINGS system_events_show_zero_values = 1;" + f""" + SYSTEM FLUSH LOGS; + + WITH arrayJoin(ProfileEvents) as pe + SELECT pe.1, pe.2 + FROM system.query_log + WHERE query_id = '{query_id}' + """ ) ) return { event: int(value) - for event, value in [line.split("\t") for line in _events.lines] + for event, value in [line.split("\t") for line in events.lines] } def check_backup_and_restore( - storage_policy, backup_destination, size=1000, backup_name=None, check_events=False + storage_policy, + backup_destination, + size=1000, + backup_name=None, ): - s3_backup_events = ( - "WriteBufferFromS3Microseconds", - "WriteBufferFromS3Bytes", - "WriteBufferFromS3RequestsErrors", - ) - s3_restore_events = ( - "ReadBufferFromS3Microseconds", - "ReadBufferFromS3Bytes", - "ReadBufferFromS3RequestsErrors", - ) - node.query( f""" DROP TABLE IF EXISTS data SYNC; @@ -72,16 +72,17 @@ def check_backup_and_restore( """ ) try: - events_before_backups = get_events(s3_backup_events) - node.query(f"BACKUP TABLE data TO {backup_destination}") - events_after_backups = get_events(s3_backup_events) - events_before_restore = get_events(s3_restore_events) + backup_query_id = uuid.uuid4().hex + node.query( + f"BACKUP TABLE data TO {backup_destination}", query_id=backup_query_id + ) + restore_query_id = uuid.uuid4().hex node.query( f""" RESTORE TABLE data AS data_restored FROM {backup_destination}; - """ + """, + query_id=restore_query_id, ) - events_after_restore = get_events(s3_restore_events) node.query( """ SELECT throwIf( @@ -91,55 +92,10 @@ def check_backup_and_restore( ); """ ) - if check_events and backup_name: - objects = node.cluster.minio_client.list_objects( - "root", f"data/backups/multipart/{backup_name}/" - ) - backup_meta_size = 0 - for obj in objects: - if ".backup" in obj.object_name: - backup_meta_size = obj.size - break - backup_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" - ).strip() - ) - restore_total_size = int( - node.query( - f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" - ).strip() - ) - # backup - # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test - assert ( - abs( - backup_total_size - - ( - events_after_backups["WriteBufferFromS3Bytes"] - - events_before_backups["WriteBufferFromS3Bytes"] - - backup_meta_size - ) - ) - < 100 - ) - assert ( - events_after_backups["WriteBufferFromS3Microseconds"] - > events_before_backups["WriteBufferFromS3Microseconds"] - ) - assert events_after_backups["WriteBufferFromS3RequestsErrors"] == 0 - # restore - assert ( - events_after_restore["ReadBufferFromS3Bytes"] - - events_before_restore["ReadBufferFromS3Bytes"] - - backup_meta_size - == restore_total_size - ) - assert ( - events_after_restore["ReadBufferFromS3Microseconds"] - > events_before_restore["ReadBufferFromS3Microseconds"] - ) - assert events_after_restore["ReadBufferFromS3RequestsErrors"] == 0 + return [ + get_events_for_query(backup_query_id), + get_events_for_query(restore_query_id), + ] finally: node.query( """ @@ -224,17 +180,63 @@ def test_backup_to_s3_multipart(): storage_policy = "default" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore( + (backup_events, restore_events) = check_backup_and_restore( storage_policy, backup_destination, size=1000000, backup_name=backup_name, - check_events=True, ) assert node.contains_in_log( f"copyDataToS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}" ) + s3_backup_events = ( + "WriteBufferFromS3Microseconds", + "WriteBufferFromS3Bytes", + "WriteBufferFromS3RequestsErrors", + ) + s3_restore_events = ( + "ReadBufferFromS3Microseconds", + "ReadBufferFromS3Bytes", + "ReadBufferFromS3RequestsErrors", + ) + + objects = node.cluster.minio_client.list_objects( + "root", f"data/backups/multipart/{backup_name}/" + ) + backup_meta_size = 0 + for obj in objects: + if ".backup" in obj.object_name: + backup_meta_size = obj.size + break + backup_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'BACKUP_CREATED' AND name like '%{backup_name}%'" + ).strip() + ) + restore_total_size = int( + node.query( + f"SELECT sum(total_size) FROM system.backups WHERE status = 'RESTORED' AND name like '%{backup_name}%'" + ).strip() + ) + # backup + # NOTE: ~35 bytes is used by .lock file, so set up 100 bytes to avoid flaky test + assert ( + abs( + backup_total_size + - (backup_events["WriteBufferFromS3Bytes"] - backup_meta_size) + ) + < 100 + ) + assert backup_events["WriteBufferFromS3Microseconds"] > 0 + assert "WriteBufferFromS3RequestsErrors" not in backup_events + # restore + assert ( + restore_events["ReadBufferFromS3Bytes"] - backup_meta_size == restore_total_size + ) + assert restore_events["ReadBufferFromS3Microseconds"] > 0 + assert "ReadBufferFromS3RequestsErrors" not in restore_events + def test_backup_to_s3_native_copy(): storage_policy = "policy_s3" From 29dc9abfcab495f66689826fdbb8ee7a81ab4c7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 10:58:23 +0200 Subject: [PATCH 27/79] Fix test_backup_restore_s3 after logging for native copying changed Check profile events instead of some odd logs. Signed-off-by: Azat Khuzhin --- .../test_backup_restore_s3/test.py | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index bb14fa4824b..8701bf0d832 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -244,9 +244,12 @@ def test_backup_to_s3_native_copy(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -258,9 +261,12 @@ def test_backup_to_s3_native_copy_other_bucket(): backup_destination = ( f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) - check_backup_and_restore(storage_policy, backup_destination) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination + ) + # single part upload + assert backup_events["S3CopyObject"] > 0 + assert restore_events["S3CopyObject"] > 0 assert node.contains_in_log( f"copyS3File: Single operation copy has completed. Bucket: root, Key: data/backups/{backup_name}" ) @@ -270,9 +276,12 @@ def test_backup_to_s3_native_copy_multipart(): storage_policy = "policy_s3" backup_name = new_backup_name() backup_destination = f"S3('http://minio1:9001/root/data/backups/multipart/{backup_name}', 'minio', 'minio123')" - check_backup_and_restore(storage_policy, backup_destination, size=1000000) - assert node.contains_in_log("BackupWriterS3.*using native copy") - assert node.contains_in_log("BackupReaderS3.*using native copy") + (backup_events, restore_events) = check_backup_and_restore( + storage_policy, backup_destination, size=1000000 + ) + # multi part upload + assert backup_events["S3CreateMultipartUpload"] > 0 + assert restore_events["S3CreateMultipartUpload"] > 0 assert node.contains_in_log( f"copyS3File: Multipart upload has completed. Bucket: root, Key: data/backups/multipart/{backup_name}/" ) From 1590ffa3b1eee26d66ae3aec3ac32c63acdea153 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:22:57 +0200 Subject: [PATCH 28/79] Remove copyS3FileNative() Signed-off-by: Azat Khuzhin --- src/IO/S3/copyS3File.cpp | 22 ++++------------------ src/IO/S3/copyS3File.h | 25 +++++++------------------ 2 files changed, 11 insertions(+), 36 deletions(-) diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 3f18d3b2145..2c6557d97e7 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -809,23 +809,6 @@ void copyDataToS3File( } -void copyS3FileNative( - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata, - ThreadPoolCallbackRunner schedule, - bool for_disk_s3) -{ - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; - helper.performCopy(); -} - void copyS3File( const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, @@ -841,7 +824,10 @@ void copyS3File( bool for_disk_s3) { if (settings.allow_native_copy) - copyS3FileNative(s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + { + CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + helper.performCopy(); + } else copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index d41f34c103c..2c848076e9b 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -19,9 +19,14 @@ using CreateReadBuffer = std::function()>; /// Copies a file from S3 to S3. /// The same functionality can be done by using the function copyData() and the classes ReadBufferFromS3 and WriteBufferFromS3 -/// however copyS3FileNative() is faster and spends less network traffic and memory. +/// however copyS3File() is faster and spends less network traffic and memory. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. -void copyS3FileNative( +/// +/// Note, that it tries to copy file using native copy (CopyObject), but if it +/// has been disabled (with settings.allow_native_copy) it is fallbacks to +/// read-write copy (copyDataToS3File()). +void copyS3File( + const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -51,22 +56,6 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_s3 = false); -/// Tries to copy file using native copy (copyS3FileNative()), if this is not -/// possible it will fallback to read-write copy (copyDataToS3File()) -void copyS3File( - const CreateReadBuffer & create_read_buffer, - const std::shared_ptr & s3_client, - const String & src_bucket, - const String & src_key, - size_t src_offset, - size_t src_size, - const String & dest_bucket, - const String & dest_key, - const S3Settings::RequestSettings & settings, - const std::optional> & object_metadata = std::nullopt, - ThreadPoolCallbackRunner schedule_ = {}, - bool for_disk_s3 = false); - } #endif From 559d3281782c22fa380e85e188d2a15e404a4c19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 17:16:02 +0200 Subject: [PATCH 29/79] Rename BACKUP setting native_copy to allow_s3_native_copy Signed-off-by: Azat Khuzhin --- src/Backups/BackupFactory.h | 2 +- src/Backups/BackupIO_S3.cpp | 8 ++++---- src/Backups/BackupIO_S3.h | 4 ++-- src/Backups/BackupSettings.cpp | 2 +- src/Backups/BackupSettings.h | 2 +- src/Backups/BackupsWorker.cpp | 4 ++-- src/Backups/RestoreSettings.cpp | 2 +- src/Backups/RestoreSettings.h | 2 +- src/Backups/registerBackupEngineS3.cpp | 4 ++-- .../0_stateless/02801_backup_native_copy.reference | 8 ++++---- tests/queries/0_stateless/02801_backup_native_copy.sh | 8 ++++---- 11 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Backups/BackupFactory.h b/src/Backups/BackupFactory.h index 642f5cb07b9..e95aeddb086 100644 --- a/src/Backups/BackupFactory.h +++ b/src/Backups/BackupFactory.h @@ -35,7 +35,7 @@ public: std::shared_ptr backup_coordination; std::optional backup_uuid; bool deduplicate_files = true; - bool native_copy = true; + bool allow_s3_native_copy = true; }; static BackupFactory & instance(); diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 6531948c872..60fea9e2008 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -101,7 +101,7 @@ namespace BackupReaderS3::BackupReaderS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupReaderDefault(&Poco::Logger::get("BackupReaderS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -110,7 +110,7 @@ BackupReaderS3::BackupReaderS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } BackupReaderS3::~BackupReaderS3() = default; @@ -184,7 +184,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s BackupWriterS3::BackupWriterS3( - const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_) + const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_) : BackupWriterDefault(&Poco::Logger::get("BackupWriterS3"), context_) , s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) @@ -193,7 +193,7 @@ BackupWriterS3::BackupWriterS3( { request_settings.updateFromSettings(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint - request_settings.allow_native_copy = native_copy; + request_settings.allow_native_copy = allow_s3_native_copy; } void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 16b2abfea3d..a93d6119786 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -17,7 +17,7 @@ namespace DB class BackupReaderS3 : public BackupReaderDefault { public: - BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupReaderS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupReaderS3() override; bool fileExists(const String & file_name) override; @@ -38,7 +38,7 @@ private: class BackupWriterS3 : public BackupWriterDefault { public: - BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool native_copy, const ContextPtr & context_); + BackupWriterS3(const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, bool allow_s3_native_copy, const ContextPtr & context_); ~BackupWriterS3() override; bool fileExists(const String & file_name) override; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 8e9fe7956f9..b6d776d0347 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes M(Bool, async) \ M(Bool, decrypt_files_from_encrypted_disks) \ M(Bool, deduplicate_files) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(UInt64, shard_num) \ M(UInt64, replica_num) \ M(Bool, internal) \ diff --git a/src/Backups/BackupSettings.h b/src/Backups/BackupSettings.h index e21b70ee25f..7cec2d9693d 100644 --- a/src/Backups/BackupSettings.h +++ b/src/Backups/BackupSettings.h @@ -39,7 +39,7 @@ struct BackupSettings bool deduplicate_files = true; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// 1-based shard index to store in the backup. 0 means all shards. /// Can only be used with BACKUP ON CLUSTER. diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index fddd4f34bb6..c08b110075e 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -348,7 +348,7 @@ void BackupsWorker::doBackup( backup_create_params.backup_coordination = backup_coordination; backup_create_params.backup_uuid = backup_settings.backup_uuid; backup_create_params.deduplicate_files = backup_settings.deduplicate_files; - backup_create_params.native_copy = backup_settings.native_copy; + backup_create_params.allow_s3_native_copy = backup_settings.allow_s3_native_copy; BackupMutablePtr backup = BackupFactory::instance().createBackup(backup_create_params); /// Write the backup. @@ -648,7 +648,7 @@ void BackupsWorker::doRestore( backup_open_params.backup_info = backup_info; backup_open_params.base_backup_info = restore_settings.base_backup_info; backup_open_params.password = restore_settings.password; - backup_open_params.native_copy = restore_settings.native_copy; + backup_open_params.allow_s3_native_copy = restore_settings.allow_s3_native_copy; BackupPtr backup = BackupFactory::instance().createBackup(backup_open_params); String current_database = context->getCurrentDatabase(); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 4dd75911a91..2009ca4c1ff 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -161,7 +161,7 @@ namespace M(RestoreAccessCreationMode, create_access) \ M(Bool, allow_unresolved_access_dependencies) \ M(RestoreUDFCreationMode, create_function) \ - M(Bool, native_copy) \ + M(Bool, allow_s3_native_copy) \ M(Bool, internal) \ M(String, host_id) \ M(OptionalUUID, restore_uuid) diff --git a/src/Backups/RestoreSettings.h b/src/Backups/RestoreSettings.h index 59d73c83d69..1861e219dba 100644 --- a/src/Backups/RestoreSettings.h +++ b/src/Backups/RestoreSettings.h @@ -108,7 +108,7 @@ struct RestoreSettings RestoreUDFCreationMode create_function = RestoreUDFCreationMode::kCreateIfNotExists; /// Whether native copy is allowed (optimization for cloud storages, that sometimes could have bugs) - bool native_copy = true; + bool allow_s3_native_copy = true; /// Internal, should not be specified by user. bool internal = false; diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index ef8ced94590..bd705e4d70f 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -107,12 +107,12 @@ void registerBackupEngineS3(BackupFactory & factory) if (params.open_mode == IBackup::OpenMode::READ) { - auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto reader = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique(backup_name_for_logging, archive_params, params.base_backup_info, reader, params.context); } else { - auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.native_copy, params.context); + auto writer = std::make_shared(S3::URI{s3_uri}, access_key_id, secret_access_key, params.allow_s3_native_copy, params.context); return std::make_unique( backup_name_for_logging, archive_params, diff --git a/tests/queries/0_stateless/02801_backup_native_copy.reference b/tests/queries/0_stateless/02801_backup_native_copy.reference index 659df5e9b25..f9b008cde2e 100644 --- a/tests/queries/0_stateless/02801_backup_native_copy.reference +++ b/tests/queries/0_stateless/02801_backup_native_copy.reference @@ -1,4 +1,4 @@ -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 -RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS native_copy = 1 1 -RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS native_copy = 0 0 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +BACKUP TABLE data TO S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 +RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, \'backups/default/data_native_copy\') SETTINGS allow_s3_native_copy = 1 1 +RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, \'backups/default/data_no_native_copy\') SETTINGS allow_s3_native_copy = 0 0 diff --git a/tests/queries/0_stateless/02801_backup_native_copy.sh b/tests/queries/0_stateless/02801_backup_native_copy.sh index 966d7ae9ce8..015dcb19b82 100755 --- a/tests/queries/0_stateless/02801_backup_native_copy.sh +++ b/tests/queries/0_stateless/02801_backup_native_copy.sh @@ -15,28 +15,28 @@ $CLICKHOUSE_CLIENT -nm -q " " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS native_copy=true" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' " query_id=$(random_str 10) -$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS native_copy=false" +$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false" $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id' From 1844ac37d76ac1a660681acb6b79af8af860d5ff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 28 Jun 2023 19:12:52 +0200 Subject: [PATCH 30/79] Remove create_read_buffer argument for copyS3File() Signed-off-by: Azat Khuzhin --- src/Backups/BackupIO_S3.cpp | 15 --------------- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 14 ++------------ src/IO/S3/copyS3File.cpp | 7 ++++++- src/IO/S3/copyS3File.h | 6 +++--- 4 files changed, 11 insertions(+), 31 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 60fea9e2008..d487ec6e80e 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -152,13 +152,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); - auto create_read_buffer = [this, path_in_backup] - { - return readFile(path_in_backup); - }; - copyS3File( - create_read_buffer, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -208,17 +202,8 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - auto create_read_buffer = [src_disk, src_path, copy_encrypted, settings = read_settings.adjustBufferSize(start_pos + length)] - { - if (copy_encrypted) - return src_disk->readEncryptedFile(src_path, settings); - else - return src_disk->readFile(src_path, settings); - }; - LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( - create_read_buffer, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 3c19af188dc..e46ca3d0828 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -435,12 +435,7 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -456,12 +451,7 @@ void S3ObjectStorage::copyObject( // NOLINT auto settings_ptr = s3_settings.get(); auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - auto create_read_buffer = [this, object_from] - { - return readObject(object_from); - }; - - copyS3File(create_read_buffer, client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 2c6557d97e7..2de2ccd0f9f 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -810,7 +810,6 @@ void copyDataToS3File( void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, @@ -829,7 +828,13 @@ void copyS3File( helper.performCopy(); } else + { + auto create_read_buffer = [&] + { + return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); + }; copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + } } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 2c848076e9b..5d35e5ebe2d 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -23,10 +23,10 @@ using CreateReadBuffer = std::function()>; /// The parameters `src_offset` and `src_size` specify a part in the source to copy. /// /// Note, that it tries to copy file using native copy (CopyObject), but if it -/// has been disabled (with settings.allow_native_copy) it is fallbacks to -/// read-write copy (copyDataToS3File()). +/// has been disabled (with settings.allow_native_copy) or request failed +/// because it is a known issue, it is fallbacks to read-write copy +/// (copyDataToS3File()). void copyS3File( - const CreateReadBuffer & create_read_buffer, const std::shared_ptr & s3_client, const String & src_bucket, const String & src_key, From b95836363085160a20bddfceaaf0709a0e721870 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 8 Jul 2023 14:32:34 +0200 Subject: [PATCH 31/79] tests: temporary fix the 02802_clickhouse_disks_s3_copy In #51135 the behavior of the `clickhouse-disks copy` had been changed, let's temporary update the test (and continue discussion about this change in that PR). Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh index f879b7a5621..33321607728 100755 --- a/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh +++ b/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.sh @@ -19,6 +19,8 @@ function run_test_for_disk() grep -o -e "Single part upload has completed." -e "Single operation copy has completed." } clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test + # NOTE: this is due to "copy" does works like "cp -R from to/" instead of "cp from to" + clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy/test clickhouse-disks -C "$config" --disk "$disk" remove $CLICKHOUSE_DATABASE/test.copy } From ac972661f9718b9d15e5bb49c63b2dff7d296fe3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 08:21:18 +0200 Subject: [PATCH 32/79] Add exclusion for "API mode: {}" from S3 in 00002_log_and_exception_messages_formatting Signed-off-by: Azat Khuzhin --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index acb6117f937..86fe01dc0e3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,7 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From 3c18a181c997f1f43e759d72eeadcc5d4f35142d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 9 Jul 2023 13:54:18 +0200 Subject: [PATCH 33/79] Fix using of pools from the main thread Otherwise it is not possible to use clickhouse-disks with S3: $ clickhouse-disks -C /src/ch/clickhouse/tests/queries/0_stateless/02802_clickhouse_disks_s3_copy.xml --log-level test --disk s3_plain_native_copy copy default/test default/test.copy Failed to make request to: http://localhost:11111/test?list-type=2&max-keys=1&prefix=clickhouse-disks%2Fdefault%2Ftest.copy: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR), Stack trace (when copying this message, always include the lines below): 0. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/exception:134: Poco::Exception::Exception(String const&, int) @ 0x000000001ad7c872 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 1. ./.cmake-llvm16/./src/Common/Exception.cpp:94: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x0000000011e2c4b7 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 2. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(int, char const (&) [34]) @ 0x000000000d341e58 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 3. ./.cmake-llvm16/./src/Common/MemoryTrackerSwitcher.h:19: DB::(anonymous namespace)::SingleEndpointHTTPSessionPool::allocObject() @ 0x0000000012010e5a in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 4. ./.cmake-llvm16/./src/Common/PoolBase.h:174: PoolBase::get(long) @ 0x0000000012011a6f in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 5. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: DB::makePooledHTTPSession(Poco::URI const&, Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200ec69 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 6. ./.cmake-llvm16/./src/IO/HTTPCommon.cpp:0: DB::makePooledHTTPSession(Poco::URI const&, DB::ConnectionTimeouts const&, unsigned long, bool, bool) @ 0x000000001200d909 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 7. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/variant:797: void DB::S3::PocoHTTPClient::makeRequestInternalImpl(Aws::Http::HttpRequest&, DB::S3::ClientConfigurationPerRequest const&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f5157 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 8. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::S3::PocoHTTPClient::makeRequestInternal(Aws::Http::HttpRequest&, std::shared_ptr&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f465d in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 9. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:622: DB::S3::PocoHTTPClient::MakeRequest(std::shared_ptr const&, Aws::Utils::RateLimits::RateLimiterInterface*, Aws::Utils::RateLimits::RateLimiterInterface*) const @ 0x00000000163f4454 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 10. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSClient.cpp:506: Aws::Client::AWSClient::AttemptOneRequest(std::shared_ptr const&, Aws::AmazonWebServiceRequest const&, char const*, char const*, char const*) const @ 0x000000001ae2a922 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 11. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:612: Aws::Client::AWSClient::AttemptExhaustively(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae28299 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 12. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: Aws::Client::AWSXMLClient::MakeRequest(Aws::Http::URI const&, Aws::AmazonWebServiceRequest const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c9ed in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 13. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/source/client/AWSXmlClient.cpp:66: Aws::Client::AWSXMLClient::MakeRequest(Aws::AmazonWebServiceRequest const&, Aws::Endpoint::AWSEndpoint const&, Aws::Http::HttpMethod, char const*, char const*, char const*) const @ 0x000000001ae3c995 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 14. ./.cmake-llvm16/./contrib/aws/generated/src/aws-cpp-sdk-s3/source/S3Client.cpp:0: Aws::S3::S3Client::ListObjectsV2(Aws::S3::Model::ListObjectsV2Request const&) const @ 0x000000001aee6666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 15. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:160: DB::S3::Client::ListObjectsV2(DB::S3::ExtendedRequest const&) const @ 0x00000000163cee42 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 16. ./.cmake-llvm16/./contrib/aws/src/aws-cpp-sdk-core/include/aws/core/utils/Outcome.h:120: DB::S3ObjectStorage::listObjects(String const&, std::vector>&, int) const @ 0x0000000016b582e2 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 17. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:543: DB::IObjectStorage::existsOrHasAnyChild(String const&) const @ 0x000000001644ebe9 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 18. ./.cmake-llvm16/./src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp:0: DB::MetadataStorageFromPlainObjectStorage::exists(String const&) const @ 0x0000000016b54a64 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 19. ./.cmake-llvm16/./src/Disks/IDisk.cpp:145: DB::IDisk::copyDirectoryContent(String const&, std::shared_ptr const&, String const&) @ 0x0000000016b38fa0 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 20. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/string:1499: DB::CommandCopy::execute(std::vector> const&, std::shared_ptr&, Poco::Util::LayeredConfiguration&) @ 0x0000000012050403 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 21. ./.cmake-llvm16/./contrib/llvm-project/libcxx/include/vector:434: DB::DisksApp::main(std::vector> const&) @ 0x000000001204bf02 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 22. ./.cmake-llvm16/./base/poco/Util/src/Application.cpp:0: Poco::Util::Application::run() @ 0x000000001ac7a666 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 23. ./.cmake-llvm16/./programs/disks/DisksApp.cpp:0: mainEntryClickHouseDisks(int, char**) @ 0x000000001204c550 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 24. ./.cmake-llvm16/./programs/main.cpp:0: main @ 0x000000000cfbadc4 in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse 25. ? @ 0x00007ffff7dc9850 in ? 26. __libc_start_main @ 0x00007ffff7dc990a in ? 27. _start @ 0x000000000cfba1ee in /src/ch/clickhouse/.cmake-llvm16/programs/clickhouse (version 23.7.1.1) AWSXmlClient: HTTP response code: -1 Resolved remote host IP address: Request ID: Exception name: Error message: Code: 49. DB::Exception: current_thread is not initialized. (LOGICAL_ERROR) (version 23.7.1.1) 0 response headers: If the signature check failed. This could be because of a time skew. Attempting to adjust the signer. Request failed, now waiting 1600 ms before attempting again. Signed-off-by: Azat Khuzhin --- src/Common/MemoryTrackerSwitcher.h | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/MemoryTrackerSwitcher.h b/src/Common/MemoryTrackerSwitcher.h index 0fefcbb280a..3c99fd12353 100644 --- a/src/Common/MemoryTrackerSwitcher.h +++ b/src/Common/MemoryTrackerSwitcher.h @@ -6,17 +6,13 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - struct MemoryTrackerSwitcher { explicit MemoryTrackerSwitcher(MemoryTracker * new_tracker) { + /// current_thread is not initialized for the main thread, so simply do not switch anything if (!current_thread) - throw Exception(ErrorCodes::LOGICAL_ERROR, "current_thread is not initialized"); + return; auto * thread_tracker = CurrentThread::getMemoryTracker(); prev_untracked_memory = current_thread->untracked_memory; @@ -28,6 +24,10 @@ struct MemoryTrackerSwitcher ~MemoryTrackerSwitcher() { + /// current_thread is not initialized for the main thread, so simply do not switch anything + if (!current_thread) + return; + CurrentThread::flushUntrackedMemory(); auto * thread_tracker = CurrentThread::getMemoryTracker(); @@ -35,6 +35,7 @@ struct MemoryTrackerSwitcher thread_tracker->setParent(prev_memory_tracker_parent); } +private: MemoryTracker * prev_memory_tracker_parent = nullptr; Int64 prev_untracked_memory = 0; }; From 522b9ebf8c62cc564f9a2bcef5802d739a208318 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 26 Jan 2023 05:52:12 -0800 Subject: [PATCH 34/79] Implement KQL-style formatting for Interval --- .../operations/settings/settings-formats.md | 20 +++++++ src/Common/IntervalKind.cpp | 21 +++++++ src/Common/IntervalKind.h | 4 ++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 ++ src/Core/SettingsEnums.h | 2 + src/DataTypes/DataTypeInterval.cpp | 9 ++- src/DataTypes/DataTypeInterval.h | 1 + .../Serializations/SerializationInterval.cpp | 55 +++++++++++++++++++ .../Serializations/SerializationInterval.h | 20 +++++++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 11 ++++ src/Parsers/Kusto/ParserKQLTimespan.cpp | 27 +++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 +++++ ...02366_kql_native_interval_format.reference | 20 +++++++ .../02366_kql_native_interval_format.sql | 22 ++++++++ 16 files changed, 227 insertions(+), 5 deletions(-) create mode 100644 src/DataTypes/Serializations/SerializationInterval.cpp create mode 100644 src/DataTypes/Serializations/SerializationInterval.h create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.cpp create mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.reference create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..7ab234399b9 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_format {#interval_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kql` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 411fc2886e7..9c653eefedb 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -10,6 +10,27 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +Int64 IntervalKind::toAvgNanoseconds() const +{ + static constexpr Int64 NANOSECONDS_PER_MICROSECOND = 1000; + static constexpr auto NANOSECONDS_PER_MILLISECOND = NANOSECONDS_PER_MICROSECOND * 1000; + static constexpr auto NANOSECONDS_PER_SECOND = NANOSECONDS_PER_MILLISECOND * 1000; + + switch (kind) + { + case IntervalKind::Millisecond: + return NANOSECONDS_PER_MILLISECOND; + case IntervalKind::Microsecond: + return NANOSECONDS_PER_MICROSECOND; + case IntervalKind::Nanosecond: + return 1; + default: + return toAvgSeconds() * NANOSECONDS_PER_SECOND; + } + + UNREACHABLE(); +} + Int32 IntervalKind::toAvgSeconds() const { switch (kind) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index b46805655b1..6893286f196 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -29,6 +29,10 @@ struct IntervalKind constexpr std::string_view toString() const { return magic_enum::enum_name(kind); } + /// Returns number of nanoseconds in one interval. + /// For `Month`, `Quarter` and `Year` the function returns an average number of nanoseconds. + Int64 toAvgNanoseconds() const; + /// Returns number of seconds in one interval. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. Int32 toAvgSeconds() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..a2d030102ad 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..48528081258 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kql", FormatSettings::IntervalFormat::KQL}, + {"numeric", FormatSettings::IntervalFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..94dc4f9aee5 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 9faf0cec2d8..1649cb591cf 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,16 +1,16 @@ -#include -#include - +#include "DataTypeInterval.h" +#include "DataTypeFactory.h" +#include "Serializations/SerializationInterval.h" namespace DB { +SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const { return typeid(rhs) == typeid(*this) && kind == static_cast(rhs).kind; } - void registerDataTypeInterval(DataTypeFactory & factory) { factory.registerSimpleDataType("IntervalNanosecond", [] { return DataTypePtr(std::make_shared(IntervalKind::Nanosecond)); }); @@ -25,5 +25,4 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } - } diff --git a/src/DataTypes/DataTypeInterval.h b/src/DataTypes/DataTypeInterval.h index 7de56c13b56..c398a54268e 100644 --- a/src/DataTypes/DataTypeInterval.h +++ b/src/DataTypes/DataTypeInterval.h @@ -24,6 +24,7 @@ public: explicit DataTypeInterval(IntervalKind kind_) : kind(kind_) {} + SerializationPtr doGetDefaultSerialization() const override; std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); } const char * getFamilyName() const override { return "Interval"; } String getSQLCompatibleName() const override { return "TEXT"; } diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp new file mode 100644 index 00000000000..cccb64ed390 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -0,0 +1,55 @@ +#include "SerializationInterval.h" + +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NOT_IMPLEMENTED; +} + +using ColumnInterval = DataTypeInterval::ColumnType; + +SerializationInterval::SerializationInterval(IntervalKind kind_) + : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +{ +} + +void SerializationInterval::serializeText( + const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +{ + const auto * interval_column = checkAndGetColumn(column); + if (!interval_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + + if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) + nested_serialization->serializeText(column, row_num, ostr, format_settings); + else if (format == FormatSettings::IntervalFormat::KQL) + { + const auto & value = interval_column->getData()[row_num]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); +} + +void SerializationInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & format_settings, + [[maybe_unused]] const bool whole) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} +} diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h new file mode 100644 index 00000000000..be5f914235f --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -0,0 +1,20 @@ +#pragma once + +#include "SerializationCustomSimpleText.h" + +#include + +namespace DB +{ +class SerializationInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationInterval(IntervalKind kind_); + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + +private: + IntervalKind kind; +}; +} diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ad991aa0335..fef15f0a56d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -86,6 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.interval.format = settings.interval_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 07d4a7ede4a..bc2e6aa7297 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,6 +77,17 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; + enum class IntervalFormat + { + KQL, + Numeric + }; + + struct + { + IntervalFormat format = IntervalFormat::Numeric; + } interval; + bool input_format_ipv4_default_on_conversion_error = false; bool input_format_ipv6_default_on_conversion_error = false; diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/ParserKQLTimespan.cpp new file mode 100644 index 00000000000..d8ef4ed0227 --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.cpp @@ -0,0 +1,27 @@ +#include "ParserKQLTimespan.h" + +#include + +namespace DB +{ +std::string ParserKQLTimespan::compose(const Int64 ticks) +{ + static constexpr Int64 TICKS_PER_SECOND = 10000000; + static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; + static constexpr auto TICKS_PER_HOUR = TICKS_PER_MINUTE * 60; + static constexpr auto TICKS_PER_DAY = TICKS_PER_HOUR * 24; + + const auto abs_ticks = std::abs(ticks); + std::string result = ticks < 0 ? "-" : ""; + if (abs_ticks >= TICKS_PER_DAY) + result.append(std::format("{}.", abs_ticks / TICKS_PER_DAY)); + + result.append(std::format( + "{:02}:{:02}:{:02}", (abs_ticks / TICKS_PER_HOUR) % 24, (abs_ticks / TICKS_PER_MINUTE) % 60, (abs_ticks / TICKS_PER_SECOND) % 60)); + + if (const auto fractional_second = abs_ticks % TICKS_PER_SECOND) + result.append(std::format(".{:07}", fractional_second)); + + return result; +} +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h new file mode 100644 index 00000000000..8251823253c --- /dev/null +++ b/src/Parsers/Kusto/ParserKQLTimespan.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +#include + +namespace DB +{ +class ParserKQLTimespan +{ +public: + static std::string compose(Int64 ticks); +}; +} diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference new file mode 100644 index 00000000000..fa319341791 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -0,0 +1,20 @@ +default format +99 +100 +1 +42 +66 +66 +5 +1 +2 +kql format +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql new file mode 100644 index 00000000000..db987265bb7 --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql @@ -0,0 +1,22 @@ +select 'default format'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); + +select 'kql format'; +set interval_format = 'kql'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); From 748c7fe04eef101988c555a064155a5878ba221c Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 27 Jan 2023 06:20:40 -0800 Subject: [PATCH 35/79] Refactor functional tests using Jinja2 --- ...02366_kql_native_interval_format.reference | 25 +++++++++++-------- .../02366_kql_native_interval_format.sql | 22 ---------------- .../02366_kql_native_interval_format.sql.j2 | 16 ++++++++++++ 3 files changed, 30 insertions(+), 33 deletions(-) delete mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql create mode 100644 tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index fa319341791..6588ca95e2a 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,14 +1,5 @@ -default format -99 -100 -1 -42 -66 -66 -5 -1 -2 -kql format +numeric +kql 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -18,3 +9,15 @@ kql format 5.00:00:00 7.00:00:00 14.00:00:00 +('00:01:12', '21.00:00:00', '00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +('72', '3', '200') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql b/tests/queries/0_stateless/02366_kql_native_interval_format.sql deleted file mode 100644 index db987265bb7..00000000000 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql +++ /dev/null @@ -1,22 +0,0 @@ -select 'default format'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); - -select 'kql format'; -set interval_format = 'kql'; -select toIntervalNanosecond(99); -select toIntervalNanosecond(100); -select toIntervalMillisecond(1); -select toIntervalSecond(42); -select toIntervalMinute(66); -select toIntervalHour(66); -select toIntervalDay(5); -select toIntervalWeek(1); -select toIntervalWeek(2); diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 new file mode 100644 index 00000000000..6f12dfd96db --- /dev/null +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -0,0 +1,16 @@ +select value from system.settings where name = 'interval_format'; + +{% for format in ['kql', 'numeric'] -%} +select '{{ format }}'; +set interval_format = '{{ format }}'; +select toIntervalNanosecond(99); +select toIntervalNanosecond(100); +select toIntervalMillisecond(1); +select toIntervalSecond(42); +select toIntervalMinute(66); +select toIntervalHour(66); +select toIntervalDay(5); +select toIntervalWeek(1); +select toIntervalWeek(2); +select toIntervalSecond(72) + toIntervalWeek(3) + toIntervalNanosecond(200); +{% endfor -%} From b5cef61ef31fe094d4c756a5032ba2a25cc3b25d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 2 Feb 2023 10:17:24 -0800 Subject: [PATCH 36/79] Refactor solution to improve extensibility --- .../Serializations/SerializationInterval.cpp | 270 ++++++++++++++++-- .../Serializations/SerializationInterval.h | 44 ++- ...02366_kql_native_interval_format.reference | 4 +- 3 files changed, 285 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index cccb64ed390..0cd34a78f8e 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,5 +1,7 @@ #include "SerializationInterval.h" +#include "SerializationCustomSimpleText.h" + #include #include #include @@ -9,47 +11,263 @@ #include +#include +#include + namespace DB { +using ColumnInterval = DataTypeInterval::ColumnType; + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; +} } -using ColumnInterval = DataTypeInterval::ColumnType; - -SerializationInterval::SerializationInterval(IntervalKind kind_) - : SerializationCustomSimpleText(DataTypeFactory::instance().get("Int64")->getDefaultSerialization()), kind(std::move(kind_)) +namespace { -} - -void SerializationInterval::serializeText( - const IColumn & column, const size_t row_num, WriteBuffer & ostr, const FormatSettings & format_settings) const +class SerializationKQLInterval : public DB::SerializationCustomSimpleText { - const auto * interval_column = checkAndGetColumn(column); +public: + explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; + void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; + +private: + DB::IntervalKind kind; +}; + +void SerializationKQLInterval::serializeText( + const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const +{ + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); - if (const auto & format = format_settings.interval.format; format == FormatSettings::IntervalFormat::Numeric) - nested_serialization->serializeText(column, row_num, ostr, format_settings); - else if (format == FormatSettings::IntervalFormat::KQL) - { - const auto & value = interval_column->getData()[row_num]; - const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); - ostr.write(interval_as_string.c_str(), interval_as_string.length()); - } - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + const auto & value = interval_column->getData()[row]; + const auto ticks = kind.toAvgNanoseconds() * value / 100; + const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationInterval::deserializeText( - [[maybe_unused]] IColumn & column, - [[maybe_unused]] ReadBuffer & istr, - [[maybe_unused]] const FormatSettings & format_settings, +void SerializationKQLInterval::deserializeText( + [[maybe_unused]] DB::IColumn & column, + [[maybe_unused]] DB::ReadBuffer & istr, + [[maybe_unused]] const DB::FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); +} + +template Method> +void dispatch( + std::span> serializations, + const Method method, + const DB::FormatSettings::IntervalFormat format, + Args &&... args) +{ + const auto format_index = magic_enum::enum_index(format); + if (!format_index) + throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); + + const auto & serialization = serializations[*format_index]; + if (!serialization) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization.get()->*method)(std::forward(args)...); +} +} + +namespace DB +{ +SerializationInterval::SerializationInterval(IntervalKind kind_) +{ + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) + = std::make_unique(std::move(kind_)); + serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) + = std::make_unique>(); +} + +void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + field, + istr, + settings); +} + +void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::deserializeBinary), + settings.interval.format, + column, + istr, + settings); +} + +void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulk, + FormatSettings::IntervalFormat::Numeric, + column, + istr, + limit, + avg_value_size_hint); +} + +void SerializationInterval::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + + +void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + dispatch( + serializations, + &ISerialization::deserializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + limit, + settings, + state, + cache); +} + + +void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); +} + + +void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); +} + +void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast(&ISerialization::serializeBinary), + settings.interval.format, + field, + ostr, + settings); +} + +void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch( + serializations, + static_cast( + &ISerialization::serializeBinary), + settings.interval.format, + column, + row, + ostr, + settings); +} + +void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); +} + +void SerializationInterval::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); +} + +void SerializationInterval::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); +} + +void SerializationInterval::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const +{ + dispatch( + serializations, + &ISerialization::serializeBinaryBulkWithMultipleStreams, + FormatSettings::IntervalFormat::Numeric, + column, + offset, + limit, + settings, + state); +} + +void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextEscaped( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextQuoted( + const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); +} + +void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const +{ + dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index be5f914235f..5cd13991471 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,20 +1,54 @@ #pragma once -#include "SerializationCustomSimpleText.h" +#include "ISerialization.h" +#include #include namespace DB { -class SerializationInterval : public SerializationCustomSimpleText +class SerializationInterval : public ISerialization { public: explicit SerializationInterval(IntervalKind kind_); - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &, bool whole) const override; + void deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void deserializeBinaryBulkStatePrefix(DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; + void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkStateSuffix(SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextEscaped(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - IntervalKind kind; + std::array, magic_enum::enum_count()> serializations; }; } diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 6588ca95e2a..9d8043f2556 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -9,7 +9,7 @@ kql 5.00:00:00 7.00:00:00 14.00:00:00 -('00:01:12', '21.00:00:00', '00:00:00.0000002') +('00:01:12','21.00:00:00','00:00:00.0000002') numeric 99 100 @@ -20,4 +20,4 @@ numeric 5 1 2 -('72', '3', '200') +(72,3,200) From b673aa8e6b736b339b3644a3bb91bf6d6b2ccf2d Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 27 Feb 2023 12:47:21 -0800 Subject: [PATCH 37/79] Use the dialect configuration --- .../operations/settings/settings-formats.md | 20 --------------- docs/en/operations/settings/settings.md | 20 +++++++++++++++ src/Core/Settings.h | 1 - src/Core/SettingsEnums.cpp | 4 --- src/Core/SettingsEnums.h | 2 -- src/Formats/FormatFactory.cpp | 11 +++++++- ...02366_kql_native_interval_format.reference | 25 +++++++++---------- .../02366_kql_native_interval_format.sql.j2 | 8 +++--- 8 files changed, 45 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 7ab234399b9..1b22a6d1223 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,26 +242,6 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) -## interval_format {#interval_format} - -Allows choosing different output formats of the text representation of interval types. - -Possible values: - -- `kql` - KQL-style output format. - - ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -- `numeric` - Numeric output format. - - ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -Default value: `numeric`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8b969f87a4d..4d5de1ace94 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,6 +98,26 @@ Default value: 0. ``` +## dialect {#dialect} + +Allows choosing support for different query languages. + +Possible values: + +- `clickhouse` - ClickHouse SQL. +- `kusto` - Microsoft KQL. +- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. + +In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +Default value: `clickhouse`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a2d030102ad..3250bdf0750 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,7 +906,6 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ - M(IntervalFormat, interval_format, FormatSettings::IntervalFormat::Numeric, "Textual representation of Interval. Possible values: 'kql', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 48528081258..612f8689480 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,10 +79,6 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) -IMPLEMENT_SETTING_ENUM_WITH_RENAME(IntervalFormat, ErrorCodes::BAD_ARGUMENTS, - {{"kql", FormatSettings::IntervalFormat::KQL}, - {"numeric", FormatSettings::IntervalFormat::Numeric}}) - IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 94dc4f9aee5..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,8 +72,6 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) -DECLARE_SETTING_ENUM_WITH_RENAME(IntervalFormat, FormatSettings::IntervalFormat) - DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index fef15f0a56d..f26de67169a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,6 +19,15 @@ #include +namespace +{ +DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) +{ + return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL + : DB::FormatSettings::IntervalFormat::Numeric; +} +} + namespace DB { @@ -86,7 +95,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = settings.interval_format; + format_settings.interval.format = convert(context->getSettingsRef().dialect); format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 9d8043f2556..06603a2fec6 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,16 +1,4 @@ -numeric -kql -00:00:00 -00:00:00.0000001 -00:00:00.0010000 -00:00:42 -01:06:00 -2.18:00:00 -5.00:00:00 -7.00:00:00 -14.00:00:00 -('00:01:12','21.00:00:00','00:00:00.0000002') -numeric +clickhouse 99 100 1 @@ -21,3 +9,14 @@ numeric 1 2 (72,3,200) +kusto_auto +00:00:00 +00:00:00.0000001 +00:00:00.0010000 +00:00:42 +01:06:00 +2.18:00:00 +5.00:00:00 +7.00:00:00 +14.00:00:00 +('00:01:12','21.00:00:00','00:00:00.0000002') diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index 6f12dfd96db..a6b4535cc39 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,8 +1,6 @@ -select value from system.settings where name = 'interval_format'; - -{% for format in ['kql', 'numeric'] -%} -select '{{ format }}'; -set interval_format = '{{ format }}'; +{% for dialect in ['clickhouse', 'kusto_auto'] -%} +select '{{ dialect }}'; +set dialect = '{{ dialect }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 2d2debe3ce7b1698b85a5771afad077daa6eae4b Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:15:19 -0700 Subject: [PATCH 38/79] Introduce a separate setting for interval output formatting --- .../operations/settings/settings-formats.md | 20 +++ src/Core/Settings.h | 1 + src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 2 + .../Serializations/SerializationInterval.cpp | 135 +++++------------- .../Serializations/SerializationInterval.h | 33 ++++- src/Formats/FormatFactory.cpp | 11 +- src/Formats/FormatSettings.h | 6 +- ...02366_kql_native_interval_format.reference | 25 ++-- .../02366_kql_native_interval_format.sql.j2 | 8 +- 10 files changed, 117 insertions(+), 128 deletions(-) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..53fd9f8ba5b 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -242,6 +242,26 @@ See also: - [DateTime data type.](../../sql-reference/data-types/datetime.md) - [Functions for working with dates and times.](../../sql-reference/functions/date-time-functions.md) +## interval_output_format {#interval_output_format} + +Allows choosing different output formats of the text representation of interval types. + +Possible values: + +- `kusto` - KQL-style output format. + + ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. + +- `numeric` - Numeric output format. + + ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. + +Default value: `numeric`. + +See also: + +- [Interval](../../sql-reference/data-types/special-data-types/interval.md) + ## input_format_ipv4_default_on_conversion_error {#input_format_ipv4_default_on_conversion_error} Deserialization of IPv4 will use default values instead of throwing exception on conversion error. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3250bdf0750..2a78fda03b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -906,6 +906,7 @@ class IColumn; \ M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic', 'best_effort' and 'best_effort_us'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ + M(IntervalOutputFormat, interval_output_format, FormatSettings::IntervalOutputFormat::Numeric, "Textual representation of Interval. Possible values: 'kusto', 'numeric'.", 0) \ \ M(Bool, input_format_ipv4_default_on_conversion_error, false, "Deserialization of IPv4 will use default values instead of throwing exception on conversion error.", 0) \ M(Bool, input_format_ipv6_default_on_conversion_error, false, "Deserialization of IPV6 will use default values instead of throwing exception on conversion error.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 612f8689480..1e2cbce9309 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -79,6 +79,10 @@ IMPLEMENT_SETTING_ENUM(DateTimeOutputFormat, ErrorCodes::BAD_ARGUMENTS, {"iso", FormatSettings::DateTimeOutputFormat::ISO}, {"unix_timestamp", FormatSettings::DateTimeOutputFormat::UnixTimestamp}}) +IMPLEMENT_SETTING_ENUM(IntervalOutputFormat, ErrorCodes::BAD_ARGUMENTS, + {{"kusto", FormatSettings::IntervalOutputFormat::Kusto}, + {"numeric", FormatSettings::IntervalOutputFormat::Numeric}}) + IMPLEMENT_SETTING_AUTO_ENUM(LogsLevel, ErrorCodes::BAD_ARGUMENTS) IMPLEMENT_SETTING_AUTO_ENUM(LogQueriesType, ErrorCodes::BAD_ARGUMENTS) diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 1c5be910ef7..c2783447441 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -72,6 +72,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeInputFormat, FormatSettings::DateTimeIn DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOutputFormat, FormatSettings::DateTimeOutputFormat) +DECLARE_SETTING_ENUM_WITH_RENAME(IntervalOutputFormat, FormatSettings::IntervalOutputFormat) + DECLARE_SETTING_ENUM_WITH_RENAME(ParquetVersion, FormatSettings::ParquetVersion) enum class LogsLevel diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 0cd34a78f8e..9ffa2b31ec3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -1,19 +1,9 @@ #include "SerializationInterval.h" -#include "SerializationCustomSimpleText.h" - -#include -#include -#include -#include +#include #include #include -#include - -#include -#include - namespace DB { using ColumnInterval = DataTypeInterval::ColumnType; @@ -24,80 +14,39 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } -} -namespace +void SerializationKustoInterval::serializeText( + const IColumn & column, const size_t row, WriteBuffer & ostr, const FormatSettings &) const { -class SerializationKQLInterval : public DB::SerializationCustomSimpleText -{ -public: - explicit SerializationKQLInterval(DB::IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } - - void serializeText(const DB::IColumn & column, size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings & settings) const override; - void deserializeText(DB::IColumn & column, DB::ReadBuffer & istr, const DB::FormatSettings & settings, bool whole) const override; - -private: - DB::IntervalKind kind; -}; - -void SerializationKQLInterval::serializeText( - const DB::IColumn & column, const size_t row, DB::WriteBuffer & ostr, const DB::FormatSettings &) const -{ - const auto * interval_column = checkAndGetColumn(column); + const auto * interval_column = checkAndGetColumn(column); if (!interval_column) - throw DB::Exception(DB::ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Expected column of underlying type of Interval"); const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = DB::ParserKQLTimespan::compose(ticks); + const auto interval_as_string = ParserKQLTimespan::compose(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } -void SerializationKQLInterval::deserializeText( - [[maybe_unused]] DB::IColumn & column, - [[maybe_unused]] DB::ReadBuffer & istr, - [[maybe_unused]] const DB::FormatSettings & settings, +void SerializationKustoInterval::deserializeText( + [[maybe_unused]] IColumn & column, + [[maybe_unused]] ReadBuffer & istr, + [[maybe_unused]] const FormatSettings & settings, [[maybe_unused]] const bool whole) const { - throw DB::Exception( - DB::ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, "Deserialization is not implemented for {}", kind.toNameOfFunctionToIntervalDataType()); } -template Method> -void dispatch( - std::span> serializations, - const Method method, - const DB::FormatSettings::IntervalFormat format, - Args &&... args) +SerializationInterval::SerializationInterval(IntervalKind interval_kind_) : interval_kind(std::move(interval_kind_)) { - const auto format_index = magic_enum::enum_index(format); - if (!format_index) - throw DB::Exception(DB::ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, "No such format exists"); - - const auto & serialization = serializations[*format_index]; - if (!serialization) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); - - (serialization.get()->*method)(std::forward(args)...); -} -} - -namespace DB -{ -SerializationInterval::SerializationInterval(IntervalKind kind_) -{ - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::KQL).value()) - = std::make_unique(std::move(kind_)); - serializations.at(magic_enum::enum_index(FormatSettings::IntervalFormat::Numeric).value()) - = std::make_unique>(); } void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, field, istr, settings); @@ -106,9 +55,8 @@ void SerializationInterval::deserializeBinary(Field & field, ReadBuffer & istr, void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::deserializeBinary), - settings.interval.format, + settings.interval.output_format, column, istr, settings); @@ -117,19 +65,13 @@ void SerializationInterval::deserializeBinary(IColumn & column, ReadBuffer & ist void SerializationInterval::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const { dispatch( - serializations, - &ISerialization::deserializeBinaryBulk, - FormatSettings::IntervalFormat::Numeric, - column, - istr, - limit, - avg_value_size_hint); + &ISerialization::deserializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, istr, limit, avg_value_size_hint); } void SerializationInterval::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::deserializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } @@ -141,9 +83,8 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( SubstreamsCache * cache) const { dispatch( - serializations, &ISerialization::deserializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, limit, settings, @@ -154,41 +95,40 @@ void SerializationInterval::deserializeBinaryBulkWithMultipleStreams( void SerializationInterval::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextCSV, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextCSV, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextEscaped, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextEscaped, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextJSON, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextJSON, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextQuoted, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextQuoted, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeTextRaw, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeTextRaw, settings.interval.output_format, column, istr, settings); } void SerializationInterval::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::deserializeWholeText, settings.interval.format, column, istr, settings); + dispatch(&ISerialization::deserializeWholeText, settings.interval.output_format, column, istr, settings); } void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast(&ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, field, ostr, settings); @@ -197,10 +137,9 @@ void SerializationInterval::serializeBinary(const Field & field, WriteBuffer & o void SerializationInterval::serializeBinary(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { dispatch( - serializations, static_cast( &ISerialization::serializeBinary), - settings.interval.format, + settings.interval.output_format, column, row, ostr, @@ -209,29 +148,27 @@ void SerializationInterval::serializeBinary(const IColumn & column, size_t row, void SerializationInterval::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { - dispatch(serializations, &ISerialization::serializeBinaryBulk, FormatSettings::IntervalFormat::Numeric, column, ostr, offset, limit); + dispatch(&ISerialization::serializeBinaryBulk, FormatSettings::IntervalOutputFormat::Numeric, column, ostr, offset, limit); } void SerializationInterval::serializeBinaryBulkStatePrefix( const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch( - serializations, &ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalFormat::Numeric, column, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStatePrefix, FormatSettings::IntervalOutputFormat::Numeric, column, settings, state); } void SerializationInterval::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - dispatch(serializations, &ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalFormat::Numeric, settings, state); + dispatch(&ISerialization::serializeBinaryBulkStateSuffix, FormatSettings::IntervalOutputFormat::Numeric, settings, state); } void SerializationInterval::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { dispatch( - serializations, &ISerialization::serializeBinaryBulkWithMultipleStreams, - FormatSettings::IntervalFormat::Numeric, + FormatSettings::IntervalOutputFormat::Numeric, column, offset, limit, @@ -241,33 +178,33 @@ void SerializationInterval::serializeBinaryBulkWithMultipleStreams( void SerializationInterval::serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeText, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeText, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextCSV(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextCSV, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextCSV, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextEscaped( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextEscaped, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextEscaped, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextJSON(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextJSON, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextJSON, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextQuoted( const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextQuoted, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextQuoted, settings.interval.output_format, column, row, ostr, settings); } void SerializationInterval::serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const { - dispatch(serializations, &ISerialization::serializeTextRaw, settings.interval.format, column, row, ostr, settings); + dispatch(&ISerialization::serializeTextRaw, settings.interval.output_format, column, row, ostr, settings); } } diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index 5cd13991471..e1f0aac0e78 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -1,12 +1,26 @@ #pragma once #include "ISerialization.h" +#include "SerializationCustomSimpleText.h" +#include #include #include namespace DB { +class SerializationKustoInterval : public SerializationCustomSimpleText +{ +public: + explicit SerializationKustoInterval(IntervalKind kind_) : SerializationCustomSimpleText(nullptr), kind(kind_) { } + + void serializeText(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const override; + +private: + IntervalKind kind; +}; + class SerializationInterval : public ISerialization { public: @@ -49,6 +63,23 @@ public: void serializeTextRaw(const IColumn & column, size_t row, WriteBuffer & ostr, const FormatSettings & settings) const override; private: - std::array, magic_enum::enum_count()> serializations; + template Method> + void dispatch(const Method method, const FormatSettings::IntervalOutputFormat format, Args &&... args) const + { + const ISerialization * serialization = nullptr; + if (format == FormatSettings::IntervalOutputFormat::Kusto) + serialization = &serialization_kusto; + else if (format == FormatSettings::IntervalOutputFormat::Numeric) + serialization = &serialization_numeric; + + if (!serialization) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Option {} is not implemented", magic_enum::enum_name(format)); + + (serialization->*method)(std::forward(args)...); + } + + IntervalKind interval_kind; + SerializationKustoInterval serialization_kusto{interval_kind}; + SerializationNumber serialization_numeric; }; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f26de67169a..7f33643366d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -19,15 +19,6 @@ #include -namespace -{ -DB::FormatSettings::IntervalFormat convert(const DB::Dialect dialect) -{ - return dialect == DB::Dialect::kusto || dialect == DB::Dialect::kusto_auto ? DB::FormatSettings::IntervalFormat::KQL - : DB::FormatSettings::IntervalFormat::Numeric; -} -} - namespace DB { @@ -95,7 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; - format_settings.interval.format = convert(context->getSettingsRef().dialect); + format_settings.interval.output_format = settings.interval_output_format; format_settings.input_format_ipv4_default_on_conversion_error = settings.input_format_ipv4_default_on_conversion_error; format_settings.input_format_ipv6_default_on_conversion_error = settings.input_format_ipv6_default_on_conversion_error; format_settings.bool_true_representation = settings.bool_true_representation; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index bc2e6aa7297..e870c61a476 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -77,15 +77,15 @@ struct FormatSettings DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; - enum class IntervalFormat + enum class IntervalOutputFormat { - KQL, + Kusto, Numeric }; struct { - IntervalFormat format = IntervalFormat::Numeric; + IntervalOutputFormat output_format = IntervalOutputFormat::Numeric; } interval; bool input_format_ipv4_default_on_conversion_error = false; diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.reference b/tests/queries/0_stateless/02366_kql_native_interval_format.reference index 06603a2fec6..8a12c6885c4 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.reference +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.reference @@ -1,15 +1,5 @@ -clickhouse -99 -100 -1 -42 -66 -66 -5 -1 -2 -(72,3,200) -kusto_auto +numeric +kusto 00:00:00 00:00:00.0000001 00:00:00.0010000 @@ -20,3 +10,14 @@ kusto_auto 7.00:00:00 14.00:00:00 ('00:01:12','21.00:00:00','00:00:00.0000002') +numeric +99 +100 +1 +42 +66 +66 +5 +1 +2 +(72,3,200) diff --git a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 index a6b4535cc39..0731687222d 100644 --- a/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 +++ b/tests/queries/0_stateless/02366_kql_native_interval_format.sql.j2 @@ -1,6 +1,8 @@ -{% for dialect in ['clickhouse', 'kusto_auto'] -%} -select '{{ dialect }}'; -set dialect = '{{ dialect }}'; +select value from system.settings where name = 'interval_output_format'; + +{% for format in ['kusto', 'numeric'] -%} +select '{{ format }}'; +set interval_output_format = '{{ format }}'; select toIntervalNanosecond(99); select toIntervalNanosecond(100); select toIntervalMillisecond(1); From 54c7e6504fc0d01ad813cea44be5ebced5c91798 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 26 Jun 2023 11:55:23 -0700 Subject: [PATCH 39/79] Fix style issues --- src/DataTypes/Serializations/SerializationInterval.cpp | 1 - src/DataTypes/Serializations/SerializationInterval.h | 5 +++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index 9ffa2b31ec3..a6d22481b88 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -12,7 +12,6 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; - extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; } void SerializationKustoInterval::serializeText( diff --git a/src/DataTypes/Serializations/SerializationInterval.h b/src/DataTypes/Serializations/SerializationInterval.h index e1f0aac0e78..a4e6c204e4f 100644 --- a/src/DataTypes/Serializations/SerializationInterval.h +++ b/src/DataTypes/Serializations/SerializationInterval.h @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class SerializationKustoInterval : public SerializationCustomSimpleText { public: From c2514279e4fdda66491ade958cb334f284d19fba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:31:30 +0300 Subject: [PATCH 40/79] Update settings.md --- docs/en/operations/settings/settings.md | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4d5de1ace94..8b969f87a4d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,26 +98,6 @@ Default value: 0. ``` -## dialect {#dialect} - -Allows choosing support for different query languages. - -Possible values: - -- `clickhouse` - ClickHouse SQL. -- `kusto` - Microsoft KQL. -- `kusto_auto` - Tries ClickHouse SQL first, then Microsoft KQL. - -In mode `clickhouse`, ClickHouse outputs intervals as their underlying numeric representation. For example, `toIntervalDay(2)` would be formatted as `2`. - -In modes `kusto` and `kusto_auto`, ClickHouse outputs intervals in [KQL format](https://learn.microsoft.com/en-us/dotnet/standard/base-types/standard-timespan-format-strings#the-constant-c-format-specifier). For example, `toIntervalDay(2)` would be formatted as `2.00:00:00`. Please note that for interval types of varying length (ie. `IntervalMonth` and `IntervalYear`) the average number of seconds per interval is taken into account. - -Default value: `clickhouse`. - -See also: - -- [Interval](../../sql-reference/data-types/special-data-types/interval.md) - ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). From 8b3848affed8dfbe61a805f7ceeca6689798e4cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:15 +0300 Subject: [PATCH 41/79] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index 1649cb591cf..b8b7640300c 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -1,6 +1,6 @@ -#include "DataTypeInterval.h" -#include "DataTypeFactory.h" -#include "Serializations/SerializationInterval.h" +#include +#include +#include namespace DB { From ba267f05c9dd16c7d02f3f2ec249f72d11014712 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 6 Jul 2023 01:32:42 +0300 Subject: [PATCH 42/79] Update DataTypeInterval.cpp --- src/DataTypes/DataTypeInterval.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/DataTypes/DataTypeInterval.cpp b/src/DataTypes/DataTypeInterval.cpp index b8b7640300c..f8fe8bb3b4b 100644 --- a/src/DataTypes/DataTypeInterval.cpp +++ b/src/DataTypes/DataTypeInterval.cpp @@ -2,8 +2,10 @@ #include #include + namespace DB { + SerializationPtr DataTypeInterval::doGetDefaultSerialization() const { return std::make_shared(kind); } bool DataTypeInterval::equals(const IDataType & rhs) const @@ -25,4 +27,5 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } + } From 9a74b971846c74f566d3ad8a2cc466961a374d56 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 07:50:29 -0700 Subject: [PATCH 43/79] Improve naming --- .../Serializations/SerializationInterval.cpp | 4 ++-- .../{ParserKQLTimespan.cpp => Formatters.cpp} | 4 ++-- src/Parsers/Kusto/Formatters.h | 10 ++++++++++ src/Parsers/Kusto/ParserKQLTimespan.h | 14 -------------- 4 files changed, 14 insertions(+), 18 deletions(-) rename src/Parsers/Kusto/{ParserKQLTimespan.cpp => Formatters.cpp} (90%) create mode 100644 src/Parsers/Kusto/Formatters.h delete mode 100644 src/Parsers/Kusto/ParserKQLTimespan.h diff --git a/src/DataTypes/Serializations/SerializationInterval.cpp b/src/DataTypes/Serializations/SerializationInterval.cpp index a6d22481b88..59086d8aef3 100644 --- a/src/DataTypes/Serializations/SerializationInterval.cpp +++ b/src/DataTypes/Serializations/SerializationInterval.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { @@ -23,7 +23,7 @@ void SerializationKustoInterval::serializeText( const auto & value = interval_column->getData()[row]; const auto ticks = kind.toAvgNanoseconds() * value / 100; - const auto interval_as_string = ParserKQLTimespan::compose(ticks); + const auto interval_as_string = formatKQLTimespan(ticks); ostr.write(interval_as_string.c_str(), interval_as_string.length()); } diff --git a/src/Parsers/Kusto/ParserKQLTimespan.cpp b/src/Parsers/Kusto/Formatters.cpp similarity index 90% rename from src/Parsers/Kusto/ParserKQLTimespan.cpp rename to src/Parsers/Kusto/Formatters.cpp index d8ef4ed0227..f12af479445 100644 --- a/src/Parsers/Kusto/ParserKQLTimespan.cpp +++ b/src/Parsers/Kusto/Formatters.cpp @@ -1,10 +1,10 @@ -#include "ParserKQLTimespan.h" +#include "Formatters.h" #include namespace DB { -std::string ParserKQLTimespan::compose(const Int64 ticks) +std::string formatKQLTimespan(const Int64 ticks) { static constexpr Int64 TICKS_PER_SECOND = 10000000; static constexpr auto TICKS_PER_MINUTE = TICKS_PER_SECOND * 60; diff --git a/src/Parsers/Kusto/Formatters.h b/src/Parsers/Kusto/Formatters.h new file mode 100644 index 00000000000..16f52baf941 --- /dev/null +++ b/src/Parsers/Kusto/Formatters.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +#include + +namespace DB +{ +std::string formatKQLTimespan(Int64 ticks); +} diff --git a/src/Parsers/Kusto/ParserKQLTimespan.h b/src/Parsers/Kusto/ParserKQLTimespan.h deleted file mode 100644 index 8251823253c..00000000000 --- a/src/Parsers/Kusto/ParserKQLTimespan.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - -#include - -namespace DB -{ -class ParserKQLTimespan -{ -public: - static std::string compose(Int64 ticks); -}; -} From e592c7c353fdc8ad7d0e1f23bfcf35fe52273490 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 11 Jul 2023 10:32:33 +0800 Subject: [PATCH 44/79] Fix integration tests - recover clickhouse_stay_alive_command lost by incomplete copy-paste --- tests/integration/helpers/cluster.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3b2c5ff40bb..8b9f75e2545 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3238,6 +3238,9 @@ class ClickHouseInstance: self.clickhouse_start_command = clickhouse_start_command.replace( "{main_config_file}", self.main_config_name ) + self.clickhouse_stay_alive_command = "bash -c \"trap 'pkill tail' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!\"".format( + clickhouse_start_command + ) self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, "docker-compose.yml") @@ -4327,7 +4330,7 @@ class ClickHouseInstance: entrypoint_cmd = self.clickhouse_start_command if self.stay_alive: - entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace( + entrypoint_cmd = self.clickhouse_stay_alive_command.replace( "{main_config_file}", self.main_config_name ) else: From 1739bb306abe11210127fd384c5a278c3e59556d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 10 May 2023 02:26:32 +0800 Subject: [PATCH 45/79] Projection & optimize_aggregators_of_group_by_keys Fix projection with optimize_aggregators_of_group_by_keys = true --- src/Parsers/ASTProjectionSelectQuery.cpp | 7 +++++++ ...jection_optimize_aggregators_of_group_by_keys.reference | 1 + ...10_projection_optimize_aggregators_of_group_by_keys.sql | 7 +++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index da3d9286f0a..7b5889753fa 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -142,6 +142,13 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const } if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); + + auto settings_query = std::make_shared(); + SettingsChanges settings_changes; + settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_query->changes = std::move(settings_changes); + settings_query->is_standalone = false; + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); return node; } diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference new file mode 100644 index 00000000000..738b2b9b918 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.reference @@ -0,0 +1 @@ +2012-10-23 diff --git a/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql new file mode 100644 index 00000000000..ef80dcd7143 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_aggregators_of_group_by_keys.sql @@ -0,0 +1,7 @@ +drop table if exists proj; + +CREATE TABLE proj(date Date, PROJECTION maxdate( SELECT max(date) GROUP BY date )) ENGINE = MergeTree ORDER BY tuple() as select toDate('2012-10-24')-number%100 from numbers(1e2); + +SELECT max(date) FROM proj PREWHERE date != '2012-10-24'; + +drop table proj; From a33b90c648e6797c9272a2a31b689fc58bd680f1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 16 Jun 2023 10:35:55 +0800 Subject: [PATCH 46/79] Projection & optimize_group_by_function_keys --- src/Parsers/ASTProjectionSelectQuery.cpp | 1 + ..._optimize_group_by_function_keys.reference | 0 ...ection_optimize_group_by_function_keys.sql | 31 +++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference create mode 100644 tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 7b5889753fa..0cfdc3762a1 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -146,6 +146,7 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const auto settings_query = std::make_shared(); SettingsChanges settings_changes; settings_changes.insertSetting("optimize_aggregators_of_group_by_keys", false); + settings_changes.insertSetting("optimize_group_by_function_keys", false); settings_query->changes = std::move(settings_changes); settings_query->is_standalone = false; select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_query)); diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql new file mode 100644 index 00000000000..bc9b3151900 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_optimize_group_by_function_keys.sql @@ -0,0 +1,31 @@ +drop table if exists proj; + +create table proj ( + bool_value UInt8, + zero_integer_value Int32, + integer_value Int32, + float_value Float32, + datetime_value DateTime, + string_value String, + projection test_projection ( + select + toStartOfDay (toDateTime (datetime_value)) as Day, + datetime_value, + float_value, + count( + distinct if(zero_integer_value = 1, string_value, NULL) + ) + group by + Day, + datetime_value, + float_value + ) + ) engine MergeTree +partition by + toDate (datetime_value) +order by + bool_value; + +insert into proj values (1, 1, 1, 1, '2012-10-24 21:30:00', 'ab'); + +drop table proj; From 1b403ab3e6ac52b0f56f071388fc07fae74379d6 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 30 May 2023 12:31:22 -0700 Subject: [PATCH 47/79] Fix crc32(WeakHash) issue for s390x --- src/Common/HashTable/Hash.h | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 189603fcbf5..87107fa9f82 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -57,28 +57,25 @@ inline DB::UInt64 intHash64(DB::UInt64 x) inline uint32_t s390x_crc32_u8(uint32_t crc, uint8_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u16(uint32_t crc, uint16_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint32_t s390x_crc32_u32(uint32_t crc, uint32_t v) { - return crc32_be(crc, reinterpret_cast(&v), sizeof(v)); + v = std::byteswap(v); + return crc32c_le_vx(crc, reinterpret_cast(&v), sizeof(v)); } inline uint64_t s390x_crc32(uint64_t crc, uint64_t v) { - uint64_t _crc = crc; - uint32_t value_h, value_l; - value_h = (v >> 32) & 0xffffffff; - value_l = v & 0xffffffff; - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_h), sizeof(uint32_t)); - _crc = crc32_be(static_cast(_crc), reinterpret_cast(&value_l), sizeof(uint32_t)); - return _crc; + v = std::byteswap(v); + return crc32c_le_vx(static_cast(crc), reinterpret_cast(&v), sizeof(uint64_t)); } #endif From 52246ab5b544a3fa5c9482871f90e5a2517c2710 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 11 Jul 2023 23:23:15 +0800 Subject: [PATCH 48/79] Ignore setting constraints for projection ASTs --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/InterpreterSetQuery.cpp | 5 +- src/Interpreters/InterpreterSetQuery.h | 2 +- src/Interpreters/SelectQueryOptions.h | 8 +++ .../optimizeUseAggregateProjection.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 50 ++++++++++++------- 6 files changed, 47 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6ea15312ec4..32812151b59 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3181,7 +3181,7 @@ void InterpreterSelectQuery::initSettings() { auto & query = getSelectQuery(); if (query.settings()) - InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + InterpreterSetQuery(query.settings(), context).executeForCurrentContext(options.ignore_setting_constraints); auto & client_info = context->getClientInfo(); auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; diff --git a/src/Interpreters/InterpreterSetQuery.cpp b/src/Interpreters/InterpreterSetQuery.cpp index c25de7c55ea..6db57a4f950 100644 --- a/src/Interpreters/InterpreterSetQuery.cpp +++ b/src/Interpreters/InterpreterSetQuery.cpp @@ -24,10 +24,11 @@ BlockIO InterpreterSetQuery::execute() } -void InterpreterSetQuery::executeForCurrentContext() +void InterpreterSetQuery::executeForCurrentContext(bool ignore_setting_constraints) { const auto & ast = query_ptr->as(); - getContext()->checkSettingsConstraints(ast.changes); + if (!ignore_setting_constraints) + getContext()->checkSettingsConstraints(ast.changes); getContext()->applySettingsChanges(ast.changes); getContext()->resetSettingsToDefaultValue(ast.default_settings); } diff --git a/src/Interpreters/InterpreterSetQuery.h b/src/Interpreters/InterpreterSetQuery.h index bcd4022f9bb..2438762f347 100644 --- a/src/Interpreters/InterpreterSetQuery.h +++ b/src/Interpreters/InterpreterSetQuery.h @@ -23,7 +23,7 @@ public: /** Set setting for current context (query context). * It is used for interpretation of SETTINGS clause in SELECT query. */ - void executeForCurrentContext(); + void executeForCurrentContext(bool ignore_setting_constraints = false); bool supportsTransactions() const override { return true; } diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index e6895ed243b..c91329c869c 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -51,6 +51,8 @@ struct SelectQueryOptions bool settings_limit_offset_done = false; bool is_explain = false; /// The value is true if it's explain statement. bool is_create_parameterized_view = false; + /// Bypass setting constraints for some internal queries such as projection ASTs. + bool ignore_setting_constraints = false; /// These two fields are used to evaluate shardNum() and shardCount() function when /// prefer_localhost_replica == 1 and local instance is selected. They are needed because local @@ -141,6 +143,12 @@ struct SelectQueryOptions return *this; } + SelectQueryOptions & ignoreSettingConstraints(bool value = true) + { + ignore_setting_constraints = value; + return *this; + } + SelectQueryOptions & setInternal(bool value = false) { is_internal = value; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 2959178b2e5..6e76d6e67a7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -69,7 +69,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( projection.query_ast, context, Pipe(std::make_shared(metadata_snapshot->getSampleBlock())), - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreASTOptimizations().ignoreSettingConstraints()); const auto & analysis_result = interpreter.getAnalysisResult(); const auto & query_analyzer = interpreter.getQueryAnalyzer(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index e568fba0495..73fb279d51c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -7,18 +7,18 @@ #include #include #include +#include #include #include -#include #include #include -#include -#include -#include -#include #include #include +#include +#include +#include +#include #include @@ -109,9 +109,16 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -220,9 +227,16 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, + result.query_ast, + query_context, + storage, + {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState} + .modify() + .ignoreAlias() + .ignoreASTOptimizations() + .ignoreSettingConstraints()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -241,7 +255,8 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.sample_block_for_keys.insert({nullptr, key.type, key.name}); auto it = partition_column_name_to_value_index.find(key.name); if (it == partition_column_name_to_value_index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "minmax_count projection can only have keys about partition columns. It's a bug"); result.partition_value_indices.push_back(it->second); } } @@ -282,7 +297,8 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) Pipe(std::make_shared(block)), SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns - : QueryProcessingStage::WithMergeableState}) + : QueryProcessingStage::WithMergeableState} + .ignoreSettingConstraints()) .buildQueryPipeline(); builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. @@ -353,8 +369,8 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri { if (if_not_exists) return; - throw Exception(ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", - projection.name); + throw Exception( + ErrorCodes::ILLEGAL_PROJECTION, "Cannot add projection {}: projection with this name already exists", projection.name); } auto insert_it = projections.cend(); @@ -363,10 +379,10 @@ void ProjectionsDescription::add(ProjectionDescription && projection, const Stri insert_it = projections.cbegin(); else if (!after_projection.empty()) { - auto it = std::find_if(projections.cbegin(), projections.cend(), [&after_projection](const auto & projection_) - { - return projection_.name == after_projection; - }); + auto it = std::find_if( + projections.cbegin(), + projections.cend(), + [&after_projection](const auto & projection_) { return projection_.name == after_projection; }); if (it != projections.cend()) ++it; insert_it = it; From 3a54182f778ca96cb107f7d934719553447a2733 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:55:55 +0200 Subject: [PATCH 49/79] Fix test_extreme_deduplication flakiness This is part of **"Why my PR is not merged?"** patch set series. Since #50107 there one more tunable that can increase the cleanup period - max_cleanup_delay_period. Plus, increase the delay for the cleanup thread, otherwise it is too fragile. Signed-off-by: Azat Khuzhin --- .../test_extreme_deduplication/configs/conf.d/merge_tree.xml | 1 + tests/integration/test_extreme_deduplication/test.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml index d9062e8da20..6f1d05d125e 100644 --- a/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml +++ b/tests/integration/test_extreme_deduplication/configs/conf.d/merge_tree.xml @@ -4,6 +4,7 @@ 1 1 0 + 1 1 diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 71f783d37c9..9604809f0b5 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -50,7 +50,7 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV(node.query("SELECT count() FROM simple")) == TSV("2\n") # wait clean thread - time.sleep(2) + time.sleep(5) assert ( TSV.toMat( From 2720e9177d9be212e469ee5339decd8dda05a58d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jul 2023 13:29:45 +0200 Subject: [PATCH 50/79] Fix flakiness of test_keeper_s3_snapshot This is part of "Why my PR is not merged?" patch set series. Signed-off-by: Azat Khuzhin --- docker/test/integration/runner/Dockerfile | 1 + .../test_keeper_s3_snapshot/test.py | 29 ++++++++++++------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d6c127c8421..0d1fa00b214 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -98,6 +98,7 @@ RUN python3 -m pip install --no-cache-dir \ redis \ requests-kerberos \ tzlocal==2.1 \ + retry \ urllib3 # Hudi supports only spark 3.3.*, not 3.4 diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 3e19bc4822c..84ffc964621 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -1,6 +1,7 @@ import pytest from helpers.cluster import ClickHouseCluster from time import sleep +from retry import retry from kazoo.client import KazooClient @@ -88,15 +89,19 @@ def test_s3_upload(started_cluster): for obj in list(cluster.minio_client.list_objects("snapshots")) ] - saved_snapshots = get_saved_snapshots() - assert set(saved_snapshots) == set( - [ - "snapshot_50.bin.zstd", - "snapshot_100.bin.zstd", - "snapshot_150.bin.zstd", - "snapshot_200.bin.zstd", - ] - ) + # Keeper sends snapshots asynchornously, hence we need to retry. + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots(): + assert set(get_saved_snapshots()) == set( + [ + "snapshot_50.bin.zstd", + "snapshot_100.bin.zstd", + "snapshot_150.bin.zstd", + "snapshot_200.bin.zstd", + ] + ) + + _check_snapshots() destroy_zk_client(node1_zk) node1.stop_clickhouse(kill=True) @@ -108,9 +113,11 @@ def test_s3_upload(started_cluster): for _ in range(200): node2_zk.create("/test", sequence=True) - saved_snapshots = get_saved_snapshots() + @retry(AssertionError, tries=10, delay=2) + def _check_snapshots_without_quorum(): + assert len(get_saved_snapshots()) > 4 - assert len(saved_snapshots) > 4 + _check_snapshots_without_quorum() success_upload_message = "Successfully uploaded" assert node2.contains_in_log(success_upload_message) or node3.contains_in_log( From 7e1b5d899744bbd1162e0bb4ba22851c576068ff Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:21:23 +0200 Subject: [PATCH 51/79] updated docs for toYearWeek() --- .../functions/date-time-functions.md | 14 +++++++++----- .../functions/date-time-functions.md | 15 +++++++++------ 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3f61e7a214d..4717813a56d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -694,10 +694,14 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we Returns year and week for a date. The year in the result may be different from the year in the date argument for the first and the last week of the year. -The mode argument works exactly like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. +The mode argument works like the mode argument to `toWeek()`. For the single-argument syntax, a mode value of 0 is used. `toISOYear()` is a compatibility function that is equivalent to `intDiv(toYearWeek(date,3),100)`. +:::warning +The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. + + **Syntax** ``` sql @@ -707,13 +711,13 @@ toYearWeek(t[, mode[, timezone]]) **Example** ``` sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` ``` text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 17ab04b7799..d251163df8d 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -599,24 +599,27 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we ## toYearWeek(date[,mode]) {#toyearweek} Возвращает год и неделю для даты. Год в результате может отличаться от года в аргументе даты для первой и последней недели года. -Аргумент mode работает точно так же, как аргумент mode [toWeek()](#toweek). Если mode не задан, используется режим 0. +Аргумент mode работает так же, как аргумент mode [toWeek()](#toweek), значение mode по умолчанию -- `0`. -`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)`. +`toISOYear() ` эквивалентно `intDiv(toYearWeek(date,3),100)` + +:::warning +Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). **Пример** Запрос: ```sql -SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; +SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9, toYearWeek(toDate('2022-01-01')) AS prev_yearWeek; ``` Результат: ```text -┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ -│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ -└────────────┴───────────┴───────────┴───────────┘ +┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┬─prev_yearWeek─┐ +│ 2016-12-27 │ 201652 │ 201652 │ 201701 │ 202152 │ +└────────────┴───────────┴───────────┴───────────┴───────────────┘ ``` ## age From a87474cee1aeb924cb11c043308c36888f6a9a26 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 17:25:49 +0200 Subject: [PATCH 52/79] rm \n --- docs/en/sql-reference/functions/date-time-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4717813a56d..ddd6a367e31 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -701,7 +701,6 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. - **Syntax** ``` sql From 0de7fb5c620eb6d7ee010f2053737f95577b77f8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:12:59 +0300 Subject: [PATCH 53/79] typofix1 --- docs/en/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index fa5826fbdda..ce1a4f4d283 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -700,6 +700,7 @@ The mode argument works like the mode argument to `toWeek()`. For the single-arg :::warning The week number returned by `toYearWeek()` can be different from what the `toWeek()` returns. `toWeek()` always returns week number in the context of the given year, and in case `toWeek()` returns `0`, `toYearWeek()` returns the value corresponding to the last week of previous year. See `prev_yearWeek` in example below. +::: **Syntax** From 54723d4822438cae9dc009c5b28e4f5fbadd1741 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 13 Jul 2023 20:14:06 +0300 Subject: [PATCH 54/79] typofix2 --- docs/ru/sql-reference/functions/date-time-functions.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 8f31fcebf8b..4db8a1ec6f8 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -605,6 +605,7 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we :::warning Однако, есть отличие в работе функций `toWeek()` и `toYearWeek()`. `toWeek()` возвращает номер недели в контексте заданного года, и в случае, когда `toWeek()` вернёт `0`, `toYearWeek()` вернёт значение, соответствующее последней неделе предыдущего года (см. `prev_yearWeek` в примере). +::: **Пример** From f7505f0c7749d06f53dbfaf345f00127ca5ad012 Mon Sep 17 00:00:00 2001 From: zvonand Date: Thu, 13 Jul 2023 19:41:45 +0200 Subject: [PATCH 55/79] update docs --- docs/en/sql-reference/data-types/datetime.md | 1 + docs/en/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 6 +++++- docs/ru/sql-reference/data-types/datetime.md | 1 + docs/ru/sql-reference/data-types/datetime64.md | 1 + .../sql-reference/functions/type-conversion-functions.md | 8 +++++++- 6 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 0da273e01ad..fe279edb709 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -143,5 +143,6 @@ Time shifts for multiple days. Some pacific islands changed their timezone offse - [The `date_time_input_format` setting](../../operations/settings/settings.md#settings-date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings.md#settings-date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-datetime) - [The `Date` data type](../../sql-reference/data-types/date.md) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 793691850b1..3b80e8b1a8b 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -119,6 +119,7 @@ FROM dt; - [The `date_time_input_format` setting](../../operations/settings/settings-formats.md#date_time_input_format) - [The `date_time_output_format` setting](../../operations/settings/settings-formats.md#date_time_output_format) - [The `timezone` server configuration parameter](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [The `session_timezone` setting](../../operations/settings/settings.md#session_timezone) - [Operators for working with dates and times](../../sql-reference/operators/index.md#operators-for-working-with-dates-and-times) - [`Date` data type](../../sql-reference/data-types/date.md) - [`DateTime` data type](../../sql-reference/data-types/datetime.md) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 214c885bc0e..36f40b37238 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -399,7 +399,11 @@ toDateTime(expr[, time_zone ]) - `expr` — The value. [String](/docs/en/sql-reference/data-types/string.md), [Int](/docs/en/sql-reference/data-types/int-uint.md), [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md). - `time_zone` — Time zone. [String](/docs/en/sql-reference/data-types/string.md). -If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +:::note +If `expr` is a number, it is interpreted as the number of seconds since the beginning of the Unix Epoch (as Unix timestamp). +If `expr` is a [String](/docs/en/sql-reference/data-types/string.md), it may be interpreted as a Unix timestamp or as a string representation of date / date with time. +Thus, parsing of short numbers' string representations (up to 4 digits) is explicitly disabled due to ambiguity, e.g. a string `'1999'` may be both a year (an incomplete string representation of Date / DateTime) or a unix timestamp. Longer numeric strings are allowed. +::: **Returned value** diff --git a/docs/ru/sql-reference/data-types/datetime.md b/docs/ru/sql-reference/data-types/datetime.md index e8d4a3ee9fd..80d844a1713 100644 --- a/docs/ru/sql-reference/data-types/datetime.md +++ b/docs/ru/sql-reference/data-types/datetime.md @@ -122,6 +122,7 @@ FROM dt - [Настройка `date_time_input_format`](../../operations/settings/index.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/index.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime64`](datetime64.md) diff --git a/docs/ru/sql-reference/data-types/datetime64.md b/docs/ru/sql-reference/data-types/datetime64.md index da2f81f4828..78ad43e4764 100644 --- a/docs/ru/sql-reference/data-types/datetime64.md +++ b/docs/ru/sql-reference/data-types/datetime64.md @@ -102,6 +102,7 @@ FROM dt; - [Настройка `date_time_input_format`](../../operations/settings/settings.md#settings-date_time_input_format) - [Настройка `date_time_output_format`](../../operations/settings/settings.md) - [Конфигурационный параметр сервера `timezone`](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) +- [Параметр `session_timezone`](../../operations/settings/settings.md#session_timezone) - [Операторы для работы с датой и временем](../../sql-reference/operators/index.md#operators-datetime) - [Тип данных `Date`](date.md) - [Тип данных `DateTime`](datetime.md) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index d5e6246fe9e..e53104d8d71 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -284,7 +284,13 @@ toDateTime(expr[, time_zone ]) - `expr` — Значение для преобразования. [String](/docs/ru/sql-reference/data-types/string.md), [Int](/docs/ru/sql-reference/data-types/int-uint.md), [Date](/docs/ru/sql-reference/data-types/date.md) или [DateTime](/docs/ru/sql-reference/data-types/datetime.md). - `time_zone` — Часовой пояс. [String](/docs/ru/sql-reference/data-types/string.md). -Если `expr` является числом, оно интерпретируется как количество секунд от начала unix эпохи. +:::note +Если `expr` является числом, то оно интерпретируется как число секунд с начала Unix-эпохи (Unix Timestamp). + +Если же `expr` -- [строка (String)](/docs/ru/sql-reference/data-types/string.md), то оно может быть интерпретировано и как Unix Timestamp, и как строковое представление даты / даты со временем. +Ввиду неоднозначности запрещён парсинг строк длиной 4 и меньше. Так, строка `'1999'` могла бы представлять собой как год (неполное строковое представление даты или даты со временем), так и Unix Timestamp. +Строки длиной 5 символов и более не несут неоднозначности, а следовательно, их парсинг разрешён. +::: **Возвращаемое значение** From 8ea3bf4ade73ef3188026d5ac64db48ab7fa09a5 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 14 Jul 2023 16:09:22 +0800 Subject: [PATCH 56/79] improve ch to arrow --- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c3685e813d3..899b84cc132 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -208,7 +208,7 @@ namespace DB const String & column_name, ColumnPtr & column, const DataTypePtr & column_type, - const PaddedPODArray * null_bytemap, + const PaddedPODArray *, arrow::ArrayBuilder * array_builder, String format_name, size_t start, @@ -231,7 +231,9 @@ namespace DB /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); + + /// Pass null null_map, because fillArrowArray will decide whether nested_type is nullable, if nullable, it will create a new null_map from nested_column + fillArrowArray(column_name, nested_column, nested_type, nullptr, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } From 103a689edc1291e6b243c8000c8100b043caf8c6 Mon Sep 17 00:00:00 2001 From: therealnick233 Date: Fri, 14 Jul 2023 18:36:42 +0800 Subject: [PATCH 57/79] Fixed typo in combinators.md line 303, 'Jonh' -> 'John' --- docs/en/sql-reference/aggregate-functions/combinators.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index a395b350a55..18ff5073e3f 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -300,7 +300,7 @@ SELECT groupArrayResample(30, 75, 30)(name, age) FROM people Consider the results. -`Jonh` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. +`John` is out of the sample because he’s too young. Other people are distributed according to the specified age intervals. Now let’s count the total number of people and their average wage in the specified age intervals. From f7705fff58e871913028a1683c6d7b21a82b9c16 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 14 Jul 2023 12:45:05 +0200 Subject: [PATCH 58/79] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 1036e1a97e1..932a10be0ab 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**v23.6 Release Webinar**](https://clickhouse.com/company/events/v23-6-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-06) - Jun 29 - 23.6 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**v23.7 Release Webinar**](https://clickhouse.com/company/events/v23-7-community-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-07) - Jul 27 - 23.7 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 @@ -34,13 +34,13 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.4 Release Webinar**](https://www.youtube.com/watch?v=4rrf6bk_mOg) Faster Parquet Reading, Asynchonous Connections to Reoplicas, Trailing Comma before FROM, extractKeyValuePairs, integrations updates, and so much more! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) - ## Interested in joining ClickHouse and making it your full time job? + ## Interested in joining ClickHouse and making it your full-time job? -We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker as well as a doer - we’ll definitely click! +We are a globally diverse and distributed team, united behind a common goal of creating industry-leading, real-time analytics. Here, you will have an opportunity to solve some of the most cutting-edge technical challenges and have direct ownership of your work and vision. If you are a contributor by nature, a thinker and a doer - we’ll definitely click! Check out our **current openings** here: https://clickhouse.com/company/careers From dff5d78126318782cc7f3935c38013b010695c07 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Sun, 18 Jun 2023 02:03:14 +0000 Subject: [PATCH 59/79] Add jemalloc support --- contrib/jemalloc-cmake/CMakeLists.txt | 4 +- .../internal/jemalloc_internal_defs.h.in | 435 ++++++++++++++++++ 2 files changed, 438 insertions(+), 1 deletion(-) create mode 100644 contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 20025dfc63e..9a4ad372be3 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) @@ -148,6 +148,8 @@ elseif (ARCH_PPC64LE) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_ppc64le") elseif (ARCH_RISCV64) set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_riscv64") +elseif (ARCH_S390X) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_s390x") else () message (FATAL_ERROR "internal jemalloc: This arch is not supported") endif () diff --git a/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..531f2bca0c2 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_s390x/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,435 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#define JEMALLOC_OVERRIDE___LIBC_CALLOC +#define JEMALLOC_OVERRIDE___LIBC_FREE +#define JEMALLOC_OVERRIDE___LIBC_MALLOC +#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN +#define JEMALLOC_OVERRIDE___LIBC_REALLOC +#define JEMALLOC_OVERRIDE___LIBC_VALLOC +#define JEMALLOC_OVERRIDE___LIBC_PVALLOC +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +#define JEMALLOC_HAVE_SECURE_GETENV + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* JEMALLOC_PAGEID enabled page id */ +/* #undef JEMALLOC_PAGEID */ + +/* JEMALLOC_HAVE_PRCTL checks prctl */ +#define JEMALLOC_HAVE_PRCTL + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 20 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +#define JEMALLOC_BIG_ENDIAN + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +#define JEMALLOC_ENABLE_CXX + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +/* If defined, realloc(ptr, 0) defaults to "free" instead of "alloc". */ +#define JEMALLOC_ZERO_REALLOC_DEFAULT_FREE + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From efa78a377eaebd5a8c369f56732e63a51eb6fd07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:16 +0300 Subject: [PATCH 60/79] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 9a4ad372be3..ab4d9d46c63 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X )) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From b2546c8bb524df19a376f925ec14d1a730b72768 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:54:34 +0300 Subject: [PATCH 61/79] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index ab4d9d46c63..f2b37d7efae 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (SANITIZE OR NOT ( - ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR + ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_AARCH64 OR ARCH_PPC64LE OR ARCH_RISCV64 OR ARCH_S390X)) OR (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) From f0f69a2e3a7dabb0169c785e34f6c4a6639ccd83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 16:55:57 +0300 Subject: [PATCH 62/79] Update CMakeLists.txt --- contrib/jemalloc-cmake/CMakeLists.txt | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index f2b37d7efae..15e965ed841 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -17,17 +17,17 @@ if (NOT ENABLE_JEMALLOC) endif () if (NOT OS_LINUX) - message (WARNING "jemalloc support on non-linux is EXPERIMENTAL") + message (WARNING "jemalloc support on non-Linux is EXPERIMENTAL") endif() if (OS_LINUX) - # ThreadPool select job randomly, and there can be some threads that had been - # performed some memory heavy task before and will be inactive for some time, - # but until it will became active again, the memory will not be freed since by - # default each thread has it's own arena, but there should be not more then + # ThreadPool select job randomly, and there can be some threads that have been + # performed some memory-heavy tasks before and will be inactive for some time, + # but until it becomes active again, the memory will not be freed since, by + # default, each thread has its arena, but there should be no more than # 4*CPU arenas (see opt.nareans description). # - # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # By enabling percpu_arena number of arenas is limited to the number of CPUs, and hence # this problem should go away. # # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to @@ -38,7 +38,7 @@ if (OS_LINUX) else() set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000") endif() -# CACHE variable is empty, to allow changing defaults without necessity +# CACHE variable is empty to allow changing defaults without the necessity # to purge cache set (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE "" CACHE STRING "Change default configuration string of JEMalloc" ) if (JEMALLOC_CONFIG_MALLOC_CONF_OVERRIDE) @@ -174,7 +174,7 @@ target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1) # jemalloc provides support for two different libunwind flavors: the original HP libunwind and the one coming with gcc / g++ / libstdc++. # The latter is identified by `JEMALLOC_PROF_LIBGCC` and uses `_Unwind_Backtrace` method instead of `unw_backtrace`. -# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracing. +# At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking. # # ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1). target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) From f762a0e61523c1e46eb690a59a2452e4e8d0f3af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 18:56:53 +0300 Subject: [PATCH 63/79] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 932a10be0ab..f0a7dbe2408 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ Also, keep an eye out for upcoming meetups around the world. Somewhere else you ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" -* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.7, one convenient video! Watch it now! +* **Recording available**: [**v23.6 Release Webinar**](https://www.youtube.com/watch?v=cuf_hYn7dqU) All the features of 23.6, one convenient video! Watch it now! * **All release webinar recordings**: [YouTube playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3jAlSy1JxyP8zluvXaN3nxU) From 32bc5866ea1b2622bbd0b18d92d748b4d6a4f86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:04:58 +0200 Subject: [PATCH 64/79] Update build.md for clang-16 --- docs/en/development/build.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..dfec20be58e 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -42,7 +42,7 @@ sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test For other Linux distribution - check the availability of LLVM's [prebuild packages](https://releases.llvm.org/download.html). -As of April 2023, any version of Clang >= 15 will work. +As of April 2023, clang-16 or higher will work. GCC as a compiler is not supported. To build with a specific Clang version: @@ -86,8 +86,8 @@ The build requires the following components: - Git (used to checkout the sources, not needed for the build) - CMake 3.20 or newer -- Compiler: Clang 15 or newer -- Linker: lld 15 or newer +- Compiler: clang-16 or newer +- Linker: lld-16 or newer - Ninja - Yasm - Gawk From 94112895a160bb9eddc226aff55f05f56b55b143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 21:09:58 +0300 Subject: [PATCH 65/79] Add RISC-V 64 to the docs --- docs/en/development/build.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..d55f7d8cced 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -11,7 +11,8 @@ Supported platforms: - x86_64 - AArch64 -- Power9 (experimental) +- PowerPC 64 LE (experimental) +- RISC-V 64 (experimental) ## Building on Ubuntu From 6abdde12cafb74b45cf5cb650caac133606dab45 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 18:31:45 +0200 Subject: [PATCH 66/79] Print Zxid in keeper stat command in hex (so as ZooKeeper) Signed-off-by: Azat Khuzhin --- src/Coordination/FourLetterCommand.cpp | 18 ++++++++++++++++-- .../test_keeper_four_word_command/test.py | 4 ++-- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 0490213d296..d1ff03482b6 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -18,6 +18,20 @@ #include #include +namespace +{ + +String formatZxid(int64_t zxid) +{ + /// ZooKeeper print zxid in hex and + String hex = getHexUIntLowercase(zxid); + /// without leading zeros + trimLeft(hex, '0'); + return "0x" + hex; +} + +} + namespace DB { @@ -348,7 +362,7 @@ String ServerStatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); @@ -381,7 +395,7 @@ String StatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", toString(keeper_info.last_zxid)); + write("Zxid", formatZxid(keeper_info.last_zxid)); write("Mode", keeper_info.getRole()); write("Node count", toString(keeper_info.total_nodes_count)); diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index b546d879c75..1d5bc6a6541 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -329,7 +329,7 @@ def test_cmd_srvr(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) > 10 + assert int(result["Zxid"], 16) > 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" @@ -369,7 +369,7 @@ def test_cmd_stat(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) >= 10 + assert int(result["Zxid"], 16) >= 10 assert result["Mode"] == "leader" assert result["Node count"] == "14" From 9307e60c20fd634da1e88f2e839a30c963e7f4c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 23:21:26 +0300 Subject: [PATCH 67/79] Revert "Millisecond and microsecond support in date_diff / age functions" --- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 6 +- .../functions/date-time-functions.md | 2 - src/Core/DecimalFunctions.h | 6 +- src/Functions/DateTimeTransforms.h | 71 ++---------- src/Functions/TransformDateTime64.h | 7 +- src/Functions/dateDiff.cpp | 58 +++------- src/Functions/toStartOfInterval.cpp | 1 + ...0479_date_and_datetime_to_number.reference | 1 - .../00479_date_and_datetime_to_number.sql | 1 - .../02160_special_functions.reference | 10 -- .../0_stateless/02160_special_functions.sql | 12 -- .../02477_age_datetime64.reference | 106 ------------------ .../0_stateless/02477_age_datetime64.sql | 65 ----------- 14 files changed, 29 insertions(+), 323 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 40ede6e0b62..3f61e7a214d 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -718,7 +718,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 microsecond. +Returns the `unit` component of the difference between `startdate` and `enddate`. The difference is calculated using a precision of 1 second. E.g. the difference between `2021-12-29` and `2022-01-01` is 3 days for `day` unit, 0 months for `month` unit, 0 years for `year` unit. For an alternative to `age`, see function `date\_diff`. @@ -734,8 +734,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) @@ -811,8 +809,6 @@ Aliases: `dateDiff`, `DATE_DIFF`, `timestampDiff`, `timestamp_diff`, `TIMESTAMP_ - `unit` — The type of interval for result. [String](../../sql-reference/data-types/string.md). Possible values: - - `microsecond` (possible abbreviations: `us`, `u`) - - `millisecond` (possible abbreviations: `ms`) - `second` (possible abbreviations: `ss`, `s`) - `minute` (possible abbreviations: `mi`, `n`) - `hour` (possible abbreviations: `hh`, `h`) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 864ae6e905e..17ab04b7799 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -621,7 +621,7 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d ## age -Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 микросекунду. +Вычисляет компонент `unit` разницы между `startdate` и `enddate`. Разница вычисляется с точностью в 1 секунду. Например, разница между `2021-12-29` и `2022-01-01` 3 дня для единицы `day`, 0 месяцев для единицы `month`, 0 лет для единицы `year`. **Синтаксис** @@ -635,8 +635,6 @@ age('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) @@ -710,8 +708,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — единица измерения времени, в которой будет выражено возвращаемое значение функции. [String](../../sql-reference/data-types/string.md). Возможные значения: - - `microsecond` (возможные сокращения: `us`, `u`) - - `millisecond` (возможные сокращения: `ms`) - `second` (возможные сокращения: `ss`, `s`) - `minute` (возможные сокращения: `mi`, `n`) - `hour` (возможные сокращения: `hh`, `h`) diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index e4b70322477..53dadc23c6d 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -643,8 +643,6 @@ date_diff('unit', startdate, enddate, [timezone]) - `unit` — `value`对应的时间单位。类型为[String](../../sql-reference/data-types/string.md)。 可能的值: - - `microsecond` - - `millisecond` - `second` - `minute` - `hour` diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 17d95650730..357cff2c541 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -48,11 +48,7 @@ inline auto scaleMultiplier(UInt32 scale) /** Components of DecimalX value: * whole - represents whole part of decimal, can be negative or positive. - * fractional - for fractional part of decimal. - * - * 0.123 represents 0 / 0.123 - * -0.123 represents 0 / -0.123 - * -1.123 represents -1 / 0.123 + * fractional - for fractional part of decimal, always positive. */ template struct DecimalComponents diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index a1c880f6956..510a88db2b6 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -19,9 +19,6 @@ namespace DB { -static constexpr auto microsecond_multiplier = 1000000; -static constexpr auto millisecond_multiplier = 1000; - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -1380,36 +1377,6 @@ struct ToRelativeSecondNumImpl using FactorTransform = ZeroTransform; }; -template -struct ToRelativeSubsecondNumImpl -{ - static constexpr auto name = "toRelativeSubsecondNumImpl"; - - static inline Int64 execute(const DateTime64 & t, DateTime64::NativeType scale, const DateLUTImpl &) - { - static_assert(scale_multiplier == 1000 || scale_multiplier == 1000000); - if (scale == scale_multiplier) - return t.value; - if (scale > scale_multiplier) - return t.value / (scale / scale_multiplier); - return t.value * (scale_multiplier / scale); - } - static inline Int64 execute(UInt32 t, const DateLUTImpl &) - { - return t * scale_multiplier; - } - static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; - } - static inline Int64 execute(UInt16 d, const DateLUTImpl & time_zone) - { - return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); - } - - using FactorTransform = ZeroTransform; -}; - struct ToYYYYMMImpl { static constexpr auto name = "toYYYYMM"; @@ -1509,47 +1476,25 @@ struct ToYYYYMMDDhhmmssImpl using FactorTransform = ZeroTransform; }; -struct DateTimeComponentsWithFractionalPart : public DateLUTImpl::DateTimeComponents -{ - UInt16 millisecond; - UInt16 microsecond; -}; - struct ToDateTimeComponentsImpl { static constexpr auto name = "toDateTimeComponents"; - static inline DateTimeComponentsWithFractionalPart execute(const DateTime64 & t, DateTime64::NativeType scale_multiplier, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int64 t, const DateLUTImpl & time_zone) { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - - if (t.value < 0 && components.fractional) - { - components.fractional = scale_multiplier + (components.whole ? Int64(-1) : Int64(1)) * components.fractional; - --components.whole; - } - Int64 fractional = components.fractional; - if (scale_multiplier > microsecond_multiplier) - fractional = fractional / (scale_multiplier / microsecond_multiplier); - else if (scale_multiplier < microsecond_multiplier) - fractional = fractional * (microsecond_multiplier / scale_multiplier); - - constexpr Int64 divider = microsecond_multiplier/ millisecond_multiplier; - UInt16 millisecond = static_cast(fractional / divider); - UInt16 microsecond = static_cast(fractional % divider); - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(components.whole), millisecond, microsecond}; + return time_zone.toDateTimeComponents(t); } - static inline DateTimeComponentsWithFractionalPart execute(UInt32 t, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt32 t, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(static_cast(t)), 0, 0}; + return time_zone.toDateTimeComponents(static_cast(t)); } - static inline DateTimeComponentsWithFractionalPart execute(Int32 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(Int32 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(ExtendedDayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(ExtendedDayNum(d)); } - static inline DateTimeComponentsWithFractionalPart execute(UInt16 d, const DateLUTImpl & time_zone) + static inline DateLUTImpl::DateTimeComponents execute(UInt16 d, const DateLUTImpl & time_zone) { - return DateTimeComponentsWithFractionalPart{time_zone.toDateTimeComponents(DayNum(d)), 0, 0}; + return time_zone.toDateTimeComponents(DayNum(d)); } using FactorTransform = ZeroTransform; diff --git a/src/Functions/TransformDateTime64.h b/src/Functions/TransformDateTime64.h index fcee2753066..3dab9efeb6b 100644 --- a/src/Functions/TransformDateTime64.h +++ b/src/Functions/TransformDateTime64.h @@ -5,7 +5,7 @@ namespace DB { -/** Transform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. +/** Tansform-type wrapper for DateTime64, simplifies DateTime64 support for given Transform. * * Depending on what overloads of Transform::execute() are available, when called with DateTime64 value, * invokes Transform::execute() with either: @@ -80,10 +80,7 @@ public: } else { - auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); - if (t.value < 0 && components.fractional) - --components.whole; - + const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier); return wrapped_transform.execute(static_cast(components.whole), std::forward(args)...); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 6bfbbb7c735..8361e9db166 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -174,13 +174,12 @@ public: { auto res = static_cast(transform_y.execute(y, timezone_y)) - static_cast(transform_x.execute(x, timezone_x)); - DateTimeComponentsWithFractionalPart a_comp; - DateTimeComponentsWithFractionalPart b_comp; + DateLUTImpl::DateTimeComponents a_comp; + DateLUTImpl::DateTimeComponents b_comp; Int64 adjust_value; - auto x_microseconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); - auto y_microseconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); - - if (x_microseconds <= y_microseconds) + auto x_seconds = TransformDateTime64>(transform_x.getScaleMultiplier()).execute(x, timezone_x); + auto y_seconds = TransformDateTime64>(transform_y.getScaleMultiplier()).execute(y, timezone_y); + if (x_seconds <= y_seconds) { a_comp = TransformDateTime64(transform_x.getScaleMultiplier()).execute(x, timezone_x); b_comp = TransformDateTime64(transform_y.getScaleMultiplier()).execute(y, timezone_y); @@ -193,16 +192,14 @@ public: adjust_value = 1; } - if constexpr (std::is_same_v>>) { if ((a_comp.date.month > b_comp.date.month) || ((a_comp.date.month == b_comp.date.month) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -213,9 +210,8 @@ public: || ((x_month_in_quarter == y_month_in_quarter) && ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -223,9 +219,8 @@ public: if ((a_comp.date.day > b_comp.date.day) || ((a_comp.date.day == b_comp.date.day) && ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second)))) + ))) res += adjust_value; } else if constexpr (std::is_same_v>>) @@ -235,44 +230,25 @@ public: if ((x_day_of_week > y_day_of_week) || ((x_day_of_week == y_day_of_week) && (a_comp.time.hour > b_comp.time.hour)) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.hour > b_comp.time.hour) || ((a_comp.time.hour == b_comp.time.hour) && ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))))) res += adjust_value; } else if constexpr (std::is_same_v>>) { if ((a_comp.time.minute > b_comp.time.minute) - || ((a_comp.time.minute == b_comp.time.minute) && ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))))) + || ((a_comp.time.minute == b_comp.time.minute) && (a_comp.time.second > b_comp.time.second))) res += adjust_value; } else if constexpr (std::is_same_v>>) { - if ((a_comp.time.second > b_comp.time.second) - || ((a_comp.time.second == b_comp.time.second) && ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if ((a_comp.millisecond > b_comp.millisecond) - || ((a_comp.millisecond == b_comp.millisecond) && (a_comp.microsecond > b_comp.microsecond))) - res += adjust_value; - } - else if constexpr (std::is_same_v>>) - { - if (a_comp.microsecond > b_comp.microsecond) + if (a_comp.time.second > b_comp.time.second) res += adjust_value; } return res; @@ -397,10 +373,6 @@ public: impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "millisecond" || unit == "ms") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); - else if (unit == "microsecond" || unit == "us" || unit == "u") - impl.template dispatchForColumns>(x, y, timezone_x, timezone_y, res->getData()); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Function {} does not support '{}' unit", getName(), unit); diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 48bf88cb14c..649242d0d86 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference index 168b733d702..1375ccb1542 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.reference @@ -4,4 +4,3 @@ 201707 20170721 20170721112233 -19691231235959 diff --git a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql index 1e35e99a802..71151690028 100644 --- a/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql +++ b/tests/queries/0_stateless/00479_date_and_datetime_to_number.sql @@ -4,4 +4,3 @@ SELECT toYYYYMMDDhhmmss(toDate('2017-07-21')); SELECT toYYYYMM(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDD(toDateTime('2017-07-21T11:22:33')); SELECT toYYYYMMDDhhmmss(toDateTime('2017-07-21T11:22:33')); -SELECT toYYYYMMDDhhmmss(toDateTime64('1969-12-31 23:59:59.900', 3)); diff --git a/tests/queries/0_stateless/02160_special_functions.reference b/tests/queries/0_stateless/02160_special_functions.reference index 5e7e3383d8d..3a1dcd88902 100644 --- a/tests/queries/0_stateless/02160_special_functions.reference +++ b/tests/queries/0_stateless/02160_special_functions.reference @@ -33,14 +33,4 @@ Hello 2021-01-01 1 1 -86400000 -172800000 -86461000 -86401299 -701 -701 -800 -60200201 -60 -10 1 diff --git a/tests/queries/0_stateless/02160_special_functions.sql b/tests/queries/0_stateless/02160_special_functions.sql index 64919536be3..6d18e7d0d25 100644 --- a/tests/queries/0_stateless/02160_special_functions.sql +++ b/tests/queries/0_stateless/02160_special_functions.sql @@ -41,16 +41,4 @@ SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR); SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01'); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-03'::Date32); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:01:01'::DateTime); -SELECT DATEDIFF(millisecond, '2021-01-01'::Date, '2021-01-02 00:00:01.299'::DateTime64); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299'::DateTime64, '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.299999'::DateTime64(6), '2021-01-02'::Date); -SELECT DATEDIFF(millisecond, '2021-01-01 23:59:59.2'::DateTime64(1), '2021-01-02'::Date); -SELECT DATEDIFF(microsecond, '2021-01-01 23:59:59.899999'::DateTime64(6), '2021-01-02 00:01:00.100200300'::DateTime64(9)); - -SELECT DATEDIFF(microsecond, '1969-12-31 23:59:59.999950'::DateTime64(6), '1970-01-01 00:00:00.000010'::DateTime64(6)); -SELECT DATEDIFF(second, '1969-12-31 23:59:59.123'::DateTime64(6), '1970-01-01 00:00:09.123'::DateTime64(6)); - SELECT EXISTS (SELECT 1); diff --git a/tests/queries/0_stateless/02477_age_datetime64.reference b/tests/queries/0_stateless/02477_age_datetime64.reference index c8c716e1e9a..3b4459dd26d 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.reference +++ b/tests/queries/0_stateless/02477_age_datetime64.reference @@ -111,109 +111,3 @@ SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), ma 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); 1 --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -5100200 -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); -5100200 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -5100 -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); -5099 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -4 -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -5 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -4 -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); -4 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -3 -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -2 -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -3 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -2 -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -2 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -1 -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); -1 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -16 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -15 -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); -15 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -5 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -4 -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); -4 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -8 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -7 -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); -7 --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -2349 -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -2 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -5 -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -4 diff --git a/tests/queries/0_stateless/02477_age_datetime64.sql b/tests/queries/0_stateless/02477_age_datetime64.sql index 889137395a3..1bed93991ca 100644 --- a/tests/queries/0_stateless/02477_age_datetime64.sql +++ b/tests/queries/0_stateless/02477_age_datetime64.sql @@ -75,68 +75,3 @@ SELECT age('second', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), SELECT age('second', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDateTime64('2015-08-18 00:00:10', 3, 'UTC'))); SELECT age('day', materialize(toDateTime64('2015-08-18 00:00:00', 0, 'UTC')), materialize(toDate('2015-08-19', 'UTC'))); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDateTime64('2015-08-19 00:00:00', 3, 'UTC'))); - --- DateTime64 vs DateTime64 with fractional part -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400005', 9, 'UTC')); -SELECT age('microsecond', toDateTime64('2015-08-18 20:30:36.100200005', 9, 'UTC'), toDateTime64('2015-08-18 20:30:41.200400004', 9, 'UTC')); - -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550299', 6, 'UTC')); -SELECT age('millisecond', toDateTime64('2015-08-18 20:30:36.450299', 6, 'UTC'), toDateTime64('2015-08-18 20:30:41.550298', 6, 'UTC')); - -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999003', 6, 'UTC'), toDateTime64('2023-03-01 19:18:41.999002', 6, 'UTC')); -SELECT age('second', toDateTime64('2023-03-01 19:18:36.999', 3, 'UTC'), toDateTime64('2023-03-01 19:18:41.001', 3, 'UTC')); - -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.300', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 20:35:36.100', 3, 'UTC')); -SELECT age('minute', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 20:35:36.200100', 6, 'UTC')); - -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:31:36.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:37.200', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.300', 3, 'UTC'), toDateTime64('2015-01-01 23:30:36.200', 3, 'UTC')); -SELECT age('hour', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-01 23:30:36.200100', 6, 'UTC')); - -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 19:30:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:28:36.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:35.200', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-04 20:30:36.199', 3, 'UTC')); -SELECT age('day', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-04 20:30:36.200100', 6, 'UTC')); - -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 19:30:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:29:36.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:35.200', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200', 3, 'UTC'), toDateTime64('2015-01-15 20:30:36.100', 3, 'UTC')); -SELECT age('week', toDateTime64('2015-01-01 20:30:36.200101', 6, 'UTC'), toDateTime64('2015-01-15 20:30:36.200100', 6, 'UTC')); - -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-01 20:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 19:30:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:29:36.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:35.200', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-05-02 20:30:36.100', 3, 'UTC')); -SELECT age('month', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-05-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-01 20:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 19:30:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:29:36.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:35.200', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200', 3, 'UTC'), toDateTime64('2016-04-02 20:30:36.100', 3, 'UTC')); -SELECT age('quarter', toDateTime64('2015-01-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2016-04-02 20:30:36.200100', 6, 'UTC')); - -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-01-02 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-01 20:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 19:30:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:29:36.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:35.200', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200', 3, 'UTC'), toDateTime64('2023-02-02 20:30:36.100', 3, 'UTC')); -SELECT age('year', toDateTime64('2015-02-02 20:30:36.200101', 6, 'UTC'), toDateTime64('2023-02-02 20:30:36.200100', 6, 'UTC')); - --- DateTime64 vs DateTime64 with negative time -SELECT age('millisecond', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.350', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:58.001', 3, 'UTC'), toDateTime64('1970-01-01 00:00:00.35', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.001', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); -SELECT age('second', toDateTime64('1969-12-31 23:59:50.003', 3, 'UTC'), toDateTime64('1969-12-31 23:59:55.002', 3, 'UTC')); \ No newline at end of file From cbe0a0e748cd0301292bb8079bd093e998d3031c Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:04:56 +0200 Subject: [PATCH 68/79] upd threshold due to dt64 range change --- src/IO/ReadHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index b8ce162ec91..2636898c1b3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1005,8 +1005,8 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re } } } - /// 9908870400 is time_t value for 2184-01-01 UTC (a bit over the last year supported by DateTime64) - else if (whole >= 9908870400LL) + /// 10413792000 is time_t value for 2300-01-01 UTC (a bit over the last year supported by DateTime64) + else if (whole >= 10413792000LL) { /// Unix timestamp with subsecond precision, already scaled to integer. /// For disambiguation we support only time since 2001-09-09 01:46:40 UTC and less than 30 000 years in future. From 761cc7afa584438824d76ed2ed9385a0b340fc04 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sat, 15 Jul 2023 03:17:18 +0200 Subject: [PATCH 69/79] add tests --- .../0_stateless/01802_toDateTime64_large_values.reference | 4 ++++ .../queries/0_stateless/01802_toDateTime64_large_values.sql | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference index e60b1c30314..f3810cc3d4b 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.reference +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.reference @@ -8,3 +8,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); 2205-12-12 12:12:12.000000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +2299-12-31 23:59:59.000 diff --git a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql index d82d4433b2d..5c2e65188c3 100644 --- a/tests/queries/0_stateless/01802_toDateTime64_large_values.sql +++ b/tests/queries/0_stateless/01802_toDateTime64_large_values.sql @@ -4,4 +4,7 @@ SELECT toDateTime64('2205-12-12 12:12:12', 0, 'UTC'); SELECT toDateTime64('2205-12-12 12:12:12', 0, 'Asia/Istanbul'); SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); -SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); \ No newline at end of file +SELECT toDateTime64('2205-12-12 12:12:12', 6, 'Asia/Istanbul'); + +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); +SELECT toDateTime64('2299-12-31 23:59:59', 3, 'UTC'); \ No newline at end of file From 7d3b28f9c398e4d224243004a6f5c5eddfc4cce2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 14 Jul 2023 21:22:16 +0200 Subject: [PATCH 70/79] Bump version of the distributed_ddl_entry_format_version to 5 by default This will enable the following features for distributed DDL queries: - opentelemetry support (#41484) - initial_query_id pass through (#50015) Signed-off-by: Azat Khuzhin --- src/Core/Settings.h | 2 +- ...distributed_ddl_output_mode_long.reference | 32 +++++++++---------- .../02761_ddl_initial_query_id.reference | 2 +- .../0_stateless/02761_ddl_initial_query_id.sh | 5 +-- 4 files changed, 21 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f1a314fed37..d54645bb0de 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -629,7 +629,7 @@ class IColumn; M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ - M(UInt64, distributed_ddl_entry_format_version, 3, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ + M(UInt64, distributed_ddl_entry_format_version, 5, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference index 4397810b68d..bc33c2fa913 100644 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.reference @@ -27,19 +27,19 @@ localhost 9000 57 Code: 57. Error: Table default.never_throw already exists. (TA localhost 9000 0 1 0 localhost 1 \N \N 1 0 distributed_ddl_queue -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 -2 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N -2 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.none ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.none already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.none ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.null_status ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.null_status already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.null_status ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 0 1 1 +5 localhost 9000 test_shard_localhost CREATE TABLE default.never_throw ON CLUSTER test_shard_localhost (`n` Int32) ENGINE = Memory 1 localhost 9000 Finished 57 Code: 57. DB::Error: Table default.never_throw already exists. (TABLE_ALREADY_EXISTS) 1 1 +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 1 Inactive \N \N \N \N +5 localhost 9000 test_unavailable_shard DROP TABLE IF EXISTS default.never_throw ON CLUSTER test_unavailable_shard 1 localhost 9000 Finished 0 1 1 diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference index 5c6f448eed5..aac7a0892e1 100644 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.reference +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.reference @@ -1,4 +1,4 @@ -default distributed_ddl_entry_format_version +distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION) DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost distributed_ddl_entry_format_version=PRESERVE_INITIAL_QUERY_ID_VERSION DROP TABLE IF EXISTS default.foo diff --git a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh index b8b35ef01f7..83890c3f335 100755 --- a/tests/queries/0_stateless/02761_ddl_initial_query_id.sh +++ b/tests/queries/0_stateless/02761_ddl_initial_query_id.sh @@ -4,9 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -echo "default distributed_ddl_entry_format_version" +echo "distributed_ddl_entry_format_version=OPENTELEMETRY_ENABLED_VERSION (older then PRESERVE_INITIAL_QUERY_ID_VERSION)" +OPENTELEMETRY_ENABLED_VERSION=4 query_id="$(random_str 10)" -$CLICKHOUSE_CLIENT --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" +$CLICKHOUSE_CLIENT --distributed_ddl_entry_format_version=$OPENTELEMETRY_ENABLED_VERSION --query_id "$query_id" --distributed_ddl_output_mode=none -q "DROP TABLE IF EXISTS foo ON CLUSTER test_shard_localhost" $CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" $CLICKHOUSE_CLIENT -q "SELECT query FROM system.query_log WHERE initial_query_id = '$query_id' AND type != 'QueryStart'" From 290c145acfc5accb2344b63356904af5f0a784ba Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 15 Jul 2023 14:53:46 +0000 Subject: [PATCH 71/79] Add array_agg as alias of groupArray for PostgreSQL compatibility --- .../aggregate-functions/reference/grouparray.md | 2 ++ src/AggregateFunctions/AggregateFunctionGroupArray.cpp | 1 + tests/queries/0_stateless/02813_array_agg.reference | 6 ++++++ tests/queries/0_stateless/02813_array_agg.sql | 10 ++++++++++ 4 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02813_array_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_agg.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md index 18048fa4f71..ad678443df6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/grouparray.md +++ b/docs/en/sql-reference/aggregate-functions/reference/grouparray.md @@ -44,3 +44,5 @@ Result: ``` The groupArray function will remove ᴺᵁᴸᴸ value based on the above results. + +- Alias: `array_agg`. diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..1c54038929b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/tests/queries/0_stateless/02813_array_agg.reference b/tests/queries/0_stateless/02813_array_agg.reference new file mode 100644 index 00000000000..202f6aa4bac --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.reference @@ -0,0 +1,6 @@ +['hello, world!','hello, world!','hello, world!','hello, world!','hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] +['hello, world!'] diff --git a/tests/queries/0_stateless/02813_array_agg.sql b/tests/queries/0_stateless/02813_array_agg.sql new file mode 100644 index 00000000000..91d8d0774d3 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_agg.sql @@ -0,0 +1,10 @@ +drop table if exists t; +create table t (n Int32, s String) engine=MergeTree order by n; + +insert into t select number, 'hello, world!' from numbers (5); + +select array_agg(s) from t; + +select aRray_Agg(s) from t group by n; + +drop table t; From 20b77e946a03605907320d13dbfacfb0f413f828 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:00:23 +0200 Subject: [PATCH 72/79] There is no point in detecting flaky tests --- tests/ci/clickhouse_helper.py | 24 ------------------- tests/ci/compatibility_check.py | 2 -- tests/ci/fast_test_check.py | 2 -- tests/ci/functional_test_check.py | 2 -- tests/ci/install_check.py | 2 -- tests/ci/integration_test_check.py | 2 -- tests/ci/report.py | 2 +- tests/ci/stress_check.py | 2 -- tests/ci/style_check.py | 2 -- tests/ci/unit_tests_check.py | 2 -- tests/integration/ci-runner.py | 38 ++++++++---------------------- 11 files changed, 11 insertions(+), 69 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index 64b64896f66..9410b37d69f 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -190,27 +190,3 @@ def prepare_tests_results_for_clickhouse( result.append(current_row) return result - - -def mark_flaky_tests( - clickhouse_helper: ClickHouseHelper, check_name: str, test_results: TestResults -) -> None: - try: - query = f"""SELECT DISTINCT test_name -FROM checks -WHERE - check_start_time BETWEEN now() - INTERVAL 3 DAY AND now() - AND check_name = '{check_name}' - AND (test_status = 'FAIL' OR test_status = 'FLAKY') - AND pull_request_number = 0 -""" - - tests_data = clickhouse_helper.select_json_each_row("default", query) - master_failed_tests = {row["test_name"] for row in tests_data} - logging.info("Found flaky tests: %s", ", ".join(master_failed_tests)) - - for test_result in test_results: - if test_result.status == "FAIL" and test_result.name in master_failed_tests: - test_result.status = "FLAKY" - except Exception as ex: - logging.error("Exception happened during flaky tests fetch %s", ex) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 04203617dca..97de7fed2d5 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -231,7 +230,6 @@ def main(): ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index d5198e5c3d7..460e17acd37 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -14,7 +14,6 @@ from github import Github from build_check import get_release_or_pr from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -190,7 +189,6 @@ def main(): state, description, test_results, additional_logs = process_results(output_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) s3_path_prefix = os.path.join( get_release_or_pr(pr_info, get_version_from_repo())[0], pr_info.sha, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 9279b19b187..b773d1eddd9 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -16,7 +16,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -368,7 +367,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index d619ce96cee..73e1a6ef739 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_builds_filter from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -345,7 +344,6 @@ def main(): return ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, args.check_name, test_results) description = format_description(description) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index e6b2203fb65..222b2197117 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -15,7 +15,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -276,7 +275,6 @@ def main(): state = override_status(state, check_name, invert=validate_bugfix_check) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) s3_helper = S3Helper() report_url = upload_results( diff --git a/tests/ci/report.py b/tests/ci/report.py index a9014acec12..8b301d08d56 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -349,7 +349,7 @@ def create_test_html_report( has_log_urls = True row = "" - has_error = test_result.status in ("FAIL", "FLAKY", "NOT_FAILED") + has_error = test_result.status in ("FAIL", "NOT_FAILED") if has_error and test_result.raw_logs is not None: row = '' row += "" + test_result.name + "" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index ac280916a2f..895eb318bc4 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -13,7 +13,6 @@ from github import Github from build_download_helper import download_all_deb_packages from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import RerunHelper, get_commit, post_commit_status @@ -168,7 +167,6 @@ def run_stress_test(docker_image_name): result_path, server_log_path, run_log_path ) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 33a5cd21f39..0871dd7ec6a 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -12,7 +12,6 @@ from typing import List, Tuple from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -189,7 +188,6 @@ def main(): state, description, test_results, additional_files = process_result(temp_path) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, NAME, test_results) report_url = upload_results( s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 5279ccde492..1c3ee303b27 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -12,7 +12,6 @@ from github import Github from build_download_helper import download_unit_tests from clickhouse_helper import ( ClickHouseHelper, - mark_flaky_tests, prepare_tests_results_for_clickhouse, ) from commit_status_helper import ( @@ -159,7 +158,6 @@ def main(): state, description, test_results, additional_logs = process_results(test_output) ch_helper = ClickHouseHelper() - mark_flaky_tests(ch_helper, check_name, test_results) report_url = upload_results( s3_helper, diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 00bca60e7db..190c79a5263 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -487,36 +487,23 @@ class ClickhouseIntegrationTestsRunner: def _update_counters(self, main_counters, current_counters, broken_tests): for test in current_counters["PASSED"]: - if ( - test not in main_counters["PASSED"] - and test not in main_counters["FLAKY"] - ): - is_flaky = False + if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) - is_flaky = True if test in main_counters["ERROR"]: main_counters["ERROR"].remove(test) - is_flaky = True if test in main_counters["BROKEN"]: main_counters["BROKEN"].remove(test) - is_flaky = True - if is_flaky: - main_counters["FLAKY"].append(test) + if test not in broken_tests: + main_counters["PASSED"].append(test) else: - if test not in broken_tests: - main_counters["PASSED"].append(test) - else: - main_counters["NOT_FAILED"].append(test) + main_counters["NOT_FAILED"].append(test) for state in ("ERROR", "FAILED"): for test in current_counters[state]: - if test in main_counters["FLAKY"]: - continue if test in main_counters["PASSED"]: main_counters["PASSED"].remove(test) - main_counters["FLAKY"].append(test) continue if test not in broken_tests: if test not in main_counters[state]: @@ -605,7 +592,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], } tests_times = defaultdict(float) for test in tests_in_group: @@ -627,7 +613,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -757,11 +742,11 @@ class ClickhouseIntegrationTestsRunner: ) log_paths.append(extras_result_path) - if len(counters["PASSED"]) + len(counters["FLAKY"]) == len(tests_in_group): + if len(counters["PASSED"]) == len(tests_in_group): logging.info("All tests from group %s passed", test_group) break if ( - len(counters["PASSED"]) + len(counters["FLAKY"]) >= 0 + len(counters["PASSED"]) >= 0 and len(counters["FAILED"]) == 0 and len(counters["ERROR"]) == 0 ): @@ -825,7 +810,7 @@ class ClickhouseIntegrationTestsRunner: result_state = "failure" if not should_fail: break - assert len(counters["FLAKY"]) == 0 or should_fail + assert should_fail logging.info("Try is OK, all tests passed, going to clear env") clear_ip_tables_and_restart_daemons() logging.info("And going to sleep for some time") @@ -835,7 +820,7 @@ class ClickhouseIntegrationTestsRunner: time.sleep(5) test_result = [] - for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): + for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): if state == "PASSED": text_state = "OK" elif state == "FAILED": @@ -928,7 +913,6 @@ class ClickhouseIntegrationTestsRunner: "PASSED": [], "FAILED": [], "SKIPPED": [], - "FLAKY": [], "BROKEN": [], "NOT_FAILED": [], } @@ -988,7 +972,6 @@ class ClickhouseIntegrationTestsRunner: "FAILED", "PASSED", "SKIPPED", - "FLAKY", "BROKEN", "NOT_FAILED", ): @@ -1004,15 +987,14 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}, flaky: {}".format( - failed_sum, len(counters["PASSED"]), len(counters["FLAKY"]) + status_text = "fail: {}, passed: {}".format( + failed_sum, len(counters["PASSED"]) ) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text result_state = "failure" - counters["FLAKY"] = [] if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" From 44fc93ba638e14054e2d17d8ba6e4e8fa578f268 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 15 Jul 2023 21:10:21 +0000 Subject: [PATCH 73/79] Automatic style fix --- tests/integration/ci-runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 190c79a5263..9629a5821b5 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -987,9 +987,7 @@ class ClickhouseIntegrationTestsRunner: ] failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) - status_text = "fail: {}, passed: {}".format( - failed_sum, len(counters["PASSED"]) - ) + status_text = "fail: {}, passed: {}".format(failed_sum, len(counters["PASSED"])) if self.soft_deadline_time < time.time(): status_text = "Timeout, " + status_text From 4f6a0c2faccdb0e6652a21467da164b12520846c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:11:22 +0200 Subject: [PATCH 74/79] Loosen the check --- tests/clickhouse-test | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 46ec19b041d..36e793f03e0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1213,17 +1213,20 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) - leftover_tables = clickhouse_execute( - args, - f"SHOW TABLES FROM {database}", - timeout=seconds_left, - settings={ - "log_comment": args.testcase_basename, - }, - ).decode().replace("\n", ", "); + # Check if the test does not cleanup its tables. + # Only for newly added tests. Please extend this check to the old tests as well. + if self.case_file >= '02800': + leftover_tables = clickhouse_execute( + args, + f"SHOW TABLES FROM {database}", + timeout=seconds_left, + settings={ + "log_comment": args.testcase_basename, + }, + ).decode().replace("\n", ", "); - if 0 != len(leftover_tables): - raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") + if 0 != len(leftover_tables): + raise Exception(f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally.") drop_database_query = f"DROP DATABASE IF EXISTS {database}" if args.replicated_database: From 095624fbd17ee7c4b1a68525c59a3d5c14ba4baf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 15 Jul 2023 23:33:11 +0200 Subject: [PATCH 75/79] Remove default argument value --- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageMergeTree.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 29880f10f28..074f01e7d03 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -373,7 +373,7 @@ void StorageMergeTree::alter( /// Always execute required mutations synchronously, because alters /// should be executed in sequential order. if (!maybe_mutation_commands.empty()) - waitForMutation(mutation_version); + waitForMutation(mutation_version, false); } { @@ -601,7 +601,7 @@ void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr quer Int64 version = startMutation(commands, query_context); if (query_context->getSettingsRef().mutations_sync > 0 || query_context->getCurrentTransaction()) - waitForMutation(version); + waitForMutation(version, false); } bool StorageMergeTree::hasLightweightDeletedMask() const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 8099f9c16aa..6aecde15117 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -191,7 +191,7 @@ private: /// and into in-memory structures. Wake up merge-mutation task. Int64 startMutation(const MutationCommands & commands, ContextPtr query_context); /// Wait until mutation with version will finish mutation for all parts - void waitForMutation(Int64 version, bool wait_for_another_mutation = false); + void waitForMutation(Int64 version, bool wait_for_another_mutation); void waitForMutation(const String & mutation_id, bool wait_for_another_mutation) override; void waitForMutation(Int64 version, const String & mutation_id, bool wait_for_another_mutation = false); void setMutationCSN(const String & mutation_id, CSN csn) override; From 66b66db39d8d511fe3df137a14586072bf538481 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 02:52:06 +0200 Subject: [PATCH 76/79] Fix ORDER BY tuple of WINDOW functions --- src/Interpreters/ExpressionAnalyzer.cpp | 11 +++++----- src/Interpreters/GetAggregatesVisitor.cpp | 20 +++++++++++++------ src/Interpreters/GetAggregatesVisitor.h | 2 +- src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/TreeRewriter.h | 5 ++--- ...4_order_by_tuple_window_function.reference | 1 + .../02814_order_by_tuple_window_function.sql | 1 + 7 files changed, 25 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.reference create mode 100644 tests/queries/0_stateless/02814_order_by_tuple_window_function.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f27d23e8e94..9a450fabd5b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1378,10 +1378,9 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool /* only_types */) { ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window); + for (const auto & expression : syntax->expressions_with_window_function) - { getRootActionsForWindowFunctions(expression->clone(), true, step.actions()); - } } void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) @@ -1760,9 +1759,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// second_stage: Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing. /** First we compose a chain of actions and remember the necessary steps from it. - * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and - * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. - */ + * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and + * throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries. + */ const ASTSelectQuery & query = *query_analyzer.getSelectQuery(); auto context = query_analyzer.getContext(); @@ -1805,7 +1804,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && (query.sampleSize() || settings.parallel_replicas_count > 1)) { - // we evaluate sampling for Merge lazily so we need to get all the columns + // we evaluate sampling for Merge lazily, so we need to get all the columns if (storage->getName() == "Merge") { const auto columns = metadata_snapshot->getColumns().getAll(); diff --git a/src/Interpreters/GetAggregatesVisitor.cpp b/src/Interpreters/GetAggregatesVisitor.cpp index dd958693d89..718721308b1 100644 --- a/src/Interpreters/GetAggregatesVisitor.cpp +++ b/src/Interpreters/GetAggregatesVisitor.cpp @@ -1,4 +1,6 @@ #include +#include + namespace DB { @@ -13,7 +15,7 @@ struct WindowExpressionsCollectorChildInfo bool window_function_in_subtree = false; }; -// This visitor travers AST and collects the list of expressions which depend on +// This visitor traverses the AST and collects the list of expressions which depend on // evaluation of window functions. Expression is collected only if // it's not a part of another expression. // @@ -26,15 +28,18 @@ struct WindowExpressionsCollectorMatcher { if (child->as() || child->as()) return false; + if (auto * select = node->as()) { - // We don't analysis WITH statement because it might contain useless aggregates + // We don't analyse the WITH statement because it might contain useless aggregates if (child == select->with()) return false; } - // We procces every expression manually + + // We process every expression manually if (auto * func = node->as()) return false; + return true; } @@ -50,6 +55,8 @@ struct WindowExpressionsCollectorMatcher ASTPtr & ast, const ASTPtr & parent) { + checkStackSize(); + if (auto * func = ast->as()) { if (func->is_window_function) @@ -67,7 +74,7 @@ struct WindowExpressionsCollectorMatcher { func->compute_after_window_functions = true; if ((!parent || !parent->as())) - expressions_with_window_functions.push_back(func); + expressions_with_window_functions.push_back(ast); } return result; @@ -75,15 +82,16 @@ struct WindowExpressionsCollectorMatcher return {}; } - std::vector expressions_with_window_functions {}; + ASTs expressions_with_window_functions; }; using WindowExpressionsCollectorVisitor = InDepthNodeVisitorWithChildInfo; -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast) +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast) { WindowExpressionsCollectorVisitor visitor; visitor.visit(ast); + return std::move(visitor.expressions_with_window_functions); } diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index 3f5804c39a0..fdf54de3e57 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -114,6 +114,6 @@ inline void assertNoAggregates(const ASTPtr & ast, const char * description) GetAggregatesVisitor(data).visit(ast); } -std::vector getExpressionsWithWindowFunctions(ASTPtr & ast); +ASTs getExpressionsWithWindowFunctions(ASTPtr & ast); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index cda5ceeb164..65b5d950975 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1287,6 +1287,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( bool is_changed = replaceAliasColumnsInQuery(query, result.storage_snapshot->metadata->getColumns(), result.array_join_result_to_source, getContext(), excluded_nodes); + /// If query is changed, we need to redo some work to correct name resolution. if (is_changed) { diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index b94043b8983..ea16c432d0f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -40,11 +40,10 @@ struct TreeRewriterResult NameSet expanded_aliases; Aliases aliases; + std::vector aggregates; - std::vector window_function_asts; - - std::vector expressions_with_window_function; + ASTs expressions_with_window_function; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. /// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v". diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql new file mode 100644 index 00000000000..8ba54fc11d0 --- /dev/null +++ b/tests/queries/0_stateless/02814_order_by_tuple_window_function.sql @@ -0,0 +1 @@ +SELECT 1 ORDER BY tuple(count() OVER ()); From 20a671b8cf2830733eea2ed7e7c6cd80d70ed81a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 15 Jul 2023 19:44:29 +0200 Subject: [PATCH 77/79] Skip protection from double decompression if inode from maps cannot be obtained Under some circumstances, like using qemu-$ARCH-static, /proc/self/maps will not contain information about /proc/self/exe. Well, strictly speaking it does contains, however qemu will not pass it to the user program:
strace $ sudo strace -s10000 -f arch-chroot . /qemu-riscv64-static /clickhouse ... execve("/qemu-riscv64-static", ["/qemu-riscv64-static", "/clickhouse"], 0x7fffffffe458 /* 20 vars */) = 0 readlinkat(AT_FDCWD, "/proc/self/exe", "/qemu-riscv64-static", 4096) = 20 openat(AT_FDCWD, "/proc/self/maps", O_RDONLY|O_CLOEXEC) = 4 [pid 3126] read(4, "00010000-00111000 r--p 00000000 fe:01 30312571 /clickhouse\n00111000-00119000 r--p 00100000 fe:01 30312571 /clickhouse\n00119000-0011a000 rw-p 00108000 fe:01 30312571 /clickhouse\n0011a000-0013d000 rw-p 00000000 00:00 0 \n4000000000-4000001000 ---p 00000000 00:00 0 \n4000001000-4000801000 rw-p 00000000 00:00 0 \n4000801000-400081a000 r--p 00000000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081a000-400081b000 ---p 00000000 00:00 0 \n400081b000-400081c000 r--p 00019000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081c000-400081e000 rw-p 0001a000 fe:01 30316932 /lib/riscv64-linux-gnu/ld-2.32.so\n400081e000-400081f000 r--p 00000000 00:00 0 \n400081f000-4000922000 r--p 00000000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000922000-4000926000 r--p 00102000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000926000-4000928000 rw-p 00106000 fe:01 30316935 /lib/riscv64-linux-gnu/libc-2.32.so\n4000928000-400092d000 rw-p 00000000 00:00 0 \n400092d000-40009af000 r--p 00000000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009af000-40009b0000 r--p 00081000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b0000-40009b1000 rw-p 00082000 fe:01 30316943 /lib/riscv64-linux-gnu/libm-2.32.so\n40009b1000-40009c5000 r--p 00000000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c5000-40009c6000 r--p 00013000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c6000-40009c7000 rw-p 00014000 fe:01 30316946 /lib/riscv64-linux-gnu/libpthread-2.32.so\n40009c7000-40009cb000 rw-p 00000000 00:00 0 \n40009cb000-40009cd000 r--p 00000000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cd000-40009ce000 r--p 00001000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009ce000-40009cf000 rw-p 00002000 fe:01 30316939 /lib/riscv64-linux-gnu/libdl-2.32.so\n40009cf000-40009d1000 rw-p 00000000 00:00 0 \n7fffe8000000-7fffeffff000 rwxp 00000000 00:00 0 \n7fffeffff000-7ffff0000000 ---p 00000000 00:00 0 \n7ffff0000000-7ffff0021000 rw-p 00000000 00:00 0 \n7ffff0021000-7ffff4000000 ---p 00000000 00:00 0 \n7ffff6b4b000-7ffff6b5b000 rw-p 00000000 00:00 0 \n7ffff71ff000-7ffff7200000 ---p 00000000 00:00 0 \n7ffff7200000-7ffff7a00000 rw-p 00000000 00:00 0\n7ffff7a00000-7ffff7a3c000 r--p 00000000 fe:01 30316953 /qemu-riscv64-static\n7ffff7a3c000-7ffff7c74000 r-xp 0003c000 fe:01 30316953 /qemu-riscv64-static\n7ffff7c74000-7ffff7d77000 r--p 00274000 fe:01 30316953 /qemu-riscv64-static\n7ffff7d77000-7ffff7dce000 r--p 00377000 fe:01 30316953 /qemu-riscv64-static\n7ffff7dce000-7ffff7df7000 rw-p 003ce000 fe:01 30316953 /qemu-riscv64-static\n7ffff7df7000-7ffff7e0c000 rw-p 00000000 00:00 0 [heap]\n7ffff7e0c000-7ffff7e70000 rw-p 00000000 00:00 0 [heap]\n7ffff7f42000-7ffff7ff9000 rw-p 00000000 00:00 0 \n7ffff7ff9000-7ffff7ffd000 r--p 00000000 00:00 0 [vvar]\n7ffff7ffd000-7ffff7fff000 r-xp 00000000 00:00 0 [vdso]\n7ffffffde000-7ffffffff000 rw-p 00000000 00:00 0 [stack]\nffffffffff600000-ffffffffff601000 --xp 00000000 00:00 0 [vsyscall]\n", 4096) = 3608 [pid 3126] read(4, "", 1024) = 0 [pid 3126] close(4) = 0 [pid 3126] write(3, "10000-111000 r-xp 00000000 fe:01 30312571", 41) = 41 [pid 3126] write(3, " /clickhouse\n", 44) = 44 [pid 3126] write(3, "111000-119000 r--p 00100000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "119000-11a000 rw-p 00108000 fe:01 30312571", 42) = 42 [pid 3126] write(3, " /clickhouse\n", 43) = 43 [pid 3126] write(3, "11a000-13d000 rw-p 00000000 00:00 0", 35) = 35 [pid 3126] write(3, " \n", 39) = 39 [pid 3126] write(3, "4000000000-4000001000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "4000001000-4000801000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " [stack]\n", 38) = 38 [pid 3126] write(3, "4000801000-400081a000 r-xp 00000000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57 [pid 3127] <... clock_nanosleep resumed>0x7ffff79ff060) = 0 [pid 3126] <... write resumed>) = 57 [pid 3127] clock_nanosleep(CLOCK_REALTIME, 0, {tv_sec=0, tv_nsec=10000000}, [pid 3126] write(3, "400081a000-400081b000 ---p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081b000-400081c000 r--p 00019000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081c000-400081e000 rw-p 0001a000 fe:01 30316932", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/ld-2.32.so\n", 57) = 57 [pid 3126] write(3, "400081e000-400081f000 r-xp 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400081f000-4000922000 r-xp 00000000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000922000-4000926000 r--p 00102000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000926000-4000928000 rw-p 00106000 fe:01 30316935", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libc-2.32.so\n", 59) = 59 [pid 3126] write(3, "4000928000-400092d000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "400092d000-40009af000 r-xp 00000000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009af000-40009b0000 r--p 00081000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b0000-40009b1000 rw-p 00082000 fe:01 30316943", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libm-2.32.so\n", 59) = 59 [pid 3126] write(3, "40009b1000-40009c5000 r-xp 00000000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c5000-40009c6000 r--p 00013000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c6000-40009c7000 rw-p 00014000 fe:01 30316946", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libpthread-2.32.so\n", 65) = 65 [pid 3126] write(3, "40009c7000-40009cb000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31 [pid 3126] write(3, "40009cb000-40009cd000 r-xp 00000000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cd000-40009ce000 r--p 00001000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009ce000-40009cf000 rw-p 00002000 fe:01 30316939", 50) = 50 [pid 3126] write(3, " /lib/riscv64-linux-gnu/libdl-2.32.so\n", 60) = 60 [pid 3126] write(3, "40009cf000-40009d1000 rw-p 00000000 00:00 0", 43) = 43 [pid 3126] write(3, " \n", 31) = 31
Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 ++++++++++--------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..4a4985120fd 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,55 +430,58 @@ int main(int/* argc*/, char* argv[]) return 1; } + int lock = -1; + /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - if (inode == 0) + /// In some cases /proc/self/maps may not contain the inode for the + /// /proc/self/exe, one of such examples are using qemu-*-static, in this + /// case maps will be proxied through the qemu, and it will remove + /// information about itself from it. + if (inode != 0) { - std::cerr << "Unable to obtain inode." << std::endl; - return 1; - } - - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) - { - perror("lock open"); - return 1; - } - - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); - - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) { - perror("fstat lock"); + perror("lock open"); return 1; } - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); - printf("No target executable - decompression only was performed.\n"); - return 0; + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) + { + perror("fstat lock"); + return 1; + } + + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); + + printf("No target executable - decompression only was performed.\n"); + return 0; + } } #endif @@ -546,21 +549,19 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - write(lock, "1", 1); -#endif + if (lock >= 0) + write(lock, "1", 1); execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } -#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - ftruncate(lock, 0); -#endif + if (lock >= 0) + ftruncate(lock, 0); printf("No target executable - decompression only was performed.\n"); } From 71d6206fc45e8622230ed2396f2662d76851eb37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 17:42:22 +0300 Subject: [PATCH 78/79] Fix Python --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 790adf9df62..abd109d00b2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1229,7 +1229,7 @@ class TestCase: .replace("\n", ", ") ) - if 0 != len(leftover_tables): + if len(leftover_tables) != 0: raise Exception( f"The test should cleanup its tables ({leftover_tables}), otherwise it is inconvenient for running it locally." ) From 59ecdbd53ec9f8abaf841d4cb0410e93d03bb32e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 22:30:04 +0200 Subject: [PATCH 79/79] Fix test 02497_storage_file_reader_selection --- .../0_stateless/02497_storage_file_reader_selection.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh index 8ea3adcbd2f..aa43e81f131 100755 --- a/tests/queries/0_stateless/02497_storage_file_reader_selection.sh +++ b/tests/queries/0_stateless/02497_storage_file_reader_selection.sh @@ -7,11 +7,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=test_02497_$CLICKHOUSE_TEST_UNIQUE_NAME.tsv echo -e 'key\nfoo\nbar' > $DATA_FILE -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap:" -$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap:" && echo 1 || echo 'Fail' +$CLICKHOUSE_LOCAL --storage_file_read_method=mmap --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 'Fail' || echo 0 -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferMMap" -$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -c "CreatedReadBufferOrdinary" +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferMMap" && echo 'Fail' || echo 0 +$CLICKHOUSE_LOCAL --storage_file_read_method=pread --print-profile-events -q "SELECT * FROM file($DATA_FILE) FORMAT Null" 2>&1 | grep -F -q "CreatedReadBufferOrdinary" && echo 1 || echo 'Fail' $CLICKHOUSE_CLIENT --storage_file_read_method=mmap -nq "SELECT * FROM file('/dev/null', 'LineAsString') FORMAT Null -- { serverError BAD_ARGUMENTS }"