From a34f42ca22c8a4820e4cbcf67cdd48a3589e3879 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 13 Jan 2024 18:48:47 +0300 Subject: [PATCH 001/117] Remove lock from the ReadProgressCallback It looks redundant (added in 5ef51ed), though it has "fix tests" in the log message, but CI reports is not available for the commits from that PR [1], so let's try. [1]: https://github.com/ClickHouse/ClickHouse/pull/37543 Also this can be a big problem, since the code under that lock (throttling or quotas with previous implementation that uses boost::atomic_shared_ptr) may sleep. Some numbers: run | time ------------------------|------ max_threads=100 before | 23.1 max_threads=100 after | 15.1 max_threads=4500 before | 4.5 max_threads=4500 after | 2.3 Query: select sum(number) from numbers_mt(2000000) settings max_threads=X, max_block_size = 1 Signed-off-by: Azat Khuzhin --- src/QueryPipeline/ReadProgressCallback.cpp | 2 -- src/QueryPipeline/ReadProgressCallback.h | 1 - tests/performance/small_block_contention.xml | 3 +++ 3 files changed, 3 insertions(+), 3 deletions(-) create mode 100644 tests/performance/small_block_contention.xml diff --git a/src/QueryPipeline/ReadProgressCallback.cpp b/src/QueryPipeline/ReadProgressCallback.cpp index 59843d8791d..e90fc24d882 100644 --- a/src/QueryPipeline/ReadProgressCallback.cpp +++ b/src/QueryPipeline/ReadProgressCallback.cpp @@ -126,8 +126,6 @@ bool ReadProgressCallback::onProgress(uint64_t read_rows, uint64_t read_bytes, c CurrentThread::updatePerformanceCountersIfNeeded(); - std::lock_guard lock(limits_and_quotas_mutex); - /// TODO: Should be done in PipelineExecutor. for (const auto & limits : storage_limits) limits.local_limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_stopwatch.elapsedMicroseconds(), limits.local_limits.timeout_overflow_mode); diff --git a/src/QueryPipeline/ReadProgressCallback.h b/src/QueryPipeline/ReadProgressCallback.h index 5dbf3344bdf..7dfed9df5da 100644 --- a/src/QueryPipeline/ReadProgressCallback.h +++ b/src/QueryPipeline/ReadProgressCallback.h @@ -41,7 +41,6 @@ private: /// The total number of bytes to read. For progress bar. std::atomic_size_t total_bytes = 0; - std::mutex limits_and_quotas_mutex; Stopwatch total_stopwatch{CLOCK_MONOTONIC_COARSE}; /// Including waiting time bool update_profile_events = true; diff --git a/tests/performance/small_block_contention.xml b/tests/performance/small_block_contention.xml new file mode 100644 index 00000000000..ce1995a0a29 --- /dev/null +++ b/tests/performance/small_block_contention.xml @@ -0,0 +1,3 @@ + + select sum(number) from numbers_mt(200000) settings max_threads=100, max_block_size = 1 format Null + From 277e8d965555b4fcd09a755282666bcae36adae6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Feb 2024 14:03:53 +0800 Subject: [PATCH 002/117] Fix usage plain metadata type with new configuration option --- src/Disks/DiskType.cpp | 48 +++++++++++++++++++ src/Disks/DiskType.h | 34 +------------ src/Disks/ObjectStorages/IObjectStorage.h | 1 + .../ObjectStorages/MetadataStorageFactory.cpp | 36 +++++++++++--- .../ObjectStorages/MetadataStorageFactory.h | 7 +++ .../ObjectStorages/ObjectStorageFactory.cpp | 43 +++++++++++++---- src/Disks/ObjectStorages/PlainObjectStorage.h | 29 +++++++++++ .../RegisterDiskObjectStorage.cpp | 24 ++-------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 21 -------- .../configs/disk_s3.xml | 7 +++ .../test_attach_backup_from_s3_plain/test.py | 25 ++++++---- 11 files changed, 178 insertions(+), 97 deletions(-) create mode 100644 src/Disks/ObjectStorages/PlainObjectStorage.h diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index 218b6ee7f26..1778ae8025b 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -1,7 +1,27 @@ #include "DiskType.h" +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + +MetadataStorageType metadataTypeFromString(const String & type) +{ + auto check_type = Poco::toLower(type); + if (check_type == "local") + return MetadataStorageType::Local; + if (check_type == "plain") + return MetadataStorageType::Plain; + if (check_type == "web") + return MetadataStorageType::StaticWeb; + + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "MetadataStorageFactory: unknown metadata storage type: {}", type); +} bool DataSourceDescription::operator==(const DataSourceDescription & other) const { @@ -14,4 +34,32 @@ bool DataSourceDescription::sameKind(const DataSourceDescription & other) const == std::tie(other.type, other.object_storage_type, other.description); } +std::string DataSourceDescription::toString() const +{ + switch (type) + { + case DataSourceType::Local: + return "local"; + case DataSourceType::RAM: + return "memory"; + case DataSourceType::ObjectStorage: + { + switch (object_storage_type) + { + case ObjectStorageType::S3: + return "s3"; + case ObjectStorageType::HDFS: + return "hdfs"; + case ObjectStorageType::Azure: + return "azure_blob_storage"; + case ObjectStorageType::Local: + return "local_blob_storage"; + case ObjectStorageType::Web: + return "web"; + case ObjectStorageType::None: + return "none"; + } + } + } +} } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 15940ea9155..36fe4d83004 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -17,7 +17,6 @@ enum class ObjectStorageType { None, S3, - S3_Plain, Azure, HDFS, Web, @@ -30,9 +29,9 @@ enum class MetadataStorageType Local, Plain, StaticWeb, - Memory, }; +MetadataStorageType metadataTypeFromString(const String & type); String toString(DataSourceType data_source_type); struct DataSourceDescription @@ -49,36 +48,7 @@ struct DataSourceDescription bool operator==(const DataSourceDescription & other) const; bool sameKind(const DataSourceDescription & other) const; - std::string toString() const - { - switch (type) - { - case DataSourceType::Local: - return "local"; - case DataSourceType::RAM: - return "memory"; - case DataSourceType::ObjectStorage: - { - switch (object_storage_type) - { - case ObjectStorageType::S3: - return "s3"; - case ObjectStorageType::S3_Plain: - return "s3_plain"; - case ObjectStorageType::HDFS: - return "hdfs"; - case ObjectStorageType::Azure: - return "azure_blob_storage"; - case ObjectStorageType::Local: - return "local_blob_storage"; - case ObjectStorageType::Web: - return "web"; - case ObjectStorageType::None: - return "none"; - } - } - } - } + std::string toString() const; }; } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 56c269a3fc5..fde97d82ad1 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -218,6 +218,7 @@ public: virtual bool isReadOnly() const { return false; } virtual bool isWriteOnce() const { return false; } + virtual bool isPlain() const { return false; } virtual bool supportParallelWrite() const { return false; } diff --git a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp index 52a0b9ec268..adc1f84372c 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp @@ -32,6 +32,35 @@ void MetadataStorageFactory::registerMetadataStorageType(const std::string & met } } +std::string MetadataStorageFactory::getCompatibilityMetadataTypeHint(const ObjectStorageType & type) +{ + switch (type) + { + case ObjectStorageType::S3: + case ObjectStorageType::HDFS: + case ObjectStorageType::Local: + case ObjectStorageType::Azure: + return "local"; + case ObjectStorageType::Web: + return "web"; + default: + return ""; + } +} + +std::string MetadataStorageFactory::getMetadataType( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const std::string & compatibility_type_hint) +{ + if (compatibility_type_hint.empty() && !config.has(config_prefix + ".metadata_type")) + { + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Expected `metadata_type` in config"); + } + + return config.getString(config_prefix + ".metadata_type", compatibility_type_hint); +} + MetadataStoragePtr MetadataStorageFactory::create( const std::string & name, const Poco::Util::AbstractConfiguration & config, @@ -39,12 +68,7 @@ MetadataStoragePtr MetadataStorageFactory::create( ObjectStoragePtr object_storage, const std::string & compatibility_type_hint) const { - if (compatibility_type_hint.empty() && !config.has(config_prefix + ".metadata_type")) - { - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Expected `metadata_type` in config"); - } - - const auto type = config.getString(config_prefix + ".metadata_type", compatibility_type_hint); + const auto type = getMetadataType(config, config_prefix, compatibility_type_hint); const auto it = registry.find(type); if (it == registry.end()) diff --git a/src/Disks/ObjectStorages/MetadataStorageFactory.h b/src/Disks/ObjectStorages/MetadataStorageFactory.h index 5f61125c599..467cd3cef98 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFactory.h +++ b/src/Disks/ObjectStorages/MetadataStorageFactory.h @@ -25,6 +25,13 @@ public: ObjectStoragePtr object_storage, const std::string & compatibility_type_hint) const; + static std::string getMetadataType( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const std::string & compatibility_type_hint = ""); + + static std::string getCompatibilityMetadataTypeHint(const ObjectStorageType & type); + private: using Registry = std::unordered_map; Registry registry; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index b3626135177..6f6ff199902 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -16,8 +16,10 @@ #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include #include +#include #include #endif +#include #include #include @@ -32,6 +34,28 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +namespace +{ + template + ObjectStoragePtr createObjectStorage( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + Args && ...args) + { + auto compatibility_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(ObjectStorageType::S3); + auto metadata_type = MetadataStorageFactory::getMetadataType(config, config_prefix, compatibility_hint); + + if (metadataTypeFromString(metadata_type) == MetadataStorageType::Plain) + { + return std::make_shared>(std::forward(args)...); + } + else + { + return std::make_shared(std::forward(args)...); + } + } +} + ObjectStorageFactory & ObjectStorageFactory::instance() { static ObjectStorageFactory factory; @@ -129,12 +153,12 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto client = getClient(config, config_prefix, context, *settings); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); - auto object_storage = std::make_shared( - std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); + auto object_storage = createObjectStorage( + config, config_prefix, std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + checkS3Capabilities(*dynamic_cast(object_storage.get()), s3_capabilities, name, uri.key); return object_storage; }); @@ -165,12 +189,12 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto client = getClient(config, config_prefix, context, *settings); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); - auto object_storage = std::make_shared( + auto object_storage = std::make_shared>( std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name, uri.key); + checkS3Capabilities(*dynamic_cast(object_storage.get()), s3_capabilities, name, uri.key); return object_storage; }); @@ -198,7 +222,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) context->getSettingsRef().hdfs_replication ); - return std::make_unique(uri, std::move(settings), config); + return createObjectStorage(config, config_prefix, uri, std::move(settings), config); }); } #endif @@ -214,7 +238,8 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) bool /* skip_access_check */) -> ObjectStoragePtr { String container_name = config.getString(config_prefix + ".container_name", "default-container"); - return std::make_unique( + return createObjectStorage( + config, config_prefix, name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), @@ -248,7 +273,7 @@ void registerWebObjectStorage(ObjectStorageFactory & factory) ErrorCodes::BAD_ARGUMENTS, "Bad URI: `{}`. Error: {}", uri, e.what()); } - return std::make_shared(uri, context); + return createObjectStorage(config, config_prefix, uri, context); }); } @@ -266,7 +291,7 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory) loadDiskLocalConfig(name, config, config_prefix, context, object_key_prefix, keep_free_space_bytes); /// keys are mapped to the fs, object_key_prefix is a directory also fs::create_directories(object_key_prefix); - return std::make_shared(object_key_prefix); + return createObjectStorage(config, config_prefix, object_key_prefix); }); } #endif diff --git a/src/Disks/ObjectStorages/PlainObjectStorage.h b/src/Disks/ObjectStorages/PlainObjectStorage.h new file mode 100644 index 00000000000..3a81b85c44b --- /dev/null +++ b/src/Disks/ObjectStorages/PlainObjectStorage.h @@ -0,0 +1,29 @@ +#pragma once +#include + +namespace DB +{ + +/// Do not encode keys, store as-is, and do not require separate disk for metadata. +/// But because of this does not support renames/hardlinks/attrs/... +/// +/// NOTE: This disk has excessive API calls. +template +class PlainObjectStorage : public BaseObjectStorage +{ +public: + template + explicit PlainObjectStorage(Args && ...args) + : BaseObjectStorage(std::forward(args)...) {} + + std::string getName() const override { return "" + BaseObjectStorage::getName(); } + + /// Notes: + /// - supports BACKUP to this disk + /// - does not support INSERT into MergeTree table on this disk + bool isWriteOnce() const override { return true; } + + bool isPlain() const override { return true; } +}; + +} diff --git a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp index 383a0b079b5..669a0102951 100644 --- a/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/RegisterDiskObjectStorage.cpp @@ -10,25 +10,6 @@ namespace DB void registerObjectStorages(); void registerMetadataStorages(); -static std::string getCompatibilityMetadataTypeHint(const ObjectStorageType & type) -{ - switch (type) - { - case ObjectStorageType::S3: - case ObjectStorageType::HDFS: - case ObjectStorageType::Local: - case ObjectStorageType::Azure: - return "local"; - case ObjectStorageType::S3_Plain: - return "plain"; - case ObjectStorageType::Web: - return "web"; - case ObjectStorageType::None: - return ""; - } - UNREACHABLE(); -} - void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check) { registerObjectStorages(); @@ -47,7 +28,10 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch std::string compatibility_metadata_type_hint; if (!config.has(config_prefix + ".metadata_type")) { - compatibility_metadata_type_hint = getCompatibilityMetadataTypeHint(object_storage->getType()); + if (object_storage->isPlain()) + compatibility_metadata_type_hint = "plain"; + else + compatibility_metadata_type_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(object_storage->getType()); } auto metadata_storage = MetadataStorageFactory::instance().create( diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index ab0fa5bed68..4ece98c5ec4 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -182,27 +182,6 @@ private: LoggerPtr log; }; -/// Do not encode keys, store as-is, and do not require separate disk for metadata. -/// But because of this does not support renames/hardlinks/attrs/... -/// -/// NOTE: This disk has excessive API calls. -class S3PlainObjectStorage : public S3ObjectStorage -{ -public: - std::string getName() const override { return "S3PlainObjectStorage"; } - - template - explicit S3PlainObjectStorage(Args && ...args) - : S3ObjectStorage("S3PlainObjectStorage", std::forward(args)...) {} - - ObjectStorageType getType() const override { return ObjectStorageType::S3_Plain; } - - /// Notes: - /// - supports BACKUP to this disk - /// - does not support INSERT into MergeTree table on this disk - bool isWriteOnce() const override { return true; } -}; - } #endif diff --git a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml index 779e4b6ae21..3166eea7ccb 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml +++ b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml @@ -8,9 +8,16 @@ minio minio123 + + object_storage + local + plain + local_plain/ + backup_disk_s3_plain + backup_disk_local_plain diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index e575c487b7a..4a8da1e6d66 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -20,17 +20,27 @@ def start_cluster(): finally: cluster.shutdown() +s3_disk_def = """disk(type=s3_plain, + endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/', + access_key_id='minio', + secret_access_key='minio123');""" + +local_disk_def = "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain'" @pytest.mark.parametrize( - "table_name,backup_name,storage_policy,min_bytes_for_wide_part", + "table_name,backup_name,storage_policy,disk_def,min_bytes_for_wide_part", [ pytest.param( - "compact", "backup_compact", "s3_backup_compact", int(1e9), id="compact" + "compact", "backup_compact_s3", "backup_disk_s3_plain", s3_disk_def, int(1e9), id="compact" ), - pytest.param("wide", "backup_wide", "s3_backup_wide", int(0), id="wide"), + pytest.param("wide", "backup_wide_s3", "backup_disk_s3_plain", s3_disk_def, int(0), id="wide"), + pytest.param( + "compact", "backup_compact_local", "backup_disk_local_plain", local_disk_def, int(1e9), id="compact" + ), + pytest.param("wide", "backup_wide_local", "backup_disk_local_plain", local_disk_def, int(0), id="wide"), ], ) -def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide_part): +def test_attach_part(table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part): node.query( f""" -- Catch any errors (NOTE: warnings are ok) @@ -45,7 +55,7 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide settings min_bytes_for_wide_part={min_bytes_for_wide_part} as select number%5 part, number key from numbers(100); - backup table ordinary_db.{table_name} TO Disk('backup_disk_s3_plain', '{backup_name}') settings deduplicate_files=0; + backup table ordinary_db.{table_name} TO Disk('{storage_policy}', '{backup_name}') settings deduplicate_files=0; drop table ordinary_db.{table_name}; attach table ordinary_db.{table_name} (part UInt8, key UInt64) @@ -53,10 +63,7 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide order by key partition by part settings max_suspicious_broken_parts=0, - disk=disk(type=s3_plain, - endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/', - access_key_id='minio', - secret_access_key='minio123'); + disk={disk_def} """ ) From 69b5bd02a915ae044b4116de759d11ae80525dc5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 26 Feb 2024 09:37:17 +0000 Subject: [PATCH 003/117] Automatic style fix --- .../test_attach_backup_from_s3_plain/test.py | 42 ++++++++++++++++--- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index 4a8da1e6d66..900366b2c9c 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -20,27 +20,57 @@ def start_cluster(): finally: cluster.shutdown() + s3_disk_def = """disk(type=s3_plain, endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/', access_key_id='minio', secret_access_key='minio123');""" -local_disk_def = "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain'" +local_disk_def = ( + "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain'" +) + @pytest.mark.parametrize( "table_name,backup_name,storage_policy,disk_def,min_bytes_for_wide_part", [ pytest.param( - "compact", "backup_compact_s3", "backup_disk_s3_plain", s3_disk_def, int(1e9), id="compact" + "compact", + "backup_compact_s3", + "backup_disk_s3_plain", + s3_disk_def, + int(1e9), + id="compact", ), - pytest.param("wide", "backup_wide_s3", "backup_disk_s3_plain", s3_disk_def, int(0), id="wide"), pytest.param( - "compact", "backup_compact_local", "backup_disk_local_plain", local_disk_def, int(1e9), id="compact" + "wide", + "backup_wide_s3", + "backup_disk_s3_plain", + s3_disk_def, + int(0), + id="wide", + ), + pytest.param( + "compact", + "backup_compact_local", + "backup_disk_local_plain", + local_disk_def, + int(1e9), + id="compact", + ), + pytest.param( + "wide", + "backup_wide_local", + "backup_disk_local_plain", + local_disk_def, + int(0), + id="wide", ), - pytest.param("wide", "backup_wide_local", "backup_disk_local_plain", local_disk_def, int(0), id="wide"), ], ) -def test_attach_part(table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part): +def test_attach_part( + table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part +): node.query( f""" -- Catch any errors (NOTE: warnings are ok) From f53f43b78d3cf2da6219ea4bdea7018d9811ae54 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 17:33:47 +0800 Subject: [PATCH 004/117] Fixes for LocalObjectStorage and plain metadata --- .../Local/LocalObjectStorage.cpp | 37 +++++++++++++++++-- .../ObjectStorages/Local/LocalObjectStorage.h | 4 ++ .../MetadataStorageFromPlainObjectStorage.cpp | 5 +-- .../ObjectStorages/ObjectStorageFactory.cpp | 31 ++++++++++------ src/Disks/ObjectStorages/PlainObjectStorage.h | 6 +++ src/Disks/ObjectStorages/S3/DiskS3Utils.cpp | 6 --- src/Disks/ObjectStorages/S3/DiskS3Utils.h | 1 - .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 + .../configs/disk_s3.xml | 4 +- .../test_attach_backup_from_s3_plain/test.py | 7 ++-- 10 files changed, 71 insertions(+), 32 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 02700b358e0..51c260cc270 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -31,6 +31,8 @@ LocalObjectStorage::LocalObjectStorage(String key_prefix_) description = *block_device_id; else description = "/"; + + fs::create_directories(getCommonKeyPrefix()); } bool LocalObjectStorage::exists(const StoredObject & object) const @@ -53,6 +55,7 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(file_path, modified_settings, read_hint, file_size); }; + LOG_TEST(log, "Read object: {}", objects[0].remote_path); switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: @@ -111,8 +114,8 @@ std::unique_ptr LocalObjectStorage::readObject( /// NOLI if (!file_size) file_size = tryGetSizeFromFilePath(path); - LOG_TEST(log, "Read object: {}", path); - return createReadBufferFromFileBase(path, patchSettings(read_settings), read_hint, file_size); + LOG_TEST(log, "Read object: {}", object.remote_path); + return createReadBufferFromFileBase(object.remote_path, patchSettings(read_settings), read_hint, file_size); } std::unique_ptr LocalObjectStorage::writeObject( /// NOLINT @@ -126,6 +129,7 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files"); LOG_TEST(log, "Write object: {}", object.remote_path); + fs::create_directories(fs::path(object.remote_path).parent_path()); return std::make_unique(object.remote_path, buf_size); } @@ -157,9 +161,34 @@ void LocalObjectStorage::removeObjectsIfExist(const StoredObjects & objects) removeObjectIfExists(object); } -ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & /* path */) const +ObjectMetadata LocalObjectStorage::getObjectMetadata(const std::string & path) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Metadata is not supported for LocalObjectStorage"); + ObjectMetadata object_metadata; + LOG_TEST(log, "Getting metadata for path: {}", path); + object_metadata.size_bytes = fs::file_size(path); + object_metadata.last_modified = Poco::Timestamp::fromEpochTime( + std::chrono::duration_cast(fs::last_write_time(path).time_since_epoch()).count()); + return object_metadata; +} + +void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int /* max_keys */) const +{ + for (const auto & entry : fs::directory_iterator(path)) + { + if (entry.is_directory()) + { + listObjects(entry.path(), children, 0); + continue; + } + + auto metadata = getObjectMetadata(entry.path()); + children.emplace_back(entry.path(), std::move(metadata)); + } +} + +bool LocalObjectStorage::existsOrHasAnyChild(const std::string & path) const +{ + return exists(StoredObject(path)); } void LocalObjectStorage::copyObject( // NOLINT diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index ed5f8c1f537..22429a99c76 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -58,6 +58,10 @@ public: ObjectMetadata getObjectMetadata(const std::string & path) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + + bool existsOrHasAnyChild(const std::string & path) const override; + void copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index b03809f5b39..4b8fc74e956 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -48,10 +48,7 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path std::string directory = object_key.serialize(); if (!directory.ends_with('/')) directory += '/'; - - RelativePathsWithMetadata files; - object_storage->listObjects(directory, files, 1); - return !files.empty(); + return object_storage->existsOrHasAnyChild(directory); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 6f6ff199902..f64c42c1403 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -36,16 +36,24 @@ namespace ErrorCodes namespace { + bool isPlainStorage( + ObjectStorageType type, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix) + { + auto compatibility_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(type); + auto metadata_type = MetadataStorageFactory::getMetadataType(config, config_prefix, compatibility_hint); + return metadataTypeFromString(metadata_type) == MetadataStorageType::Plain; + } + template ObjectStoragePtr createObjectStorage( + ObjectStorageType type, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Args && ...args) { - auto compatibility_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(ObjectStorageType::S3); - auto metadata_type = MetadataStorageFactory::getMetadataType(config, config_prefix, compatibility_hint); - - if (metadataTypeFromString(metadata_type) == MetadataStorageType::Plain) + if (isPlainStorage(type, config, config_prefix)) { return std::make_shared>(std::forward(args)...); } @@ -151,10 +159,10 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); auto client = getClient(config, config_prefix, context, *settings); - auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); + auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = createObjectStorage( - config, config_prefix, std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); + ObjectStorageType::S3, config, config_prefix, std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) @@ -187,7 +195,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); auto client = getClient(config, config_prefix, context, *settings); - auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); + auto key_generator = getKeyGenerator(uri, config, config_prefix); auto object_storage = std::make_shared>( std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); @@ -222,7 +230,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) context->getSettingsRef().hdfs_replication ); - return createObjectStorage(config, config_prefix, uri, std::move(settings), config); + return createObjectStorage(ObjectStorageType::HDFS, config, config_prefix, uri, std::move(settings), config); }); } #endif @@ -239,8 +247,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) { String container_name = config.getString(config_prefix + ".container_name", "default-container"); return createObjectStorage( - config, config_prefix, - name, + ObjectStorageType::Azure, config, config_prefix, name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), container_name); @@ -273,7 +280,7 @@ void registerWebObjectStorage(ObjectStorageFactory & factory) ErrorCodes::BAD_ARGUMENTS, "Bad URI: `{}`. Error: {}", uri, e.what()); } - return createObjectStorage(config, config_prefix, uri, context); + return createObjectStorage(ObjectStorageType::Web, config, config_prefix, uri, context); }); } @@ -291,7 +298,7 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory) loadDiskLocalConfig(name, config, config_prefix, context, object_key_prefix, keep_free_space_bytes); /// keys are mapped to the fs, object_key_prefix is a directory also fs::create_directories(object_key_prefix); - return createObjectStorage(config, config_prefix, object_key_prefix); + return createObjectStorage(ObjectStorageType::Local, config, config_prefix, object_key_prefix); }); } #endif diff --git a/src/Disks/ObjectStorages/PlainObjectStorage.h b/src/Disks/ObjectStorages/PlainObjectStorage.h index 3a81b85c44b..e0907d0b4d8 100644 --- a/src/Disks/ObjectStorages/PlainObjectStorage.h +++ b/src/Disks/ObjectStorages/PlainObjectStorage.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -24,6 +25,11 @@ public: bool isWriteOnce() const override { return true; } bool isPlain() const override { return true; } + + ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override + { + return ObjectStorageKey::createAsRelative(BaseObjectStorage::getCommonKeyPrefix(), path); + } }; } diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp index bb7b53b2d22..4b889f89f90 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.cpp @@ -15,16 +15,10 @@ namespace ErrorCodes } ObjectStorageKeysGeneratorPtr getKeyGenerator( - String type, const S3::URI & uri, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - if (type == "s3_plain") - return createObjectStorageKeysGeneratorAsIsWithPrefix(uri.key); - - chassert(type == "s3"); - bool storage_metadata_write_full_object_key = DiskObjectStorageMetadata::getWriteFullObjectKeySetting(); bool send_metadata = config.getBool(config_prefix + ".send_metadata", false); diff --git a/src/Disks/ObjectStorages/S3/DiskS3Utils.h b/src/Disks/ObjectStorages/S3/DiskS3Utils.h index 29e39d4bc1b..8524a9ccac3 100644 --- a/src/Disks/ObjectStorages/S3/DiskS3Utils.h +++ b/src/Disks/ObjectStorages/S3/DiskS3Utils.h @@ -12,7 +12,6 @@ namespace DB namespace S3 { struct URI; } ObjectStorageKeysGeneratorPtr getKeyGenerator( - String type, const S3::URI & uri, const Poco::Util::AbstractConfiguration & config, const String & config_prefix); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 5771eb1ebe0..b2a9ab8fdc3 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -561,6 +561,8 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const { + if (!key_generator) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set"); return key_generator->generate(path); } diff --git a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml index 3166eea7ccb..2edabc76c8b 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml +++ b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml @@ -10,9 +10,9 @@ object_storage - local + local_blob_storage plain - local_plain/ + /local_plain/ diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index 4a8da1e6d66..983275cc24f 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -21,11 +21,11 @@ def start_cluster(): cluster.shutdown() s3_disk_def = """disk(type=s3_plain, - endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/', + endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{}/', access_key_id='minio', secret_access_key='minio123');""" -local_disk_def = "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain'" +local_disk_def = "disk(type=object_storage, object_storage_type = 'local_blob_storage', metadata_type = 'plain', path = '/local_plain/{}/')" @pytest.mark.parametrize( "table_name,backup_name,storage_policy,disk_def,min_bytes_for_wide_part", @@ -41,6 +41,7 @@ local_disk_def = "disk(type=object_storage, object_storage_type = 'local', metad ], ) def test_attach_part(table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part): + disk_definition = disk_def.format(backup_name) node.query( f""" -- Catch any errors (NOTE: warnings are ok) @@ -63,7 +64,7 @@ def test_attach_part(table_name, backup_name, storage_policy, disk_def, min_byte order by key partition by part settings max_suspicious_broken_parts=0, - disk={disk_def} + disk={disk_definition} """ ) From fb38bd139c433ead685028f232e8c4fad5e566d2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 17:38:02 +0800 Subject: [PATCH 005/117] Remove debug logging --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 51c260cc270..4ec998a2bb0 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -55,7 +55,6 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(file_path, modified_settings, read_hint, file_size); }; - LOG_TEST(log, "Read object: {}", objects[0].remote_path); switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: @@ -109,10 +108,8 @@ std::unique_ptr LocalObjectStorage::readObject( /// NOLI std::optional read_hint, std::optional file_size) const { - const auto & path = object.remote_path; - if (!file_size) - file_size = tryGetSizeFromFilePath(path); + file_size = tryGetSizeFromFilePath(object.remote_path); LOG_TEST(log, "Read object: {}", object.remote_path); return createReadBufferFromFileBase(object.remote_path, patchSettings(read_settings), read_hint, file_size); From 978fe9fa1a069a231bb52c66b3898c6ce112a215 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 17:43:34 +0800 Subject: [PATCH 006/117] Add comments --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 4ec998a2bb0..7f34ca48f7f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -126,7 +126,11 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO throw Exception(ErrorCodes::BAD_ARGUMENTS, "LocalObjectStorage doesn't support append to files"); LOG_TEST(log, "Write object: {}", object.remote_path); + + /// Unlike real blob storage, in local fs we cannot create a file with non-existing prefix. + /// So let's create it. fs::create_directories(fs::path(object.remote_path).parent_path()); + return std::make_unique(object.remote_path, buf_size); } @@ -185,6 +189,8 @@ void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWith bool LocalObjectStorage::existsOrHasAnyChild(const std::string & path) const { + /// Unlike real object storage, existance of a prefix path can be checked by + /// just checking existence of this prefix directly, so simple exists is enough here. return exists(StoredObject(path)); } From 33788250b1f74384661cd241e2badef82c8fdbf6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 18:07:19 +0800 Subject: [PATCH 007/117] Update test.py --- tests/integration/test_attach_backup_from_s3_plain/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index 3a0fa70a715..c2f8936b82c 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -26,9 +26,8 @@ s3_disk_def = """disk(type=s3_plain, access_key_id='minio', secret_access_key='minio123');""" -local_disk_def = ( - "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain', path = '/local_plain/{}/'" -) +local_disk_def = "disk(type=object_storage, object_storage_type = 'local_blob_storage', metadata_type = 'plain', path = '/local_plain/{}/');" + @pytest.mark.parametrize( "table_name,backup_name,storage_policy,disk_def,min_bytes_for_wide_part", @@ -67,7 +66,6 @@ local_disk_def = ( ), ], ) - def test_attach_part( table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part ): From 98b27fd45fbe1109442c2313181ca4e8435e2024 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Feb 2024 23:00:27 +0800 Subject: [PATCH 008/117] Fix style check --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 7f34ca48f7f..eba57969580 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -189,7 +189,7 @@ void LocalObjectStorage::listObjects(const std::string & path, RelativePathsWith bool LocalObjectStorage::existsOrHasAnyChild(const std::string & path) const { - /// Unlike real object storage, existance of a prefix path can be checked by + /// Unlike real object storage, existence of a prefix path can be checked by /// just checking existence of this prefix directly, so simple exists is enough here. return exists(StoredObject(path)); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index b2a9ab8fdc3..eec3a5914fc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -48,6 +48,7 @@ namespace ErrorCodes { extern const int S3_ERROR; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } namespace From 416638461fe832673252445d8fabb3fe554eed49 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 27 Feb 2024 15:02:13 +0000 Subject: [PATCH 009/117] Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike --- src/Functions/array/FunctionArrayMapped.h | 4 ++-- .../03002_map_array_functions_with_low_cardinality.reference | 1 + .../03002_map_array_functions_with_low_cardinality.sql | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.reference create mode 100644 tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index 49ed9d495e2..136d3481771 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -355,7 +355,7 @@ public: { arrays.emplace_back( column_tuple->getColumnPtr(j), - recursiveRemoveLowCardinality(type_tuple.getElement(j)), + type_tuple.getElement(j), array_with_type_and_name.name + "." + tuple_names[j]); } } @@ -363,7 +363,7 @@ public: { arrays.emplace_back( column_array->getDataPtr(), - recursiveRemoveLowCardinality(array_type->getNestedType()), + array_type->getNestedType(), array_with_type_and_name.name); } diff --git a/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.reference b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql new file mode 100644 index 00000000000..8240a8f93f5 --- /dev/null +++ b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql @@ -0,0 +1,2 @@ +SELECT mapContainsKeyLike(map('aa', toLowCardinality(1), 'bb', toLowCardinality(2)), toLowCardinality('a%')); + From cb8390e9c8672bcdead0108be75021d6c6f21331 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 28 Feb 2024 13:32:43 +0800 Subject: [PATCH 010/117] Fix build --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index f64c42c1403..d0c2c9ac4f4 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -16,10 +16,10 @@ #ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include #include -#include #include #endif #include +#include #include #include From d2ea882bd8105f5d2e173a6670bf23b2917b3190 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 28 Feb 2024 21:26:19 +0000 Subject: [PATCH 011/117] Fix deadlock in parallel parsing when lots of rows are skipped due to errors --- .../Formats/Impl/ParallelParsingInputFormat.cpp | 4 +++- .../03001_parallel_parsing_deadlock.reference | 0 .../0_stateless/03001_parallel_parsing_deadlock.sh | 12 ++++++++++++ 3 files changed, 15 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03001_parallel_parsing_deadlock.reference create mode 100755 tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 8b6969bbfcc..447adb1ed48 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -224,7 +224,9 @@ Chunk ParallelParsingInputFormat::read() /// skipped all rows. For example, it can happen while using settings /// input_format_allow_errors_num/input_format_allow_errors_ratio /// and this segment contained only rows with errors. - /// Process the next unit. + /// Return this empty unit back to segmentator and process the next unit. + unit->status = READY_TO_INSERT; + segmentator_condvar.notify_all(); ++reader_ticket_number; unit = &processing_units[reader_ticket_number % processing_units.size()]; } diff --git a/tests/queries/0_stateless/03001_parallel_parsing_deadlock.reference b/tests/queries/0_stateless/03001_parallel_parsing_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh b/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh new file mode 100755 index 00000000000..1bf21dfc53b --- /dev/null +++ b/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-cpu-aarch64 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.csv +$CLICKHOUSE_LOCAL -q "select number > 1000000 ? 'error' : toString(number) from numbers(2000000) format CSV" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CSV, 'x UInt64') format Null settings input_format_allow_errors_ratio=1" +rm $DATA_FILE + From 0d4648b535a61561d122c87cf181434215753b35 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 29 Feb 2024 10:30:17 +0800 Subject: [PATCH 012/117] Fix clang-tidy --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index eba57969580..c0b45e1d46a 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -32,7 +32,7 @@ LocalObjectStorage::LocalObjectStorage(String key_prefix_) else description = "/"; - fs::create_directories(getCommonKeyPrefix()); + fs::create_directories(key_prefix); } bool LocalObjectStorage::exists(const StoredObject & object) const From 7632c2c33f357c1c616f734c7bf2502ccbfbd496 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Feb 2024 15:17:12 +0000 Subject: [PATCH 013/117] Remove non-deterministic functions in virtual columns filter --- src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ src/Storages/VirtualColumnUtils.cpp | 21 +++++++++++++++++++ ...with_non_deterministic_functions.reference | 11 ++++++++++ ...lumns_with_non_deterministic_functions.sql | 6 ++++++ 4 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.reference create mode 100644 tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8aa188cfe5c..6494ed5d844 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1082,6 +1082,8 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, true /* one_part */); auto filter_dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), nullptr); + if (!filter_dag) + return {}; // Generate valid expressions for filtering bool valid = true; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 33ff6e7104f..3e0ef1d7990 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -467,6 +467,23 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo return true; } +static bool isDeterministic(const ActionsDAG::Node * node) +{ + if (node->type != ActionsDAG::ActionType::FUNCTION) + return true; + + if (!node->function_base->isDeterministic()) + return false; + + for (const auto * child : node->children) + { + if (!isDeterministic(child)) + return false; + } + + return true; +} + static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( const ActionsDAG::Node * node, const Block * allowed_inputs, @@ -542,6 +559,10 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( } } } + else if (!isDeterministic(node)) + { + return nullptr; + } } if (allowed_inputs && !canEvaluateSubtree(node, *allowed_inputs)) diff --git a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.reference b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.reference new file mode 100644 index 00000000000..4c9646d6ffa --- /dev/null +++ b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.reference @@ -0,0 +1,11 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 diff --git a/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql new file mode 100644 index 00000000000..9f8bc6bd3d7 --- /dev/null +++ b/tests/queries/0_stateless/03002_filter_skip_virtual_columns_with_non_deterministic_functions.sql @@ -0,0 +1,6 @@ +create table test (number UInt64) engine=MergeTree order by number; +insert into test select * from numbers(100000000); +select ignore(number) from test where RAND() > 4292390314 limit 10; +select count() > 0 from test where RAND() > 4292390314; +drop table test; + From 09a392772d75b38e1b19ad6bd2a863168ea0de5c Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Feb 2024 15:34:45 +0000 Subject: [PATCH 014/117] Use isDeterministicInScopeOfQuery --- src/Storages/VirtualColumnUtils.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 3e0ef1d7990..6d66453442e 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -467,17 +467,17 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo return true; } -static bool isDeterministic(const ActionsDAG::Node * node) +static bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) { if (node->type != ActionsDAG::ActionType::FUNCTION) return true; - if (!node->function_base->isDeterministic()) + if (!node->function_base->isDeterministicInScopeOfQuery()) return false; for (const auto * child : node->children) { - if (!isDeterministic(child)) + if (!isDeterministicInScopeOfQuery(child)) return false; } @@ -559,7 +559,7 @@ static const ActionsDAG::Node * splitFilterNodeForAllowedInputs( } } } - else if (!isDeterministic(node)) + else if (!isDeterministicInScopeOfQuery(node)) { return nullptr; } From c435d5894f48d37478454b1934d000fb967e2973 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 4 Mar 2024 14:23:59 +0800 Subject: [PATCH 015/117] remove wrong assertion n quantileGK --- .../AggregateFunctionGroupArray.cpp | 13 ++++++++----- .../AggregateFunctionQuantileGK.cpp | 12 ++++-------- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index d72ddb42d9e..6af8b1018dd 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -182,11 +182,14 @@ public: if constexpr (Trait::sampler == Sampler::NONE) { - if (limit_num_elems && cur_elems.value.size() >= max_elems) + if constexpr (limit_num_elems) { - if constexpr (Trait::last) - cur_elems.value[(cur_elems.total_values - 1) % max_elems] = row_value; - return; + if (cur_elems.value.size() >= max_elems) + { + if constexpr (Trait::last) + cur_elems.value[(cur_elems.total_values - 1) % max_elems] = row_value; + return; + } } cur_elems.value.push_back(row_value, arena); @@ -236,7 +239,7 @@ public: void mergeNoSampler(Data & cur_elems, const Data & rhs_elems, Arena * arena) const { - if (!limit_num_elems) + if constexpr (!limit_num_elems) { if (rhs_elems.value.size()) cur_elems.value.insertByOffsets(rhs_elems.value, 0, rhs_elems.value.size(), arena); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp b/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp index 2e8ccb2e5e4..26737e43eef 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileGK.cpp @@ -144,7 +144,7 @@ public: count = other.count; compressed = other.compressed; - sampled.resize(other.sampled.size()); + sampled.resize_exact(other.sampled.size()); memcpy(sampled.data(), other.sampled.data(), sizeof(Stats) * other.sampled.size()); return; } @@ -180,7 +180,7 @@ public: compress(); backup_sampled.clear(); - backup_sampled.reserve(sampled.size() + other.sampled.size()); + backup_sampled.reserve_exact(sampled.size() + other.sampled.size()); double merged_relative_error = std::max(relative_error, other.relative_error); size_t merged_count = count + other.count; Int64 additional_self_delta = static_cast(std::floor(2 * other.relative_error * other.count)); @@ -268,11 +268,7 @@ public: size_t sampled_len = 0; readBinaryLittleEndian(sampled_len, buf); - if (sampled_len > compress_threshold) - throw Exception( - ErrorCodes::INCORRECT_DATA, "The number of elements {} for quantileGK exceeds {}", sampled_len, compress_threshold); - - sampled.resize(sampled_len); + sampled.resize_exact(sampled_len); for (size_t i = 0; i < sampled_len; ++i) { @@ -317,7 +313,7 @@ private: ::sort(head_sampled.begin(), head_sampled.end()); backup_sampled.clear(); - backup_sampled.reserve(sampled.size() + head_sampled.size()); + backup_sampled.reserve_exact(sampled.size() + head_sampled.size()); size_t sample_idx = 0; size_t ops_idx = 0; From a7db6688edb50f894457c414b207c25548bb18d3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Mar 2024 18:24:24 +0800 Subject: [PATCH 016/117] Update ObjectStorageFactory.cpp --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 47c02f87b23..a0578ac4454 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -246,12 +246,11 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) bool /* skip_access_check */) -> ObjectStoragePtr { AzureBlobStorageEndpoint endpoint = processAzureBlobStorageEndpoint(config, config_prefix); - return std::make_unique( + return createObjectStorage( ObjectStorageType::Azure, config, config_prefix, name, getAzureBlobContainerClient(config, config_prefix), getAzureBlobStorageSettings(config, config_prefix, context), endpoint.prefix.empty() ? endpoint.container_name : endpoint.container_name + "/" + endpoint.prefix); - }); } #endif From 47ad21dd257ff1a5751d191dfd311a7950a93111 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Mar 2024 12:17:04 +0100 Subject: [PATCH 017/117] Remove extra empty line --- .../03002_map_array_functions_with_low_cardinality.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql index 8240a8f93f5..8820a433da8 100644 --- a/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql +++ b/tests/queries/0_stateless/03002_map_array_functions_with_low_cardinality.sql @@ -1,2 +1 @@ SELECT mapContainsKeyLike(map('aa', toLowCardinality(1), 'bb', toLowCardinality(2)), toLowCardinality('a%')); - From aa6b70e5f2187be71b6bce835ecff0aa0c0bfca7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 5 Mar 2024 16:55:08 +0000 Subject: [PATCH 018/117] Add documentation to `simpleJSON` functions --- .../sql-reference/functions/json-functions.md | 392 +++++++++++++++--- 1 file changed, 342 insertions(+), 50 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 2c837ff4a42..246cb8972fb 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -5,80 +5,372 @@ sidebar_label: JSON --- There are two sets of functions to parse JSON. - - `visitParam*` (`simpleJSON*`) is made to parse a special very limited subset of a JSON, but these functions are extremely fast. + - `simpleJSON*` (`visitParam*`) is made to parse a special very limited subset of a JSON, but these functions are extremely fast. - `JSONExtract*` is made to parse normal JSON. -# visitParam functions +# simpleJSON/visitParam functions ClickHouse has special functions for working with simplified JSON. All these JSON functions are based on strong assumptions about what the JSON can be, but they try to do as little as possible to get the job done. The following assumptions are made: 1. The field name (function argument) must be a constant. -2. The field name is somehow canonically encoded in JSON. For example: `visitParamHas('{"abc":"def"}', 'abc') = 1`, but `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` +2. The field name is somehow canonically encoded in JSON. For example: `simpleJSONHas('{"abc":"def"}', 'abc') = 1`, but `simpleJSONHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` 3. Fields are searched for on any nesting level, indiscriminately. If there are multiple matching fields, the first occurrence is used. 4. The JSON does not have space characters outside of string literals. -## visitParamHas(params, name) +## simpleJSONHas -Checks whether there is a field with the `name` name. +Checks whether there is a field named `field_name`. The result is `UInt8`. -Alias: `simpleJSONHas`. +**Syntax** -## visitParamExtractUInt(params, name) - -Parses UInt64 from the value of the field named `name`. If this is a string field, it tries to parse a number from the beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns 0. - -Alias: `simpleJSONExtractUInt`. - -## visitParamExtractInt(params, name) - -The same as for Int64. - -Alias: `simpleJSONExtractInt`. - -## visitParamExtractFloat(params, name) - -The same as for Float64. - -Alias: `simpleJSONExtractFloat`. - -## visitParamExtractBool(params, name) - -Parses a true/false value. The result is UInt8. - -Alias: `simpleJSONExtractBool`. - -## visitParamExtractRaw(params, name) - -Returns the value of a field, including separators. - -Alias: `simpleJSONExtractRaw`. - -Examples: - -``` sql -visitParamExtractRaw('{"abc":"\\n\\u0000"}', 'abc') = '"\\n\\u0000"'; -visitParamExtractRaw('{"abc":{"def":[1,2,3]}}', 'abc') = '{"def":[1,2,3]}'; +```sql +simpleJSONHas(json, field_name) ``` -## visitParamExtractString(params, name) +**Parameters** -Parses the string in double quotes. The value is unescaped. If unescaping failed, it returns an empty string. +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) -Alias: `simpleJSONExtractString`. +**Returned value** -Examples: +It returns `1` if the field exists, `0` otherwise. -``` sql -visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0'; -visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺'; -visitParamExtractString('{"abc":"\\u263"}', 'abc') = ''; -visitParamExtractString('{"abc":"hello}', 'abc') = ''; +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"true","qux":1}'); + +SELECT simpleJSONHas(json, 'foo') FROM jsons; +SELECT simpleJSONHas(json, 'bar') FROM jsons; ``` +```response +1 +0 +``` +## simpleJSONExtractUInt + +Parses `UInt64` from the value of the field named `field_name`. If this is a string field, it tries to parse a number from the beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns `0`. + +**Syntax** + +```sql +simpleJSONExtractUInt(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns the number parsed from the field if the field exists and contains a number, `0` otherwise. + +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"4e3"}'); +INSERT INTO jsons VALUES ('{"foo":3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractUInt(json, 'foo') FROM jsons ORDER BY json; +``` + +```response +0 +4 +0 +3 +5 +``` + +## simpleJSONExtractInt + +Parses `Int64` from the value of the field named `field_name`. If this is a string field, it tries to parse a number from the beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns `0`. + +**Syntax** + +```sql +simpleJSONExtractInt(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns the number parsed from the field if the field exists and contains a number, `0` otherwise. + +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractInt(json, 'foo') FROM jsons ORDER BY json; +``` + +```response +0 +-4 +0 +-3 +5 +``` + +## simpleJSONExtractFloat + +Parses `Float64` from the value of the field named `field_name`. If this is a string field, it tries to parse a number from the beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns `0`. + +**Syntax** + +```sql +simpleJSONExtractFloat(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns the number parsed from the field if the field exists and contains a number, `0` otherwise. + +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractFloat(json, 'foo') FROM jsons ORDER BY json; +``` + +```response +0 +-4000 +0 +-3.4 +5 +``` + +## simpleJSONExtractBool + +Parses a true/false value from the value of the field named `field_name`. The result is `UInt8`. + +**Syntax** + +```sql +simpleJSONExtractBool(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns `1` if the value of the field is `true`, `0` otherwise. This means this function will return `0` including (and not only) in the following cases: + - If the field doesn't exists. + - If the field contains `true` as a string, e.g.: `{"field":"true"}`. + - If the field contains `1` as a numerical value. + +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":false,"bar":true}'); +INSERT INTO jsons VALUES ('{"foo":"true","qux":1}'); + +SELECT simpleJSONExtractBool(json, 'bar') FROM jsons ORDER BY json; +SELECT simpleJSONExtractBool(json, 'foo') FROM jsons ORDER BY json; +``` + +```response +0 +1 +0 +0 +``` + +## simpleJSONExtractRaw + +Returns the value of the field named `field_name` as a `String`, including separators. + +**Syntax** + +```sql +simpleJSONExtractRaw(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns the value of the field as a [`String`](../../sql-reference/data-types/string.md#string), including separators if the field exists, or an emtpy `String` otherwise. + +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":{"def":[1,2,3]}}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractRaw(json, 'foo') FROM jsons ORDER BY json; +``` + +```response + +"-4e3" +-3.4 +5 +{"def":[1,2,3]} +``` + +## simpleJSONExtractString + +Parses `String` in double quotes from the value of the field named `field_name`. + +**Syntax** + +```sql +simpleJSONExtractString(json, field_name) +``` + +**Parameters** + +- `json`: The JSON in which the field is searched for. [String](../../sql-reference/data-types/string.md#string) +- `field_name`: The name of the field to search for. [String literal](../syntax#string) + +**Returned value** + +It returns the value of a field as a [`String`](../../sql-reference/data-types/string.md#string), including separators. The value is unescaped. It returns an empty `String`: if the field doesn't contain a double quoted string, if unescaping fails or if the field doesn't exist. + +**Implementation details** + There is currently no support for code points in the format `\uXXXX\uYYYY` that are not from the basic multilingual plane (they are converted to CESU-8 instead of UTF-8). +**Example** + +Query: + +```sql +CREATE TABLE jsons +( + `json` String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"\\n\\u0000"}'); +INSERT INTO jsons VALUES ('{"foo":"\\u263"}'); +INSERT INTO jsons VALUES ('{"foo":"\\u263a"}'); +INSERT INTO jsons VALUES ('{"foo":"hello}'); + +SELECT simpleJSONExtractString(json, 'foo') FROM jsons ORDER BY json; +``` + +```response +\n\0 + +☺ + +``` + +## visitParamHas + +This function is [an alias of `simpleJSONHas`](./json-functions#simplejsonhas). + +## visitParamExtractUInt + +This function is [an alias of `simpleJSONExtractUInt`](./json-functions#simplejsonextractuint). + +## visitParamExtractInt + +This function is [an alias of `simpleJSONExtractInt`](./json-functions#simplejsonextractint). + +## visitParamExtractFloat + +This function is [an alias of `simpleJSONExtractFloat`](./json-functions#simplejsonextractfloat). + +## visitParamExtractBool + +This function is [an alias of `simpleJSONExtractBool`](./json-functions#simplejsonextractbool). + +## visitParamExtractRaw + +This function is [an alias of `simpleJSONExtractRaw`](./json-functions#simplejsonextractraw). + +## visitParamExtractString + +This function is [an alias of `simpleJSONExtractString`](./json-functions#simplejsonextractstring). + # JSONExtract functions The following functions are based on [simdjson](https://github.com/lemire/simdjson) designed for more complex JSON parsing requirements. From 981c507d8007a4f7761a83a2ecfa0956a364317d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 5 Mar 2024 17:01:54 +0000 Subject: [PATCH 019/117] Add example to `sin`. --- docs/en/sql-reference/functions/math-functions.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index b27668caf0c..fc659891b5c 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -299,6 +299,18 @@ sin(x) Type: [Float*](../../sql-reference/data-types/float.md). +**Example** + +Query: + +```sql +SELECT sin(1.23); +``` + +```response +0.9424888019316975 +``` + ## cos Returns the cosine of the argument. From 6d4514c045cc565919f9c8384710eee89354f0f3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Mar 2024 16:55:48 +0800 Subject: [PATCH 020/117] Fix test --- src/Storages/System/StorageSystemDisks.cpp | 10 +++++++++- tests/integration/test_backup_restore_s3/test.py | 12 ++++++------ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 30d64156b22..0f8a6640f2c 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -25,6 +25,8 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) {"unreserved_space", std::make_shared()}, {"keep_free_space", std::make_shared()}, {"type", std::make_shared()}, + {"object_storage_type", std::make_shared()}, + {"metadata_type", std::make_shared()}, {"is_encrypted", std::make_shared()}, {"is_read_only", std::make_shared()}, {"is_write_once", std::make_shared()}, @@ -53,6 +55,8 @@ Pipe StorageSystemDisks::read( MutableColumnPtr col_unreserved = ColumnUInt64::create(); MutableColumnPtr col_keep = ColumnUInt64::create(); MutableColumnPtr col_type = ColumnString::create(); + MutableColumnPtr col_object_storage_type = ColumnString::create(); + MutableColumnPtr col_metadata_type = ColumnString::create(); MutableColumnPtr col_is_encrypted = ColumnUInt8::create(); MutableColumnPtr col_is_read_only = ColumnUInt8::create(); MutableColumnPtr col_is_write_once = ColumnUInt8::create(); @@ -69,7 +73,9 @@ Pipe StorageSystemDisks::read( col_unreserved->insert(disk_ptr->getUnreservedSpace().value_or(std::numeric_limits::max())); col_keep->insert(disk_ptr->getKeepingFreeSpace()); auto data_source_description = disk_ptr->getDataSourceDescription(); - col_type->insert(data_source_description.toString()); + col_type->insert(data_source_description.type); + col_object_storage_type->insert(data_source_description.object_storage_type); + col_metadata_type->insert(data_source_description.metadata_type); col_is_encrypted->insert(data_source_description.is_encrypted); col_is_read_only->insert(disk_ptr->isReadOnly()); col_is_write_once->insert(disk_ptr->isWriteOnce()); @@ -91,6 +97,8 @@ Pipe StorageSystemDisks::read( res_columns.emplace_back(std::move(col_unreserved)); res_columns.emplace_back(std::move(col_keep)); res_columns.emplace_back(std::move(col_type)); + res_columns.emplace_back(std::move(col_object_storage_type)); + res_columns.emplace_back(std::move(col_metadata_type)); res_columns.emplace_back(std::move(col_is_encrypted)); res_columns.emplace_back(std::move(col_is_read_only)); res_columns.emplace_back(std::move(col_is_write_once)); diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 4d3ee8200a3..95e264107e4 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -124,15 +124,15 @@ def check_backup_and_restore( def check_system_tables(backup_query_id=None): disks = [ tuple(disk.split("\t")) - for disk in node.query("SELECT name, type FROM system.disks").split("\n") + for disk in node.query("SELECT name, type, object_storage_type, metadata_type FROM system.disks").split("\n") if disk ] expected_disks = ( - ("default", "local"), - ("disk_s3", "s3"), - ("disk_s3_cache", "s3"), - ("disk_s3_other_bucket", "s3"), - ("disk_s3_plain", "s3_plain"), + ("default", "local", "", ""), + ("disk_s3", "object_storage", "s3", "local"), + ("disk_s3_cache", "object_storage", "s3", "local"), + ("disk_s3_other_bucket", "object_storage", "s3", "local"), + ("disk_s3_plain", "object_storage", "s3", "plain"), ) assert len(expected_disks) == len(disks) for expected_disk in expected_disks: From be98c95f586762cdf20a6375917e30f296175593 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Mar 2024 09:12:26 +0000 Subject: [PATCH 021/117] Automatic style fix --- tests/integration/test_backup_restore_s3/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 95e264107e4..452a9143067 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -124,7 +124,9 @@ def check_backup_and_restore( def check_system_tables(backup_query_id=None): disks = [ tuple(disk.split("\t")) - for disk in node.query("SELECT name, type, object_storage_type, metadata_type FROM system.disks").split("\n") + for disk in node.query( + "SELECT name, type, object_storage_type, metadata_type FROM system.disks" + ).split("\n") if disk ] expected_disks = ( From 56fb61e1866e81e9a00b9b98299ddc56a54f5394 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 6 Mar 2024 10:53:39 +0000 Subject: [PATCH 022/117] Do not duplicate the first category in case of multiple categories in `FunctionDocumentation` --- src/Common/FunctionDocumentation.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/FunctionDocumentation.cpp b/src/Common/FunctionDocumentation.cpp index 2aad23b90b7..0dc5b48f9d1 100644 --- a/src/Common/FunctionDocumentation.cpp +++ b/src/Common/FunctionDocumentation.cpp @@ -36,6 +36,7 @@ std::string FunctionDocumentation::categoriesAsString() const auto it = categories.begin(); std::string res = *it; + ++it; for (; it != categories.end(); ++it) res += ", " + *it; return res; From 6f726865baf3fea606e7ff46e5d8cd98bda94f5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 6 Mar 2024 11:10:02 +0000 Subject: [PATCH 023/117] Add inline docs to functions --- src/Functions/sin.cpp | 10 +++++- src/Functions/visitParamExtractBool.cpp | 30 +++++++++++++++++- src/Functions/visitParamExtractFloat.cpp | 31 ++++++++++++++++++- src/Functions/visitParamExtractInt.cpp | 31 ++++++++++++++++++- src/Functions/visitParamExtractRaw.cpp | 30 +++++++++++++++++- src/Functions/visitParamExtractString.cpp | 30 +++++++++++++++++- src/Functions/visitParamExtractUInt.cpp | 31 ++++++++++++++++++- src/Functions/visitParamHas.cpp | 23 +++++++++++++- ...new_functions_must_be_documented.reference | 8 ----- 9 files changed, 208 insertions(+), 16 deletions(-) diff --git a/src/Functions/sin.cpp b/src/Functions/sin.cpp index dc75f4800c0..914f431adb4 100644 --- a/src/Functions/sin.cpp +++ b/src/Functions/sin.cpp @@ -13,7 +13,15 @@ using FunctionSin = FunctionMathUnary>; REGISTER_FUNCTION(Sin) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction( + FunctionDocumentation{ + .description = "Returns the sine of the argument.", + .syntax = "sin(x)", + .arguments = {{"x", "The number whose sine will be returned. (U)Int*, Float* or Decimal*."}}, + .returned_value = "The sine of x.", + .examples = {{.name = "simple", .query = "SELECT sin(1.23)", .result = "0.9424888019316975"}}, + .categories{"Mathematical", "Trigonometric"}}, + FunctionFactory::CaseInsensitive); } } diff --git a/src/Functions/visitParamExtractBool.cpp b/src/Functions/visitParamExtractBool.cpp index 31763fe54ce..2c413ec13bb 100644 --- a/src/Functions/visitParamExtractBool.cpp +++ b/src/Functions/visitParamExtractBool.cpp @@ -21,7 +21,35 @@ using FunctionSimpleJSONExtractBool = FunctionsStringSearch(); + factory.registerFunction(FunctionDocumentation{ + .description = "Parses a true/false value from the value of the field named field_name. The result is UInt8.", + .syntax = "simpleJSONExtractBool(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value + = R"(It returns 1 if the value of the field is true, 0 otherwise. This means this function will return 0 including (and not only) in the following cases: + - If the field doesn't exists. + - If the field contains true as a string, e.g.: {"field":"true"}. + - If the field contains 1 as a numerical value.)", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":false,"bar":true}'); +INSERT INTO jsons VALUES ('{"foo":"true","qux":1}'); + +SELECT simpleJSONExtractBool(json, 'bar') FROM jsons ORDER BY json; +SELECT simpleJSONExtractBool(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"(0 +1 +0 +0)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractBool", "simpleJSONExtractBool"); } diff --git a/src/Functions/visitParamExtractFloat.cpp b/src/Functions/visitParamExtractFloat.cpp index 6f6d5274050..fc839142cc7 100644 --- a/src/Functions/visitParamExtractFloat.cpp +++ b/src/Functions/visitParamExtractFloat.cpp @@ -11,7 +11,36 @@ using FunctionSimpleJSONExtractFloat = FunctionsStringSearch(); + factory.registerFunction(FunctionDocumentation{ + .description + = "Parses Float64 from the value of the field named field_name. If this is a string field, it tries to parse a number from the " + "beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns 0.", + .syntax = "simpleJSONExtractFloat(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value = "It returns the number parsed from the field if the field exists and contains a number, 0 otherwise.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractFloat(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"(0 +-4000 +0 +-3.4 +5)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractFloat", "simpleJSONExtractFloat"); } diff --git a/src/Functions/visitParamExtractInt.cpp b/src/Functions/visitParamExtractInt.cpp index e020c43e8b4..4588fc55c52 100644 --- a/src/Functions/visitParamExtractInt.cpp +++ b/src/Functions/visitParamExtractInt.cpp @@ -11,7 +11,36 @@ using FunctionSimpleJSONExtractInt = FunctionsStringSearch(); + factory.registerFunction(FunctionDocumentation{ + .description + = "Parses Int64 from the value of the field named field_name. If this is a string field, it tries to parse a number from the " + "beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns 0.", + .syntax = "simpleJSONExtractInt(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value = "It returns the number parsed from the field if the field exists and contains a number, 0 otherwise.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractInt(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"(0 +-4 +0 +-3 +5)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractInt", "simpleJSONExtractInt"); } diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index 74a83170545..296429423fe 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -61,7 +61,35 @@ using FunctionSimpleJSONExtractRaw = FunctionsStringSearchToString(); + factory.registerFunction(FunctionDocumentation{ + .description = "Returns the value of the field named field_name as a String, including separators.", + .syntax = "simpleJSONExtractRaw(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value + = "It returns the value of the field as a String including separators if the field exists, or an emtpy String otherwise.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"-4e3"}'); +INSERT INTO jsons VALUES ('{"foo":-3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":{"def":[1,2,3]}}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractRaw(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"( +"-4e3" +-3.4 +5 +{"def":[1,2,3]})"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractRaw", "simpleJSONExtractRaw"); } diff --git a/src/Functions/visitParamExtractString.cpp b/src/Functions/visitParamExtractString.cpp index 50d5f345189..8dae10638f8 100644 --- a/src/Functions/visitParamExtractString.cpp +++ b/src/Functions/visitParamExtractString.cpp @@ -22,7 +22,35 @@ using FunctionSimpleJSONExtractString = FunctionsStringSearchToString(); + factory.registerFunction(FunctionDocumentation{ + .description = R"(Parses String in double quotes from the value of the field named field_name. + + There is currently no support for code points in the format \uXXXX\uYYYY that are not from the basic multilingual plane (they are converted to CESU-8 instead of UTF-8).)", + .syntax = "simpleJSONExtractString(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value = "It returns the value of a field as a String, including separators. The value is unescaped. It returns an empty " + "String: if the field doesn't contain a double quoted string, if unescaping fails or if the field doesn't exist.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"\\n\\u0000"}'); +INSERT INTO jsons VALUES ('{"foo":"\\u263"}'); +INSERT INTO jsons VALUES ('{"foo":"\\u263a"}'); +INSERT INTO jsons VALUES ('{"foo":"hello}'); + +SELECT simpleJSONExtractString(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"(\n\0 + +☺ +)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractString", "simpleJSONExtractString"); } diff --git a/src/Functions/visitParamExtractUInt.cpp b/src/Functions/visitParamExtractUInt.cpp index fb58e417f34..777df9fdd24 100644 --- a/src/Functions/visitParamExtractUInt.cpp +++ b/src/Functions/visitParamExtractUInt.cpp @@ -12,7 +12,36 @@ using FunctionSimpleJSONExtractUInt = FunctionsStringSearch(); + factory.registerFunction(FunctionDocumentation{ + .description + = "Parses UInt64 from the value of the field named field_name. If this is a string field, it tries to parse a number from the " + "beginning of the string. If the field does not exist, or it exists but does not contain a number, it returns 0.", + .syntax = "simpleJSONExtractUInt(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value = "It returns the number parsed from the field if the field exists and contains a number, 0 otherwise.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"4e3"}'); +INSERT INTO jsons VALUES ('{"foo":3.4}'); +INSERT INTO jsons VALUES ('{"foo":5}'); +INSERT INTO jsons VALUES ('{"foo":"not1number"}'); +INSERT INTO jsons VALUES ('{"baz":2}'); + +SELECT simpleJSONExtractUInt(json, 'foo') FROM jsons ORDER BY json;)", + .result = R"(0 +4 +0 +3 +5)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamExtractUInt", "simpleJSONExtractUInt"); } diff --git a/src/Functions/visitParamHas.cpp b/src/Functions/visitParamHas.cpp index 1ed1f1d16e7..09fec782980 100644 --- a/src/Functions/visitParamHas.cpp +++ b/src/Functions/visitParamHas.cpp @@ -21,7 +21,28 @@ using FunctionSimpleJSONHas = FunctionsStringSearch(); + factory.registerFunction(FunctionDocumentation{ + .description = "Checks whether there is a field named field_name. The result is UInt8.", + .syntax = "simpleJSONHas(json, field_name)", + .arguments + = {{"json", "The JSON in which the field is searched for. String."}, + {"field_name", "The name of the field to search for. String literal."}}, + .returned_value = "It returns 1 if the field exists, 0 otherwise.", + .examples + = {{.name = "simple", + .query = R"(CREATE TABLE jsons +( + json String +) +ENGINE = Memory; + +INSERT INTO jsons VALUES ('{"foo":"true","qux":1}'); + +SELECT simpleJSONHas(json, 'foo') FROM jsons; +SELECT simpleJSONHas(json, 'bar') FROM jsons;)", + .result = R"(1 +0)"}}, + .categories{"JSON"}}); factory.registerAlias("visitParamHas", "simpleJSONHas"); } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 379eea4dbbb..0a11e8b5034 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -643,14 +643,6 @@ shardNum showCertificate sigmoid sign -simpleJSONExtractBool -simpleJSONExtractFloat -simpleJSONExtractInt -simpleJSONExtractRaw -simpleJSONExtractString -simpleJSONExtractUInt -simpleJSONHas -sin sinh sipHash128 sipHash128Keyed From 5b94f9b4115e3b7e03118b4a4f4999139e58511e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Mar 2024 15:31:19 +0100 Subject: [PATCH 024/117] Check children first --- src/Storages/VirtualColumnUtils.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 6d66453442e..e8441b96782 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -469,18 +469,18 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo static bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) { - if (node->type != ActionsDAG::ActionType::FUNCTION) - return true; - - if (!node->function_base->isDeterministicInScopeOfQuery()) - return false; - for (const auto * child : node->children) { if (!isDeterministicInScopeOfQuery(child)) return false; } + if (node->type != ActionsDAG::ActionType::FUNCTION) + return true; + + if (!node->function_base->isDeterministicInScopeOfQuery()) + return false; + return true; } From 526f162082dfbb4ad2fb5d3d807dfd2ad9b54bdd Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 29 Feb 2024 18:20:47 +0000 Subject: [PATCH 025/117] Fix logical error on bad compatibility setting value type --- src/Core/Settings.cpp | 4 ++++ .../03003_compatibility_setting_bad_value.reference | 0 .../0_stateless/03003_compatibility_setting_bad_value.sql | 2 ++ 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03003_compatibility_setting_bad_value.reference create mode 100644 tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index a38197b9eeb..fb456b46d89 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -114,7 +114,11 @@ std::vector Settings::getAllRegisteredNames() const void Settings::set(std::string_view name, const Field & value) { if (name == "compatibility") + { + if (value.getType() != Field::Types::Which::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type of value for setting 'compatibility'. Expected String, got {}", value.getTypeName()); applyCompatibilitySetting(value.get()); + } /// If we change setting that was changed by compatibility setting before /// we should remove it from settings_changed_by_compatibility_setting, /// otherwise the next time we will change compatibility setting diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.reference b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql new file mode 100644 index 00000000000..9a6f4e7944a --- /dev/null +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -0,0 +1,2 @@ +select 42 settings compatibility=NULL; -- {clientError BAD_GET} + From bdb76d9dd4b42ab4f40db0d371165665171afb4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 6 Mar 2024 16:30:22 +0000 Subject: [PATCH 026/117] Fix aspell errors --- docs/en/sql-reference/functions/json-functions.md | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 12 +++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 246cb8972fb..e920ab82988 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -264,7 +264,7 @@ simpleJSONExtractRaw(json, field_name) **Returned value** -It returns the value of the field as a [`String`](../../sql-reference/data-types/string.md#string), including separators if the field exists, or an emtpy `String` otherwise. +It returns the value of the field as a [`String`](../../sql-reference/data-types/string.md#string), including separators if the field exists, or an empty `String` otherwise. **Example** diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3614bcb7452..917b2cdcc71 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2724 +personal_ws-1.1 en 2758 AArch ACLs ALTERs @@ -843,7 +843,6 @@ SendScalars ShareAlike SharedMergeTree Shortkeys -Shortkeys SimHash Simhash SimpleAggregateFunction @@ -1703,7 +1702,6 @@ hyperscan hypot hyvor iTerm -iTerm icosahedron icudata idempotency @@ -2327,6 +2325,14 @@ shortcircuit shortkeys shoutout simdjson +simpleJSON +simpleJSONExtractBool +simpleJSONExtractFloat +simpleJSONExtractInt +simpleJSONExtractRaw +simpleJSONExtractString +simpleJSONExtractUInt +simpleJSONHas simpleLinearRegression simpleaggregatefunction simplelinearregression From 77a980373a1dab7c49e5713ba7050d218c1250c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 6 Mar 2024 16:31:27 +0000 Subject: [PATCH 027/117] Fix typo in inline doc --- src/Functions/visitParamExtractRaw.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index 296429423fe..3cdc5001e13 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -68,7 +68,7 @@ REGISTER_FUNCTION(VisitParamExtractRaw) = {{"json", "The JSON in which the field is searched for. String."}, {"field_name", "The name of the field to search for. String literal."}}, .returned_value - = "It returns the value of the field as a String including separators if the field exists, or an emtpy String otherwise.", + = "It returns the value of the field as a String including separators if the field exists, or an empty String otherwise.", .examples = {{.name = "simple", .query = R"(CREATE TABLE jsons From 50b84954e4810c94c1397504a64ca96e1a0fed55 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Mar 2024 16:29:38 +0800 Subject: [PATCH 028/117] Update .reference --- .../0_stateless/02117_show_create_table_system.reference | 2 ++ 1 file changed, 2 insertions(+) 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 7382b24afbc..5081527ceef 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -195,6 +195,8 @@ CREATE TABLE system.disks `unreserved_space` UInt64, `keep_free_space` UInt64, `type` String, + `object_storage_type` String, + `metadata_type` String, `is_encrypted` UInt8, `is_read_only` UInt8, `is_write_once` UInt8, From 31ed1966e3c5388e601edd6e97c0497153bb7196 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Mar 2024 16:44:10 +0800 Subject: [PATCH 029/117] Fix build --- src/Disks/ObjectStorages/ObjectStorageFactory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 9d7e714445a..46136ad7b12 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -166,7 +166,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name); + checkS3Capabilities(*dynamic_cast(object_storage.get()), s3_capabilities, name); return object_storage; }); @@ -202,7 +202,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) /// NOTE: should we still perform this check for clickhouse-disks? if (!skip_access_check) - checkS3Capabilities(*object_storage, s3_capabilities, name); + checkS3Capabilities(*dynamic_cast(object_storage.get()), s3_capabilities, name); return object_storage; }); From 6c69e7d4dcfdfa21cfcaa103fc1cc7c53dfe0291 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 7 Mar 2024 20:29:04 +0800 Subject: [PATCH 030/117] detect output format by file extension in clickhouse-local --- programs/local/LocalServer.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 68f0e52ce08..20974dd9751 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -327,6 +327,14 @@ static bool checkIfStdinIsRegularFile() return fstat(STDIN_FILENO, &file_stat) == 0 && S_ISREG(file_stat.st_mode); } + +static bool checkIfStdoutIsRegularFile() +{ + struct stat file_stat; + return fstat(STDOUT_FILENO, &file_stat) == 0 && S_ISREG(file_stat.st_mode); +} + + std::string LocalServer::getInitialCreateTableQuery() { if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format") && (!checkIfStdinIsRegularFile() || queries.empty())) @@ -638,7 +646,14 @@ void LocalServer::processConfig() if (config().has("macros")) global_context->setMacros(std::make_unique(config(), "macros", log)); - format = config().getString("output-format", config().getString("format", is_interactive ? "PrettyCompact" : "TSV")); + if (!config().has("output-format") && !config().has("format") && checkIfStdoutIsRegularFile()) + { + std::optional format_from_file_name; + format_from_file_name = FormatFactory::instance().tryGetFormatFromFileDescriptor(STDOUT_FILENO); + format = format_from_file_name ? *format_from_file_name : "TSV"; + } + else + format = config().getString("output-format", config().getString("format", is_interactive ? "PrettyCompact" : "TSV")); insert_format = "Values"; /// Setting value from cmd arg overrides one from config From 6d5fd2857ed50047d8acf48766165aa815ca30b9 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 7 Mar 2024 20:29:42 +0800 Subject: [PATCH 031/117] detect output format by file extension in clickhouse-client --- programs/client/Client.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index a2bd6b6016a..fac34003553 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -50,6 +50,7 @@ #include #include #include +#include namespace fs = std::filesystem; using namespace std::literals; @@ -1137,6 +1138,13 @@ void Client::processOptions(const OptionsDescription & options_description, } +static bool checkIfStdoutIsRegularFile() +{ + struct stat file_stat; + return fstat(STDOUT_FILENO, &file_stat) == 0 && S_ISREG(file_stat.st_mode); +} + + void Client::processConfig() { if (!queries.empty() && config().has("queries-file")) @@ -1173,7 +1181,14 @@ void Client::processConfig() pager = config().getString("pager", ""); is_default_format = !config().has("vertical") && !config().has("format"); - if (config().has("vertical")) + if (is_default_format && checkIfStdoutIsRegularFile()) + { + is_default_format = false; + std::optional format_from_file_name; + format_from_file_name = FormatFactory::instance().tryGetFormatFromFileDescriptor(STDOUT_FILENO); + format = format_from_file_name ? *format_from_file_name : "TabSeparated"; + } + else if (config().has("vertical")) format = config().getString("format", "Vertical"); else format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated"); From 112c1efb7da2619cb67a48ff7fbe65ecea8e44a9 Mon Sep 17 00:00:00 2001 From: HowePa <2873679104@qq.com> Date: Thu, 7 Mar 2024 20:30:24 +0800 Subject: [PATCH 032/117] test detect output format by file extension --- ..._output_format_by_file_extension.reference | 20 +++++++++++++++++++ ..._detect_output_format_by_file_extension.sh | 13 ++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02181_detect_output_format_by_file_extension.reference create mode 100755 tests/queries/0_stateless/02181_detect_output_format_by_file_extension.sh diff --git a/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.reference b/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.reference new file mode 100644 index 00000000000..7b36cc96f5e --- /dev/null +++ b/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.reference @@ -0,0 +1,20 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.sh b/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.sh new file mode 100755 index 00000000000..ec1edd710a1 --- /dev/null +++ b/tests/queries/0_stateless/02181_detect_output_format_by_file_extension.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_LOCAL -q "select * from numbers(10)" > $CLICKHOUSE_TMP/data.parquet +$CLICKHOUSE_LOCAL -q "select * from table" < $CLICKHOUSE_TMP/data.parquet + +$CLICKHOUSE_CLIENT -q "select * from numbers(10)" > $CLICKHOUSE_TMP/data.parquet +$CLICKHOUSE_LOCAL -q "select * from table" < $CLICKHOUSE_TMP/data.parquet From 2196c75dd8ddaeb1d2f18ca7b05fb4ae37550a4b Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Fri, 8 Mar 2024 11:07:04 -0400 Subject: [PATCH 033/117] Adds substring-UTF8 docs. --- .../functions/string-functions.md | 69 ++++++++++++++++++- 1 file changed, 67 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 3b49e4954ed..f9c3f91a12b 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -588,8 +588,41 @@ Result: ## substringUTF8 -Like `substring` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns the substring of a string `s` which starts at the specified byte index `offset` for Unicode code points. Byte counting starts from `1`. If `offset` is `0`, an empty string is returned. If `offset` is negative, the substring starts `pos` characters from the end of the string, rather than from the beginning. An optional argument `length` specifies the maximum number of bytes the returned substring may have. +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +substringUTF8(s, offset[, length]) +``` + +**Arguments** + +- `s`: The string to calculate a substring from. [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md) or [Enum](../../sql-reference/data-types/enum.md) +- `offset`: The starting position of the substring in `s` . [(U)Int*](../../sql-reference/data-types/int-uint.md). +- `length`: The maximum length of the substring. [(U)Int*](../../sql-reference/data-types/int-uint.md). Optional. + +**Returned value** + +A substring of `s` with `length` many bytes, starting at index `offset`. + +**Implementation details** + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Example** + +```sql +SELECT 'database' AS string, substringUTF8(string, 5), substringUTF8(string, 5, 1) +``` + +```response +┌─string───┬─substringUTF8('database', 5)─┬─substringUTF8('database', 5, 1)─┐ +│ database │ base │ b │ +└──────────┴──────────────────────────────┴─────────────────────────────────┘ +``` ## substringIndex @@ -624,7 +657,39 @@ Result: ## substringIndexUTF8 -Like `substringIndex` but for Unicode code points. Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns the substring of `s` before `count` occurrences of the delimiter `delim`, specifically for Unicode code points. + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +substringIndexUTF8(s, delim, count) +``` + +**Arguments** + +- `s`: The string to extract substring from. [String](../../sql-reference/data-types/string.md). +- `delim`: The character to split. [String](../../sql-reference/data-types/string.md). +- `count`: The number of occurrences of the delimiter to count before extracting the substring. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. If count is negative, everything to the right of the final delimiter (counting from the right) is returned. [UInt or Int](../data-types/int-uint.md) + +**Returned value** + +A substring [String](../../sql-reference/data-types/string.md) of `s` before `count` occurrences of `delim`. + +**Implementation details** + +Assumes that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Example** + +```sql +SELECT substringIndexUTF8('www.clickhouse.com', '.', 2) +``` + +```response +www.clickhouse +``` ## appendTrailingCharIfAbsent From 0336ef3557b0c3c05ef974a4c6aa6771b3aa0757 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Mar 2024 16:14:11 +0100 Subject: [PATCH 034/117] reload CI From 0b63cb237a5a72c96bbc3d4cf52ab70a5d2ad2aa Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Mar 2024 12:05:32 +0100 Subject: [PATCH 035/117] Fix --- src/Storages/System/StorageSystemDisks.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index f67d4f7acd0..eecc889f86b 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -73,9 +73,9 @@ Pipe StorageSystemDisks::read( col_unreserved->insert(disk_ptr->getUnreservedSpace().value_or(std::numeric_limits::max())); col_keep->insert(disk_ptr->getKeepingFreeSpace()); auto data_source_description = disk_ptr->getDataSourceDescription(); - col_type->insert(data_source_description.type); - col_object_storage_type->insert(data_source_description.object_storage_type); - col_metadata_type->insert(data_source_description.metadata_type); + col_type->insert(magic_enum::enum_name(data_source_description.type)); + col_object_storage_type->insert(magic_enum::enum_name(data_source_description.object_storage_type)); + col_metadata_type->insert(magic_enum::enum_name(data_source_description.metadata_type)); col_is_encrypted->insert(data_source_description.is_encrypted); col_is_read_only->insert(disk_ptr->isReadOnly()); col_is_write_once->insert(disk_ptr->isWriteOnce()); From b354d07b829a94a43cf6f3867585efac389088c1 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 11 Mar 2024 14:59:22 +0800 Subject: [PATCH 036/117] remove break --- src/Processors/Transforms/FilterTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 0f2509c7510..b3be9246f43 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -337,7 +337,6 @@ void FilterTransform::doTransform(Chunk & chunk) min_size_in_memory = size_in_memory; first_non_constant_column = i; } - break; } } (void)min_size_in_memory; /// Suppress error of clang-analyzer-deadcode.DeadStores From 61c3d917ae82bb6ae14cea3a4a7c36c19f6d3b22 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 11 Mar 2024 10:33:09 +0000 Subject: [PATCH 037/117] Use `boost::algorithm::join` --- src/Common/FunctionDocumentation.cpp | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Common/FunctionDocumentation.cpp b/src/Common/FunctionDocumentation.cpp index 0dc5b48f9d1..7b554539a4f 100644 --- a/src/Common/FunctionDocumentation.cpp +++ b/src/Common/FunctionDocumentation.cpp @@ -1,5 +1,7 @@ #include +#include + namespace DB { @@ -31,15 +33,7 @@ std::string FunctionDocumentation::examplesAsString() const std::string FunctionDocumentation::categoriesAsString() const { - if (categories.empty()) - return ""; - - auto it = categories.begin(); - std::string res = *it; - ++it; - for (; it != categories.end(); ++it) - res += ", " + *it; - return res; + return boost::algorithm::join(categories, ", "); } } From 90b27432a26c0a5204e09ff5ff5f2ae8df3055af Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Mar 2024 12:18:58 +0100 Subject: [PATCH 038/117] Update test.py --- tests/integration/test_backup_restore_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 452a9143067..f3f4837c317 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -130,7 +130,7 @@ def check_system_tables(backup_query_id=None): if disk ] expected_disks = ( - ("default", "local", "", ""), + ("default", "local", "None", "None"), ("disk_s3", "object_storage", "s3", "local"), ("disk_s3_cache", "object_storage", "s3", "local"), ("disk_s3_other_bucket", "object_storage", "s3", "local"), From 57f6263f67dd91e624003199295c840a228947a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 12:31:40 +0100 Subject: [PATCH 039/117] Lock contention fix --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/Cache/FileCache.cpp | 12 +++++++++--- src/Interpreters/Cache/FileCache.h | 2 +- src/Interpreters/Cache/FileCache_fwd.h | 1 + src/Interpreters/Cache/Guards.h | 15 ++++++++++++--- 5 files changed, 24 insertions(+), 7 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c1ac3d08245..ab1a16a3edf 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -476,6 +476,7 @@ The server successfully detected this situation and will download merged part fr M(FileSegmentRemoveMicroseconds, "File segment remove() time") \ M(FileSegmentHolderCompleteMicroseconds, "File segments holder complete() time") \ M(FileSegmentFailToIncreasePriority, "Number of times the priority was not increased due to a high contention on the cache lock") \ + M(FilesystemCacheFailToReserveSpaceBecauseOfLockContention, "Number of times space reservation was skipped due to a high contention on the cache lock") \ M(FilesystemCacheHoldFileSegments, "Filesystem cache file segments count, which were hold") \ M(FilesystemCacheUnusedHoldFileSegments, "Filesystem cache file segments count, which were hold, but not used (because of seek or LIMIT n, etc)") \ \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 9c705ddc27c..5650b9ce44e 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -27,6 +27,7 @@ namespace ProfileEvents extern const Event FilesystemCacheReserveMicroseconds; extern const Event FilesystemCacheGetOrSetMicroseconds; extern const Event FilesystemCacheGetMicroseconds; + extern const Event FilesystemCacheFailToReserveSpaceBecauseOfLockContention; } namespace DB @@ -188,9 +189,9 @@ CacheGuard::Lock FileCache::lockCache() const return cache_guard.lock(); } -CacheGuard::Lock FileCache::tryLockCache() const +CacheGuard::Lock FileCache::tryLockCache(std::optional acquire_timeout) const { - return cache_guard.tryLock(); + return acquire_timeout.has_value() ? cache_guard.tryLockFor(acquire_timeout.value()) : cache_guard.tryLock(); } FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment::Range & range, size_t file_segments_limit) const @@ -781,7 +782,12 @@ bool FileCache::tryReserve( ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheReserveMicroseconds); assertInitialized(); - auto cache_lock = lockCache(); + auto cache_lock = tryLockCache(std::chrono::milliseconds(FILECACHE_TRY_RESERVE_LOCK_TIMEOUT_MILLISECONDS)); + if (!cache_lock) + { + ProfileEvents::increment(ProfileEvents::FilesystemCacheFailToReserveSpaceBecauseOfLockContention); + return false; + } LOG_TEST( log, "Trying to reserve space ({} bytes) for {}:{}, current usage {}/{}", diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 5b665ad0271..7434b2ac78a 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -173,7 +173,7 @@ public: void deactivateBackgroundOperations(); CacheGuard::Lock lockCache() const; - CacheGuard::Lock tryLockCache() const; + CacheGuard::Lock tryLockCache(std::optional acquire_timeout = std::nullopt) const; std::vector sync(); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 06261b19db7..eaed279e7fd 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -12,6 +12,7 @@ static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; +static constexpr size_t FILECACHE_TRY_RESERVE_LOCK_TIMEOUT_MILLISECONDS = 1000; /// 1 sec. class FileCache; using FileCachePtr = std::shared_ptr; diff --git a/src/Interpreters/Cache/Guards.h b/src/Interpreters/Cache/Guards.h index 5729620d82f..0ac7cb80483 100644 --- a/src/Interpreters/Cache/Guards.h +++ b/src/Interpreters/Cache/Guards.h @@ -61,17 +61,26 @@ namespace DB */ struct CacheGuard : private boost::noncopyable { + using Mutex = std::timed_mutex; /// struct is used (not keyword `using`) to make CacheGuard::Lock non-interchangable with other guards locks /// so, we wouldn't be able to pass CacheGuard::Lock to a function which accepts KeyGuard::Lock, for example - struct Lock : public std::unique_lock + struct Lock : public std::unique_lock { - using Base = std::unique_lock; + using Base = std::unique_lock; using Base::Base; }; Lock lock() { return Lock(mutex); } + Lock tryLock() { return Lock(mutex, std::try_to_lock); } - std::mutex mutex; + + Lock tryLockFor(const std::chrono::milliseconds & acquire_timeout) + { + return Lock(mutex, std::chrono::duration(acquire_timeout)); + } + +private: + Mutex mutex; }; /** From 19d8256fa83a4e8353dcad372067085ec8f0828d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Mar 2024 14:44:19 +0100 Subject: [PATCH 040/117] Update test.py --- tests/integration/test_backup_restore_s3/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index f3f4837c317..d65fc1f09d6 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -130,11 +130,11 @@ def check_system_tables(backup_query_id=None): if disk ] expected_disks = ( - ("default", "local", "None", "None"), - ("disk_s3", "object_storage", "s3", "local"), - ("disk_s3_cache", "object_storage", "s3", "local"), - ("disk_s3_other_bucket", "object_storage", "s3", "local"), - ("disk_s3_plain", "object_storage", "s3", "plain"), + ("default", "Local", "None", "None"), + ("disk_s3", "ObjectStorage", "S3", "Local"), + ("disk_s3_cache", "ObjectStorage", "S3", "Local"), + ("disk_s3_other_bucket", "ObjectStorage", "S3", "Local"), + ("disk_s3_plain", "ObjectStorage", "S3", "Plain"), ) assert len(expected_disks) == len(disks) for expected_disk in expected_disks: From 9bada70f45654495a30e394d94a374a862c24fb5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 14:52:32 +0100 Subject: [PATCH 041/117] Remove a bunch of transitive dependencies --- src/Backups/BackupCoordinationRemote.cpp | 2 ++ src/Formats/ReadSchemaUtils.cpp | 9 +++++---- src/Interpreters/DatabaseCatalog.h | 7 +++---- src/Interpreters/GraceHashJoin.cpp | 15 ++++++--------- src/Interpreters/TemporaryDataOnDisk.h | 6 +++--- src/Planner/PlannerExpressionAnalysis.cpp | 2 ++ src/Processors/QueryPlan/AggregatingStep.cpp | 1 + src/Processors/QueryPlan/CubeStep.cpp | 7 ++++--- src/Processors/QueryPlan/SortingStep.cpp | 1 + 9 files changed, 27 insertions(+), 23 deletions(-) diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 9c509858b2a..ec652f20069 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -14,6 +14,8 @@ #include #include +#include + namespace DB { diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 5badf4301bf..b05b768899b 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -1,10 +1,11 @@ #include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include namespace DB { diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 4fe114cc493..6f05a3cea0f 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -1,15 +1,14 @@ #pragma once #include +#include +#include #include #include -#include #include #include -#include "Common/NamePrompter.h" +#include #include -#include "Storages/IStorage.h" -#include "Databases/IDatabase.h" #include #include diff --git a/src/Interpreters/GraceHashJoin.cpp b/src/Interpreters/GraceHashJoin.cpp index 5fb92a68a29..53d1f48c291 100644 --- a/src/Interpreters/GraceHashJoin.cpp +++ b/src/Interpreters/GraceHashJoin.cpp @@ -1,21 +1,18 @@ +#include +#include +#include +#include #include #include #include - -#include #include - -#include +#include #include #include #include -#include -#include - -#include - #include +#include namespace CurrentMetrics diff --git a/src/Interpreters/TemporaryDataOnDisk.h b/src/Interpreters/TemporaryDataOnDisk.h index e57d9130369..8b0649be1b1 100644 --- a/src/Interpreters/TemporaryDataOnDisk.h +++ b/src/Interpreters/TemporaryDataOnDisk.h @@ -2,11 +2,11 @@ #include -#include -#include +#include #include -#include +#include #include +#include namespace CurrentMetrics diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 52001eb27c5..30d90a68072 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index f374a7b7b10..a76bacdd97b 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 0c632c346c7..d010a3327a6 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -1,10 +1,11 @@ +#include +#include +#include +#include #include #include #include -#include #include -#include -#include namespace DB { diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 641b9036d4c..d0491cb4b82 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include From 8b5ccb4735365ef81af4debcc3180f296452268d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 15:53:46 +0100 Subject: [PATCH 042/117] Remove one template --- src/Interpreters/Aggregator.cpp | 65 +++++++++++++-------------------- src/Interpreters/Aggregator.h | 4 +- 2 files changed, 28 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 80a98683867..69625dbd57d 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2609,8 +2609,9 @@ void NO_INLINE Aggregator::mergeDataNullKey( } } -template -void NO_INLINE Aggregator::mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena) const +template +void NO_INLINE +Aggregator::mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions [[maybe_unused]]) const { if constexpr (Method::low_cardinality_optimization || Method::one_key_nullable_optimization) mergeDataNullKey(table_dst, table_src, arena); @@ -2637,7 +2638,7 @@ void NO_INLINE Aggregator::mergeDataImpl(Table & table_dst, Table & table_src, A table_src.clearAndShrink(); #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) + if (use_compiled_functions) { const auto & compiled_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; compiled_functions.merge_aggregate_states_function(dst_places.data(), src_places.data(), dst_places.size()); @@ -2787,26 +2788,16 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl( if (!no_more_keys) { + bool use_compiled_functions = false; #if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - if (prefetch) - mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool); - else - mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool); - } - else + use_compiled_functions = compiled_aggregate_functions_holder != nullptr; #endif - { - if (prefetch) - mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool); - else - mergeDataImpl( - getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool); - } + if (prefetch) + mergeDataImpl( + getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, use_compiled_functions); + else + mergeDataImpl( + getDataVariant(*res).data, getDataVariant(current).data, res->aggregates_pool, use_compiled_functions); } else if (res->without_key) { @@ -2851,26 +2842,22 @@ void NO_INLINE Aggregator::mergeBucketImpl( return; AggregatedDataVariants & current = *data[result_num]; + bool use_compiled_functions = false; #if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - if (prefetch) - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena); - else - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena); - } - else + use_compiled_functions = compiled_aggregate_functions_holder != nullptr; #endif - { - if (prefetch) - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena); - else - mergeDataImpl( - getDataVariant(*res).data.impls[bucket], getDataVariant(current).data.impls[bucket], arena); - } + if (prefetch) + mergeDataImpl( + getDataVariant(*res).data.impls[bucket], + getDataVariant(current).data.impls[bucket], + arena, + use_compiled_functions); + else + mergeDataImpl( + getDataVariant(*res).data.impls[bucket], + getDataVariant(current).data.impls[bucket], + arena, + use_compiled_functions); } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 375b8986101..67e82cdd784 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1429,8 +1429,8 @@ private: Arena * arena) const; /// Merge data from hash table `src` into `dst`. - template - void mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena) const; + template + void mergeDataImpl(Table & table_dst, Table & table_src, Arena * arena, bool use_compiled_functions) const; /// Merge data from hash table `src` into `dst`, but only for keys that already exist in dst. In other cases, merge the data into `overflows`. template From 724cc903afb9283a8369a62a836d04eceae42e57 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Mar 2024 15:56:02 +0100 Subject: [PATCH 043/117] Restart CI --- tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh b/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh index 1bf21dfc53b..6cd5c3b486c 100755 --- a/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh +++ b/tests/queries/0_stateless/03001_parallel_parsing_deadlock.sh @@ -9,4 +9,3 @@ DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.csv $CLICKHOUSE_LOCAL -q "select number > 1000000 ? 'error' : toString(number) from numbers(2000000) format CSV" > $DATA_FILE $CLICKHOUSE_LOCAL -q "select * from file($DATA_FILE, CSV, 'x UInt64') format Null settings input_format_allow_errors_ratio=1" rm $DATA_FILE - From ecc30448baa1c6283f3f0f13c83cfd1bf4428b9b Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Mar 2024 15:26:29 +0000 Subject: [PATCH 044/117] Fix filtering when optimize_use_implicit_projections=1 --- .../optimizeUseAggregateProjection.cpp | 4 ++ src/Storages/VirtualColumnUtils.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 3 + ...ions_non_deterministoc_functions.reference | 55 +++++++++++++++++++ ...rojections_non_deterministoc_functions.sql | 28 ++++++++++ 5 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.reference create mode 100644 tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.sql diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 91f4213ff43..b40fea47b3c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include #include @@ -464,6 +465,9 @@ AggregateProjectionCandidates getAggregateProjectionCandidates( // LOG_TRACE(getLogger("optimizeUseProjections"), "Query DAG: {}", dag.dag->dumpDAG()); candidates.has_filter = dag.filter_node; + /// We can't use minmax projection if filter has non-deterministic functions. + if (dag.filter_node && !VirtualColumnUtils::isDeterministicInScopeOfQuery(dag.filter_node)) + can_use_minmax_projection = false; if (can_use_minmax_projection) { diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 897090223d6..c3ac27903c9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -238,7 +238,7 @@ static bool canEvaluateSubtree(const ActionsDAG::Node * node, const Block & allo return true; } -static bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) +bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node) { for (const auto * child : node->children) { diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index b5526fc5c7f..83494872cac 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -25,6 +25,9 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block, /// Just filters block. Block should contain all the required columns. void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context); +/// Recursively checks if all functions used in DAG are deterministic in scope of query. +bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node); + /// Extract a part of predicate that can be evaluated using only columns from input_names. ActionsDAGPtr splitFilterDagForAllowedInputs(const ActionsDAG::Node * predicate, const Block * allowed_inputs); diff --git a/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.reference b/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.reference new file mode 100644 index 00000000000..8233925d609 --- /dev/null +++ b/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.reference @@ -0,0 +1,55 @@ +-- count +100000 all_10_10_0 +100000 all_1_1_0 +100000 all_2_2_0 +100000 all_3_3_0 +100000 all_4_4_0 +100000 all_5_5_0 +100000 all_6_6_0 +100000 all_7_7_0 +100000 all_8_8_0 +100000 all_9_9_0 +-- rand()%2=0: +1 all_10_10_0 +1 all_1_1_0 +1 all_2_2_0 +1 all_3_3_0 +1 all_4_4_0 +1 all_5_5_0 +1 all_6_6_0 +1 all_7_7_0 +1 all_8_8_0 +1 all_9_9_0 +-- optimize_use_implicit_projections=0 +1 all_10_10_0 +1 all_1_1_0 +1 all_2_2_0 +1 all_3_3_0 +1 all_4_4_0 +1 all_5_5_0 +1 all_6_6_0 +1 all_7_7_0 +1 all_8_8_0 +1 all_9_9_0 +-- optimize_trivial_count_query=0 +1 all_10_10_0 +1 all_1_1_0 +1 all_2_2_0 +1 all_3_3_0 +1 all_4_4_0 +1 all_5_5_0 +1 all_6_6_0 +1 all_7_7_0 +1 all_8_8_0 +1 all_9_9_0 +-- optimize_trivial_count_query=0, optimize_use_implicit_projections=0 +1 all_10_10_0 +1 all_1_1_0 +1 all_2_2_0 +1 all_3_3_0 +1 all_4_4_0 +1 all_5_5_0 +1 all_6_6_0 +1 all_7_7_0 +1 all_8_8_0 +1 all_9_9_0 diff --git a/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.sql b/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.sql new file mode 100644 index 00000000000..3be9bc3982a --- /dev/null +++ b/tests/queries/0_stateless/03008_filter_projections_non_deterministoc_functions.sql @@ -0,0 +1,28 @@ +create table test (number UInt64) engine=MergeTree order by number; +system stop merges test; +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); +INSERT INTO test select number from numbers(100000); + +select '-- count'; +SELECT count(), _part FROM test GROUP BY _part ORDER BY _part; + +select '-- rand()%2=0:'; +SELECT count() > 0 AND count() < 100000, _part FROM test WHERE rand(1)%2=1 GROUP BY _part ORDER BY _part; + +select '-- optimize_use_implicit_projections=0'; +SELECT count() > 0 AND count() < 100000, _part FROM test WHERE rand(2)%2=1 GROUP BY _part ORDER BY _part settings optimize_use_implicit_projections=0; + +select '-- optimize_trivial_count_query=0'; +SELECT count() > 0 AND count() < 100000, _part FROM test WHERE rand(3)%2=1 GROUP BY _part ORDER BY _part settings optimize_trivial_count_query=0; + +select '-- optimize_trivial_count_query=0, optimize_use_implicit_projections=0'; +SELECT count() > 0 AND count() < 100000, _part FROM test WHERE rand(4)%2=1 GROUP BY _part ORDER BY _part settings optimize_trivial_count_query=0,optimize_use_implicit_projections=0; + From 879f7f2f8c862aae51ddc5a8faebb8d07b5d4493 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 16:28:25 +0100 Subject: [PATCH 045/117] Remove more templates for JIT --- src/Interpreters/Aggregator.cpp | 185 ++++++++++++++------------------ src/Interpreters/Aggregator.h | 21 ++-- 2 files changed, 96 insertions(+), 110 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 69625dbd57d..c7ce3e46446 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1111,6 +1111,7 @@ void NO_INLINE Aggregator::executeImpl( bool all_keys_are_const, AggregateDataPtr overflow_row) const { + bool use_compiled_functions = false; if (!no_more_keys) { /// Prefetching doesn't make sense for small hash tables, because they fit in caches entirely. @@ -1118,33 +1119,47 @@ void NO_INLINE Aggregator::executeImpl( && (method.data.getBufferSizeInBytes() > min_bytes_for_prefetch); #if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions)) - { - if (prefetch) - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); - else - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); - } - else + use_compiled_functions = compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions); #endif - { - if (prefetch) - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); - else - executeImplBatch( - method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); - } + if (prefetch) + executeImplBatch( + method, + state, + aggregates_pool, + row_begin, + row_end, + aggregate_instructions, + all_keys_are_const, + use_compiled_functions, + overflow_row); + else + executeImplBatch( + method, + state, + aggregates_pool, + row_begin, + row_end, + aggregate_instructions, + all_keys_are_const, + use_compiled_functions, + overflow_row); } else { - executeImplBatch(method, state, aggregates_pool, row_begin, row_end, aggregate_instructions, all_keys_are_const, overflow_row); + executeImplBatch( + method, + state, + aggregates_pool, + row_begin, + row_end, + aggregate_instructions, + all_keys_are_const, + use_compiled_functions, + overflow_row); } } -template +template void NO_INLINE Aggregator::executeImplBatch( Method & method, State & state, @@ -1153,6 +1168,7 @@ void NO_INLINE Aggregator::executeImplBatch( size_t row_end, AggregateFunctionInstruction * aggregate_instructions, bool all_keys_are_const, + bool use_compiled_functions [[maybe_unused]], AggregateDataPtr overflow_row) const { using KeyHolder = decltype(state.getKeyHolder(0, std::declval())); @@ -1284,7 +1300,7 @@ void NO_INLINE Aggregator::executeImplBatch( aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) + if (use_compiled_functions) { const auto & compiled_aggregate_functions = compiled_aggregate_functions_holder->compiled_aggregate_functions; compiled_aggregate_functions.create_aggregate_states_function(aggregate_data); @@ -1293,20 +1309,6 @@ void NO_INLINE Aggregator::executeImplBatch( static constexpr bool skip_compiled_aggregate_functions = true; createAggregateStates(aggregate_data); } - -#if defined(MEMORY_SANITIZER) - - /// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place. - for (size_t aggregate_function_index = 0; aggregate_function_index < aggregate_functions.size(); ++aggregate_function_index) - { - if (!is_aggregate_function_compiled[aggregate_function_index]) - continue; - - auto aggregate_data_with_offset = aggregate_data + offsets_of_aggregate_states[aggregate_function_index]; - auto data_size = params.aggregates[aggregate_function_index].function->sizeOfData(); - __msan_unpoison(aggregate_data_with_offset, data_size); - } -#endif } else #endif @@ -1339,7 +1341,7 @@ void NO_INLINE Aggregator::executeImplBatch( } #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) + if (use_compiled_functions) { std::vector columns_data; @@ -1372,9 +1374,8 @@ void NO_INLINE Aggregator::executeImplBatch( for (size_t i = 0; i < aggregate_functions.size(); ++i) { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - if (is_aggregate_function_compiled[i]) - continue; + if (use_compiled_functions && is_aggregate_function_compiled[i]) + continue; #endif AggregateFunctionInstruction * inst = aggregate_instructions + i; @@ -1387,18 +1388,19 @@ void NO_INLINE Aggregator::executeImplBatch( } -template void NO_INLINE Aggregator::executeWithoutKeyImpl( AggregatedDataWithoutKey & res, - size_t row_begin, size_t row_end, + size_t row_begin, + size_t row_end, AggregateFunctionInstruction * aggregate_instructions, - Arena * arena) const + Arena * arena, + bool use_compiled_functions [[maybe_unused]]) const { if (row_begin == row_end) return; #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) + if (use_compiled_functions) { std::vector columns_data; @@ -1418,20 +1420,6 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( auto add_into_aggregate_states_function_single_place = compiled_aggregate_functions_holder->compiled_aggregate_functions.add_into_aggregate_states_function_single_place; add_into_aggregate_states_function_single_place(row_begin, row_end, columns_data.data(), res); - -#if defined(MEMORY_SANITIZER) - - /// We compile only functions that do not allocate some data in Arena. Only store necessary state in AggregateData place. - for (size_t aggregate_function_index = 0; aggregate_function_index < aggregate_functions.size(); ++aggregate_function_index) - { - if (!is_aggregate_function_compiled[aggregate_function_index]) - continue; - - auto aggregate_data_with_offset = res + offsets_of_aggregate_states[aggregate_function_index]; - auto data_size = params.aggregates[aggregate_function_index].function->sizeOfData(); - __msan_unpoison(aggregate_data_with_offset, data_size); - } -#endif } #endif @@ -1439,13 +1427,10 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( for (size_t i = 0; i < aggregate_functions.size(); ++i) { AggregateFunctionInstruction * inst = aggregate_instructions + i; - #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) - if (is_aggregate_function_compiled[i]) - continue; + if (use_compiled_functions && is_aggregate_function_compiled[i]) + continue; #endif - addBatchSinglePlace(row_begin, row_end, inst, res + inst->state_offset, arena); } } @@ -1704,16 +1689,14 @@ bool Aggregator::executeOnBlock(Columns columns, if (result.type == AggregatedDataVariants::Type::without_key) { /// TODO: Enable compilation after investigation -// #if USE_EMBEDDED_COMPILER -// if (compiled_aggregate_functions_holder) -// { -// executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); -// } -// else -// #endif - { - executeWithoutKeyImpl(result.without_key, row_begin, row_end, aggregate_functions_instructions.data(), result.aggregates_pool); - } + bool use_compiled_functions = false; + executeWithoutKeyImpl( + result.without_key, + row_begin, + row_end, + aggregate_functions_instructions.data(), + result.aggregates_pool, + use_compiled_functions); } else { @@ -1965,19 +1948,13 @@ Aggregator::convertToBlockImpl(Method & method, Table & data, Arena * arena, Are ConvertToBlockRes res; + bool use_compiled_functions = false; if (final) { #if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) - { - static constexpr bool use_compiled_functions = !Method::low_cardinality_optimization; - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); - } - else + use_compiled_functions = compiled_aggregate_functions_holder != nullptr && !Method::low_cardinality_optimization; #endif - { - res = convertToBlockImplFinal(method, data, arena, aggregates_pools, rows); - } + res = convertToBlockImplFinal(method, data, arena, aggregates_pools, use_compiled_functions, rows); } else { @@ -2059,8 +2036,12 @@ inline void Aggregator::insertAggregatesIntoColumns(Mapped & mapped, MutableColu } -template -Block Aggregator::insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool has_null_key_data [[maybe_unused]]) const +Block Aggregator::insertResultsIntoColumns( + PaddedPODArray & places, + OutputBlockColumns && out_cols, + Arena * arena, + bool has_null_key_data [[maybe_unused]], + bool use_compiled_functions [[maybe_unused]]) const { std::exception_ptr exception; size_t aggregate_functions_destroy_index = 0; @@ -2068,7 +2049,7 @@ Block Aggregator::insertResultsIntoColumns(PaddedPODArray & pl try { #if USE_EMBEDDED_COMPILER - if constexpr (use_compiled_functions) + if (use_compiled_functions) { /** For JIT compiled functions we need to resize columns before pass them into compiled code. * insert_aggregates_into_columns_function function does not throw exception. @@ -2098,14 +2079,13 @@ Block Aggregator::insertResultsIntoColumns(PaddedPODArray & pl for (; aggregate_functions_destroy_index < params.aggregates_size;) { - if constexpr (use_compiled_functions) +#if USE_EMBEDDED_COMPILER + if (use_compiled_functions && is_aggregate_function_compiled[aggregate_functions_destroy_index]) { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } + ++aggregate_functions_destroy_index; + continue; } +#endif auto & final_aggregate_column = out_cols.final_aggregate_columns[aggregate_functions_destroy_index]; size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; @@ -2127,14 +2107,13 @@ Block Aggregator::insertResultsIntoColumns(PaddedPODArray & pl for (; aggregate_functions_destroy_index < params.aggregates_size; ++aggregate_functions_destroy_index) { - if constexpr (use_compiled_functions) +#if USE_EMBEDDED_COMPILER + if (use_compiled_functions && is_aggregate_function_compiled[aggregate_functions_destroy_index]) { - if (is_aggregate_function_compiled[aggregate_functions_destroy_index]) - { - ++aggregate_functions_destroy_index; - continue; - } + ++aggregate_functions_destroy_index; + continue; } +#endif size_t offset = offsets_of_aggregate_states[aggregate_functions_destroy_index]; aggregate_functions[aggregate_functions_destroy_index]->destroyBatch(0, places.size(), places.data(), offset); @@ -2146,9 +2125,9 @@ Block Aggregator::insertResultsIntoColumns(PaddedPODArray & pl return finalizeBlock(params, getHeader(/* final */ true), std::move(out_cols), /* final */ true, places.size()); } -template -Aggregator::ConvertToBlockRes NO_INLINE -Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t) const +template +Aggregator::ConvertToBlockRes NO_INLINE Aggregator::convertToBlockImplFinal( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool use_compiled_functions [[maybe_unused]], size_t) const { /// +1 for nullKeyData, if `data` doesn't have it - not a problem, just some memory for one excessive row will be preallocated const size_t max_block_size = (return_single_block ? data.size() : std::min(params.max_block_size, data.size())) + 1; @@ -2204,7 +2183,8 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena { if (places.size() >= max_block_size) { - res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data)); + res.emplace_back( + insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data, use_compiled_functions)); places.clear(); out_cols.reset(); has_null_key_data = false; @@ -2214,12 +2194,13 @@ Aggregator::convertToBlockImplFinal(Method & method, Table & data, Arena * arena if constexpr (return_single_block) { - return insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data); + return insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data, use_compiled_functions); } else { if (out_cols.has_value()) - res.emplace_back(insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data)); + res.emplace_back( + insertResultsIntoColumns(places, std::move(out_cols.value()), arena, has_null_key_data, use_compiled_functions)); return res; } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 67e82cdd784..d7bbe5950a0 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1395,7 +1395,7 @@ private: AggregateDataPtr overflow_row) const; /// Specialization for a particular value no_more_keys. - template + template void executeImplBatch( Method & method, State & state, @@ -1404,16 +1404,17 @@ private: size_t row_end, AggregateFunctionInstruction * aggregate_instructions, bool all_keys_are_const, + bool use_compiled_functions, AggregateDataPtr overflow_row) const; /// For case when there are no keys (all aggregate into one row). - template void executeWithoutKeyImpl( AggregatedDataWithoutKey & res, size_t row_begin, size_t row_end, AggregateFunctionInstruction * aggregate_instructions, - Arena * arena) const; + Arena * arena, + bool use_compiled_functions) const; template void writeToTemporaryFileImpl( @@ -1467,12 +1468,16 @@ private: MutableColumns & final_aggregate_columns, Arena * arena) const; - template - Block insertResultsIntoColumns(PaddedPODArray & places, OutputBlockColumns && out_cols, Arena * arena, bool has_null_key_data) const; + Block insertResultsIntoColumns( + PaddedPODArray & places, + OutputBlockColumns && out_cols, + Arena * arena, + bool has_null_key_data, + bool use_compiled_functions) const; - template - ConvertToBlockRes - convertToBlockImplFinal(Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, size_t rows) const; + template + ConvertToBlockRes convertToBlockImplFinal( + Method & method, Table & data, Arena * arena, Arenas & aggregates_pools, bool use_compiled_functions, size_t rows) const; template ConvertToBlockRes From 38f41ee311d0a36d194965e5815489a25c60e449 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 16:55:30 +0100 Subject: [PATCH 046/117] Fix integration test --- tests/integration/test_disk_types/test.py | 10 +++++----- .../test_endpoint_macro_substitution/test.py | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index af482b97be3..86579911b3e 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -3,10 +3,10 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV disk_types = { - "default": "local", - "disk_s3": "s3", - "disk_hdfs": "hdfs", - "disk_encrypted": "s3", + "default": "Local", + "disk_s3": "S3", + "disk_hdfs": "HDFS", + "disk_encrypted": "S3", } @@ -55,7 +55,7 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] for name, disk_type in list(disk_types.items()): - if disk_type != "s3": + if disk_type != "S3": assert ( node.query( "SELECT name FROM system.disks WHERE type='" + disk_type + "'" diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index 42a8ddbda84..bec3d9de0e3 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -4,10 +4,10 @@ from helpers.test_tools import TSV from pyhdfs import HdfsClient disk_types = { - "default": "local", - "disk_s3": "s3", - "disk_hdfs": "hdfs", - "disk_encrypted": "s3", + "default": "Local", + "disk_s3": "S3", + "disk_hdfs": "HDFS", + "disk_encrypted": "S3", } @@ -63,7 +63,7 @@ def test_select_by_type(cluster): fs = HdfsClient(hosts=cluster.hdfs_ip) for name, disk_type in list(disk_types.items()): - if disk_type != "s3": + if disk_type != "S3": assert ( node.query( "SELECT name FROM system.disks WHERE type='" + disk_type + "'" From 5db08292455fb0c6f47fc0344382ab7cf3508e91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 17:20:53 +0100 Subject: [PATCH 047/117] Remove another template --- src/Interpreters/Aggregator.cpp | 55 ++++++++++++++------------------- src/Interpreters/Aggregator.h | 3 +- 2 files changed, 26 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c7ce3e46446..a9578b5540f 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -2906,11 +2906,12 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData return non_empty_data; } -template +template void NO_INLINE Aggregator::mergeStreamsImplCase( Arena * aggregates_pool, State & state, Table & data, + bool no_more_keys, AggregateDataPtr overflow_row, size_t row_begin, size_t row_end, @@ -2922,36 +2923,34 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( if (!arena_for_keys) arena_for_keys = aggregates_pool; - for (size_t i = row_begin; i < row_end; ++i) + if (no_more_keys) { - AggregateDataPtr aggregate_data = nullptr; - - if constexpr (!no_more_keys) + for (size_t i = row_begin; i < row_end; i++) { - auto emplace_result = state.emplaceKey(data, i, *arena_for_keys); // NOLINT - if (emplace_result.isInserted()) + auto find_result = state.findKey(data, i, *arena_for_keys); + /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. + AggregateDataPtr value = find_result.isFound() ? find_result.getMapped() : overflow_row; + places[i] = value; + } + } + else + { + for (size_t i = row_begin; i < row_end; i++) + { + auto emplace_result = state.emplaceKey(data, i, *arena_for_keys); + if (!emplace_result.isInserted()) + places[i] = emplace_result.getMapped(); + else { emplace_result.setMapped(nullptr); - aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + AggregateDataPtr aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); createAggregateStates(aggregate_data); emplace_result.setMapped(aggregate_data); + places[i] = aggregate_data; } - else - aggregate_data = emplace_result.getMapped(); } - else - { - auto find_result = state.findKey(data, i, *arena_for_keys); - if (find_result.isFound()) - aggregate_data = find_result.getMapped(); - } - - /// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys. - - AggregateDataPtr value = aggregate_data ? aggregate_data : overflow_row; - places[i] = value; } for (size_t j = 0; j < params.aggregates_size; ++j) @@ -3005,22 +3004,16 @@ void NO_INLINE Aggregator::mergeStreamsImpl( if (use_cache) { typename Method::State state(key_columns, key_sizes, aggregation_state_cache); - - if (!no_more_keys) - mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); - else - mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + mergeStreamsImplCase( + aggregates_pool, state, data, no_more_keys, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); consecutive_keys_cache_stats.update(row_end - row_begin, state.getCacheMissesSinceLastReset()); } else { typename Method::StateNoCache state(key_columns, key_sizes, aggregation_state_cache); - - if (!no_more_keys) - mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); - else - mergeStreamsImplCase(aggregates_pool, state, data, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); + mergeStreamsImplCase( + aggregates_pool, state, data, no_more_keys, overflow_row, row_begin, row_end, aggregate_columns_data, arena_for_keys); } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index d7bbe5950a0..6c357623003 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1513,11 +1513,12 @@ private: bool final, ThreadPool * thread_pool) const; - template + template void mergeStreamsImplCase( Arena * aggregates_pool, State & state, Table & data, + bool no_more_keys, AggregateDataPtr overflow_row, size_t row_begin, size_t row_end, From 16e01eb93ad449c61417dcaccd570439364b0714 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Mar 2024 18:05:51 +0100 Subject: [PATCH 048/117] Fix style --- src/Core/Settings.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index fb456b46d89..8257b94cd9f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int THERE_IS_NO_PROFILE; extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int BAD_ARGUMENTS; } IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) From a1e5161cee50650a5c4e87ca60e7ed9eb61451b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 19:25:34 +0100 Subject: [PATCH 049/117] Disable sanitizers with 02784_parallel_replicas_automatic_decision_join --- .../02784_parallel_replicas_automatic_decision_join.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index ef3e6000903..801cd22b79f 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -## Note: The analyzer doesn't support JOIN with parallel replicas yet +# Tags: no-tsan, no-asan, no-msan +# It's not clear why distributed aggregation is much slower with sanitizers (https://github.com/ClickHouse/ClickHouse/issues/60625) CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 55a82047613c607dedb592fed019d04455e8c8e8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Mar 2024 19:43:30 +0100 Subject: [PATCH 050/117] Fix test --- .../0_stateless/03003_compatibility_setting_bad_value.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql index 9a6f4e7944a..48e98798c51 100644 --- a/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql +++ b/tests/queries/0_stateless/03003_compatibility_setting_bad_value.sql @@ -1,2 +1,2 @@ -select 42 settings compatibility=NULL; -- {clientError BAD_GET} +select 42 settings compatibility=NULL; -- {clientError BAD_ARGUMENTS} From 5a71636411cb358c94e58b7caac18c22104b0e1c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 19:44:52 +0100 Subject: [PATCH 051/117] Fxi --- tests/integration/test_disk_types/test.py | 30 +++++++++++++++++++ .../test_endpoint_macro_substitution/test.py | 9 +++++- 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 86579911b3e..5047cdc605e 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -50,6 +50,36 @@ def test_different_types(cluster): assert ( fields[encrypted_col_ix] == "0" ), f"{fields[name_col_ix]} expected to be non-encrypted!" +def test_different_types(cluster): + node = cluster.instances["node"] + response = TSV.toMat(node.query("SELECT * FROM system.disks FORMAT TSVWithNames")) + + assert len(response) > len(disk_types) # at least one extra line for header + + name_col_ix = response[0].index("name") + type_col_ix = response[0].index("type") + encrypted_col_ix = response[0].index("is_encrypted") + + for fields in response[1:]: # skip header + assert len(fields) >= 7 + expected_disk_type = disk_types.get(fields[name_col_ix], "UNKNOWN") + + if expected_disk_type != "Local": + disk_type = fields[response[0].index("object_storage_type")] + else: + disk_type = fields[type_col_ix] + + assert ( + expected_disk_type == disk_type + ), f"Wrong type ({fields[type_col_ix]}) for disk {fields[name_col_ix]}!" + if "encrypted" in fields[name_col_ix]: + assert ( + fields[encrypted_col_ix] == "1" + ), f"{fields[name_col_ix]} expected to be encrypted!" + else: + assert ( + fields[encrypted_col_ix] == "0" + ), f"{fields[name_col_ix]} expected to be non-encrypted!" def test_select_by_type(cluster): diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index bec3d9de0e3..e161d8e82ff 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -45,8 +45,15 @@ def test_different_types(cluster): for fields in response[1:]: # skip header assert len(fields) >= 7 + expected_disk_type = disk_types.get(fields[name_col_ix], "UNKNOWN") + + if expected_disk_type != "Local": + disk_type = fields[response[0].index("object_storage_type")] + else: + disk_type = fields[type_col_ix] + assert ( - disk_types.get(fields[name_col_ix], "UNKNOWN") == fields[type_col_ix] + expected_disk_type == disk_type ), f"Wrong type ({fields[type_col_ix]}) for disk {fields[name_col_ix]}!" if "encrypted" in fields[name_col_ix]: assert ( From 9b055c3a43039387b42e755efddd83b9a8934ca6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 11 Mar 2024 20:38:30 +0100 Subject: [PATCH 052/117] Use assert_cast to prevent nullptr dereference on bad column types in FunctionsConversion --- src/Functions/FunctionsConversion.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 1522e76893e..f338af28240 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -4561,7 +4561,7 @@ arguments, result_type, input_rows_count); \ if (from_low_cardinality) { - const auto * col_low_cardinality = typeid_cast(arguments[0].column.get()); + const auto * col_low_cardinality = assert_cast(arguments[0].column.get()); if (skip_not_null_check && col_low_cardinality->containsNull()) throw Exception(ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN, "Cannot convert NULL value to non-Nullable type"); @@ -4586,7 +4586,7 @@ arguments, result_type, input_rows_count); \ if (to_low_cardinality) { auto res_column = to_low_cardinality->createColumn(); - auto * col_low_cardinality = typeid_cast(res_column.get()); + auto * col_low_cardinality = assert_cast(res_column.get()); if (from_low_cardinality && !src_converted_to_full_column) { From 3a26b9c89ee3083884fde341c2af418bcde2f4cf Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 11 Mar 2024 19:42:25 +0000 Subject: [PATCH 053/117] impl --- .../0_stateless/02887_mutations_subcolumns.reference | 6 +++--- tests/queries/0_stateless/02887_mutations_subcolumns.sql | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02887_mutations_subcolumns.reference b/tests/queries/0_stateless/02887_mutations_subcolumns.reference index c2d6cbbd225..1ccc83b48a3 100644 --- a/tests/queries/0_stateless/02887_mutations_subcolumns.reference +++ b/tests/queries/0_stateless/02887_mutations_subcolumns.reference @@ -5,6 +5,6 @@ 4 ttt 5 ttt 6 ttt -{"a":"1","obj":{"k1":1,"k2":null,"k3":null}} -{"a":"3","obj":{"k1":null,"k2":null,"k3":1}} -{"a":"1","obj":{"k1":1,"k2":null,"k3":null}} +1 [('k1',1)] +3 [('k3',1)] +1 [('k1',1)] diff --git a/tests/queries/0_stateless/02887_mutations_subcolumns.sql b/tests/queries/0_stateless/02887_mutations_subcolumns.sql index a01158e1b06..87b3009e929 100644 --- a/tests/queries/0_stateless/02887_mutations_subcolumns.sql +++ b/tests/queries/0_stateless/02887_mutations_subcolumns.sql @@ -40,9 +40,9 @@ INSERT INTO t_mutations_subcolumns VALUES (2, '{"k2": 1}'); INSERT INTO t_mutations_subcolumns VALUES (3, '{"k3": 1}'); ALTER TABLE t_mutations_subcolumns DELETE WHERE obj.k2 = 1; -SELECT * FROM t_mutations_subcolumns ORDER BY a FORMAT JSONEachRow; +SELECT a, arrayFilter(x -> not isNull(x.2), tupleToNameValuePairs(obj)) FROM t_mutations_subcolumns ORDER BY a; ALTER TABLE t_mutations_subcolumns DELETE WHERE isNull(obj.k1); -SELECT * FROM t_mutations_subcolumns ORDER BY a FORMAT JSONEachRow; +SELECT a, arrayFilter(x -> not isNull(x.2), tupleToNameValuePairs(obj)) FROM t_mutations_subcolumns ORDER BY a; DROP TABLE t_mutations_subcolumns; From 2e74685ba6ea8a3cc32ff0e21d0ee657517ef5a4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 11 Mar 2024 19:58:43 +0000 Subject: [PATCH 054/117] Make variant tests a bit faster --- .../02941_variant_type_2.reference | 80 +++++++++---------- .../0_stateless/02941_variant_type_2.sh | 12 +-- ...different_local_and_global_order.reference | 30 +++---- ...e_with_different_local_and_global_order.sh | 8 +- 4 files changed, 65 insertions(+), 65 deletions(-) diff --git a/tests/queries/0_stateless/02941_variant_type_2.reference b/tests/queries/0_stateless/02941_variant_type_2.reference index 4b6d53c52ac..20a5176cb5e 100644 --- a/tests/queries/0_stateless/02941_variant_type_2.reference +++ b/tests/queries/0_stateless/02941_variant_type_2.reference @@ -1,51 +1,51 @@ Memory test4 insert test4 select -1000000 -200000 -200000 -200000 -200000 -200000 -200000 -200000 +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 MergeTree compact test4 insert test4 select -1000000 -200000 -200000 -200000 -200000 -200000 -200000 -200000 +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 test4 select -1000000 -200000 -200000 -200000 -200000 -200000 -200000 -200000 +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 MergeTree wide test4 insert test4 select -1000000 -200000 -200000 -200000 -200000 -200000 -200000 -200000 +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 test4 select -1000000 -200000 -200000 -200000 -200000 -200000 -200000 -200000 +500000 +100000 +100000 +100000 +100000 +100000 +100000 +100000 diff --git a/tests/queries/0_stateless/02941_variant_type_2.sh b/tests/queries/0_stateless/02941_variant_type_2.sh index 509c537e7fc..d1fa0a777c9 100755 --- a/tests/queries/0_stateless/02941_variant_type_2.sh +++ b/tests/queries/0_stateless/02941_variant_type_2.sh @@ -12,12 +12,12 @@ CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_variant_type=1 --allow_suspic function test4_insert() { echo "test4 insert" - $CH_CLIENT -nmq "insert into test select number, NULL from numbers(200000); -insert into test select number + 200000, number from numbers(200000); -insert into test select number + 400000, 'str_' || toString(number) from numbers(200000); -insert into test select number + 600000, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(200000); -insert into test select number + 800000, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(200000); -insert into test select number + 1000000, range(number % 20 + 1)::Array(UInt64) from numbers(200000);" + $CH_CLIENT -nmq "insert into test select number, NULL from numbers(100000); +insert into test select number + 100000, number from numbers(100000); +insert into test select number + 200000, 'str_' || toString(number) from numbers(100000); +insert into test select number + 300000, ('lc_str_' || toString(number))::LowCardinality(String) from numbers(100000); +insert into test select number + 400000, tuple(number, number + 1)::Tuple(a UInt32, b UInt32) from numbers(100000); +insert into test select number + 500000, range(number % 20 + 1)::Array(UInt64) from numbers(100000);" } function test4_select diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference index 1736a307c42..4109a88997c 100644 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.reference @@ -44,9 +44,9 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -2500000 -750000 -1750000 +500000 +100000 +400000 ----------------------------------------------------------------------------------------------------------- MergeTree compact test1 insert @@ -136,14 +136,14 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -2500000 -750000 -1750000 +500000 +100000 +400000 ----------------------------------------------------------------------------------------------------------- test2 select -2500000 -750000 -1750000 +500000 +100000 +400000 ----------------------------------------------------------------------------------------------------------- MergeTree wide test1 insert @@ -233,12 +233,12 @@ str_38 str_38 \N ----------------------------------------------------------------------------------------------------------- test2 insert test2 select -2500000 -750000 -1750000 +500000 +100000 +400000 ----------------------------------------------------------------------------------------------------------- test2 select -2500000 -750000 -1750000 +500000 +100000 +400000 ----------------------------------------------------------------------------------------------------------- diff --git a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh index 3bb37719a3f..1d88757a5d6 100755 --- a/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh +++ b/tests/queries/0_stateless/02943_variant_type_with_different_local_and_global_order.sh @@ -29,10 +29,10 @@ function test1_select() function test2_insert() { echo "test2 insert" - $CH_CLIENT -q "insert into test select number, number::Variant(UInt64)::Variant(UInt64, Array(UInt64)) from numbers(1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(1000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64)) as res from numbers(2000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" - $CH_CLIENT -q "insert into test select number, if(number < 3500000, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(3000000, 1000000) settings max_insert_block_size = 100000, min_insert_block_size_rows=100000" + $CH_CLIENT -q "insert into test select number, number::Variant(UInt64)::Variant(UInt64, Array(UInt64)) from numbers(200000) settings max_insert_block_size = 10000, min_insert_block_size_rows=10000" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)) as res from numbers(200000, 200000) settings max_insert_block_size = 10000, min_insert_block_size_rows=10000" + $CH_CLIENT -q "insert into test select number, if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64)) as res from numbers(400000, 200000) settings max_insert_block_size = 10000, min_insert_block_size_rows=10000" + $CH_CLIENT -q "insert into test select number, if(number < 3500000, if(number % 2, NULL, number)::Variant(UInt64)::Variant(UInt64, String, Array(UInt64)), if(number % 2, NULL, 'str_' || toString(number))::Variant(String)::Variant(UInt64, String, Array(UInt64))) from numbers(600000, 200000) settings max_insert_block_size = 10000, min_insert_block_size_rows=10000" } function test2_select() From 120a1fdb5f817b442bf659da243407fb7003eaa1 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 11 Mar 2024 17:24:33 -0300 Subject: [PATCH 055/117] Improves varPop docs. Adds varPopStable. --- .../aggregate-functions/reference/varpop.md | 99 +++++++++++++++++-- 1 file changed, 91 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index 751688b0830..5f18bdc30f6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -1,16 +1,99 @@ --- -slug: /en/sql-reference/aggregate-functions/reference/varpop +title: "varPop" +slug: "/en/sql-reference/aggregate-functions/reference/varpop" sidebar_position: 32 --- -# varPop(x) +This page covers the `varPop` and `varPopStable` functions available in ClickHouse. -Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`. +## varPop -In other words, dispersion for a set of values. Returns `Float64`. +Calculates the population covariance between two data columns. The population covariance measures the degree to which two variables vary together. Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`. -Alias: `VAR_POP`. +**Syntax** -:::note -This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `varPopStable` function. It works slower but provides a lower computational error. -::: \ No newline at end of file +```sql +covarPop(x, y) +``` + +**Parameters** + +- `x`: The first data column. [Numeric](../../../native-protocol/columns.md) +- `y`: The second data column. [Numeric](../../../native-protocol/columns.md) + +**Returned value** + +Returns an integer of type `Float64`. + +**Implementation details** + +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable` function](#varPopStable). + +**Example** + +```sql +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data +( + x Int32, + y Int32 +) +ENGINE = Memory; + +INSERT INTO test_data VALUES (1, 2), (2, 3), (3, 5), (4, 6), (5, 8); + +SELECT + covarPop(x, y) AS covar_pop +FROM test_data; +``` + +```response +3 +``` + +## varPopStable + +Calculates population covariance between two data columns using a stable, numerically accurate method to calculate the variance. This function is designed to provide reliable results even with large datasets or values that might cause numerical instability in other implementations. + +**Syntax** + +```sql +covarPopStable(x, y) +``` + +**Parameters** + +- `x`: The first data column. [String literal](../syntax#syntax-string-literal) +- `y`: The second data column. [Expression](../syntax#syntax-expressions) + +**Returned value** + +Returns an integer of type `Float64`. + +**Implementation details** + +Unlike [`varPop()`](#varPop), this function uses a stable, numerically accurate algorithm to calculate the population variance to avoid issues like catastrophic cancellation or loss of precision. This function also handles `NaN` and `Inf` values correctly, excluding them from calculations. + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data +( + x Int32, + y Int32 +) +ENGINE = Memory; + +INSERT INTO test_data VALUES (1, 2), (2, 9), (9, 5), (4, 6), (5, 8); + +SELECT + covarPopStable(x, y) AS covar_pop_stable +FROM test_data; +``` + +```response +0.5999999999999999 +``` From 281dc8d29deba2980e6b191edefa3b62114d38a7 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Mon, 11 Mar 2024 17:48:12 -0300 Subject: [PATCH 056/117] Improves varSamp docs. Adds varSampStable docs. --- .../aggregate-functions/reference/varsamp.md | 126 ++++++++++++++++-- 1 file changed, 118 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index 9b2b94936ec..e75cb075ff8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -1,18 +1,128 @@ --- +title: "varSamp" slug: /en/sql-reference/aggregate-functions/reference/varsamp sidebar_position: 33 --- -# varSamp +This page contains information on the `varSamp` and `varSampStable` ClickHouse functions. -Calculates the amount `Σ((x - x̅)^2) / (n - 1)`, where `n` is the sample size and `x̅`is the average value of `x`. +## varSamp -It represents an unbiased estimate of the variance of a random variable if passed values from its sample. +Calculate the sample variance of a data set. -Returns `Float64`. When `n <= 1`, returns `+∞`. +**Syntax** -Alias: `VAR_SAMP`. +```sql +varSamp(expr) +``` -:::note -This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `varSampStable` function. It works slower but provides a lower computational error. -::: +**Parameters** + +- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../syntax#syntax-expressions) + +**Returned value** + +Returns a Float64 value representing the sample variance of the input data set. + +**Implementation details** + +The `varSamp()` function calculates the sample variance using the following formula: + +```plaintext +∑(x - mean(x))^2 / (n - 1) +``` + +Where: + +- `x` is each individual data point in the data set. +- `mean(x)` is the arithmetic mean of the data set. +- `n` is the number of data points in the data set. + +The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPop()` function](./varpop#varpop) instead. + +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable` function](#varSampStable). + +**Example** + +Query: + +```sql +CREATE TABLE example_table +( + id UInt64, + value Float64 +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7); + +SELECT varSamp(value) FROM example_table; +``` + +Response: + +```response +0.8650000000000091 +``` + +## varSampStable + +Calculate the sample variance of a data set using a numerically stable algorithm. + +**Syntax** + +```sql +varSampStable(expr) +``` + +**Parameters** + +- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../syntax#syntax-expressions) + +**Returned value** + +The `varSampStable()` function returns a Float64 value representing the sample variance of the input data set. + +**Implementation details** + +The `varSampStable()` function calculates the sample variance using the same formula as the [`varSamp()`](#varSamp function): + +```plaintext +∑(x - mean(x))^2 / (n - 1) +``` + +Where: +- `x` is each individual data point in the data set. +- `mean(x)` is the arithmetic mean of the data set. +- `n` is the number of data points in the data set. + +The difference between `varSampStable()` and `varSamp()` is that `varSampStable()` is designed to provide a more deterministic and stable result when dealing with floating-point arithmetic. It uses an algorithm that minimizes the accumulation of rounding errors, which can be particularly important when dealing with large data sets or data with a wide range of values. + +Like `varSamp()`, the `varSampStable()` function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPopStable()` function](./varpop#varpopstable) instead. + +**Example** + +Query: + +```sql +CREATE TABLE example_table +( + id UInt64, + value Float64 +) +ENGINE = MergeTree +ORDER BY id; + +INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7); + +SELECT varSampStable(value) FROM example_table; +``` + +Response: + +```response +0.865 +``` + +This query calculates the sample variance of the `value` column in the `example_table` using the `varSampStable()` function. The result shows that the sample variance of the values `[10.5, 12.3, 9.8, 11.2, 10.7]` is approximately 0.865, which may differ slightly from the result of `varSamp()` due to the more precise handling of floating-point arithmetic. From 563df9bdcb425810a0c2d3ecb11302e22039c048 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 11 Mar 2024 22:49:18 +0100 Subject: [PATCH 057/117] Fix multiple bugs in groupArraySorted --- .../AggregateFunctionGroupArraySorted.cpp | 8 +++++--- .../0_stateless/03008_groupSortedArray_field.reference | 3 +++ .../queries/0_stateless/03008_groupSortedArray_field.sql | 6 ++++++ 3 files changed, 14 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03008_groupSortedArray_field.reference create mode 100644 tests/queries/0_stateless/03008_groupSortedArray_field.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp index 0e9856cfab9..0692ff28f18 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp @@ -310,10 +310,12 @@ public: { for (Field & element : values) { - UInt8 is_null = 0; - readBinary(is_null, buf); - if (!is_null) + bool has_value = 0; + readBinary(has_value, buf); + if (has_value) serialization->deserializeBinary(element, buf, {}); + else + element = Field{}; } } else diff --git a/tests/queries/0_stateless/03008_groupSortedArray_field.reference b/tests/queries/0_stateless/03008_groupSortedArray_field.reference new file mode 100644 index 00000000000..a7f89ebcf58 --- /dev/null +++ b/tests/queries/0_stateless/03008_groupSortedArray_field.reference @@ -0,0 +1,3 @@ +0A01003C79A557B3C43400C4865AA84C3B4B01000650BC18F7DE0B00FAAF43E708213401008ED706EA0A9F13007228F915F5602C0100C692CA8FB81405003A6D357047EB1A01008416B7C3239EE3FF7BE9483CDC61DC01003E133A7C081AF5FFC1ECC583F7E5EA01000000000000000000000000000000000100C4865AA84C3BCBFF3B79A557B3C4B4010024C46EF500F1ECFFDB3B910AFF0ED301005E2FC14EBAEAE5FFA1D03EB14515DA +070109000000010600000001080000000103000000010500000001040000000107000000 AggregateFunction(groupArraySorted(10), Nullable(Decimal(3, 0))) +[3,4,5,6,7,8,9] diff --git a/tests/queries/0_stateless/03008_groupSortedArray_field.sql b/tests/queries/0_stateless/03008_groupSortedArray_field.sql new file mode 100644 index 00000000000..6d2aea641a5 --- /dev/null +++ b/tests/queries/0_stateless/03008_groupSortedArray_field.sql @@ -0,0 +1,6 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/61186 +SELECT hex(CAST(unhex('0A01003C79A557B3C43400C4865AA84C3B4B01000650BC18F7DE0B00FAAF43E708213401008ED706EA0A9F13007228F915F5602C0100C692CA8FB81405003A6D357047EB1A01008416B7C3239EE3FF7BE9483CDC61DC01003E133A7C081AF5FFC1ECC583F7E5EA01000000000000000000000000000000000100C4865AA84C3BCBFF3B79A557B3C4B4010024C46EF500F1ECFFDB3B910AFF0ED301005E2FC14EBAEAE5FFA1D03EB14515DA'), + 'AggregateFunction(groupArraySorted(10), Decimal(38, 38))')); + +Select hex(groupArraySortedState(10)((number < 3 ? NULL : number)::Nullable(Decimal(3))) as t), toTypeName(t) from numbers(10); +Select finalizeAggregation(unhex('070109000000010600000001080000000103000000010500000001040000000107000000')::AggregateFunction(groupArraySorted(10), Nullable(Decimal(3, 0)))); From 1b04cc0b4da6d32fd4741ea953dfed060f846d0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Mar 2024 03:56:10 +0100 Subject: [PATCH 058/117] Fix strange log message --- src/Loggers/Loggers.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Loggers/Loggers.cpp b/src/Loggers/Loggers.cpp index 1d17585cc96..cc6e4691737 100644 --- a/src/Loggers/Loggers.cpp +++ b/src/Loggers/Loggers.cpp @@ -304,6 +304,9 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log log_settings.turn_off_logger = DB::TextLog::shouldTurnOffLogger(); + log_settings.database = config.getString("text_log.database", "system"); + log_settings.table = config.getString("text_log.table", "text_log"); + split->addTextLog(DB::TextLog::getLogQueue(log_settings), text_log_level); } #endif From f6abe68b607dd1893e838b55f85245c716d41f8b Mon Sep 17 00:00:00 2001 From: Pablo Musa Date: Tue, 12 Mar 2024 10:52:22 +0100 Subject: [PATCH 059/117] Clarify sentence about clusterAllReplicas The explanation about `clusterAllReplicas` is not clear. This change tries to simplify the sentence and clarify the explanation. --- docs/en/sql-reference/table-functions/cluster.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index ad92ab39183..136ff72e4a9 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -5,7 +5,7 @@ sidebar_label: cluster title: "cluster, clusterAllReplicas" --- -Allows to access all shards in an existing cluster which configured in `remote_servers` section without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. One replica of each shard is queried. +Allows to access all shards (configured in the `remote_servers` section) of a cluster without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. Only one replica of each shard is queried. `clusterAllReplicas` function — same as `cluster`, but all replicas are queried. Each replica in a cluster is used as a separate shard/connection. From ccd47126fdcc82e57d0c648b82613b5672847702 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Mar 2024 11:01:47 +0100 Subject: [PATCH 060/117] Fix 01603_insert_select_too_many_parts flakiness CI: https://s3.amazonaws.com/clickhouse-test-reports/60695/048a042dc4963631a23358d3e454dcd8a9eaafa2/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin --- .../queries/0_stateless/01603_insert_select_too_many_parts.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql index a56b680e212..0c33c1d6b18 100644 --- a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql +++ b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql @@ -3,6 +3,8 @@ CREATE TABLE too_many_parts (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTI SYSTEM STOP MERGES too_many_parts; SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; +-- Avoid concurrent partse check to avoid flakiness +SET max_threads=1, max_insert_threads=1; -- exception is not thrown if threshold is exceeded when multi-block INSERT is already started. INSERT INTO too_many_parts SELECT * FROM numbers(10); From c78a029d15f3b8d4796b69ea370af0e3b8abf8e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Mar 2024 11:12:27 +0100 Subject: [PATCH 061/117] tests: fix typo in 01603_insert_select_too_many_parts.sql MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- .../queries/0_stateless/01603_insert_select_too_many_parts.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql index 0c33c1d6b18..9408ccf2bbf 100644 --- a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql +++ b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql @@ -3,7 +3,7 @@ CREATE TABLE too_many_parts (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTI SYSTEM STOP MERGES too_many_parts; SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; --- Avoid concurrent partse check to avoid flakiness +-- Avoid concurrent parts check to avoid flakiness SET max_threads=1, max_insert_threads=1; -- exception is not thrown if threshold is exceeded when multi-block INSERT is already started. From c9c73378eb169e3de7e7579c5ec7edd573e1b72e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 12 Mar 2024 11:52:17 +0100 Subject: [PATCH 062/117] Fix problem detected by ubsan --- .../AggregateFunctionGroupArraySorted.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp index 0692ff28f18..1ee29c362cd 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArraySorted.cpp @@ -310,12 +310,12 @@ public: { for (Field & element : values) { - bool has_value = 0; + /// We must initialize the Field type since some internal functions (like operator=) use them + new (&element) Field; + bool has_value = false; readBinary(has_value, buf); if (has_value) serialization->deserializeBinary(element, buf, {}); - else - element = Field{}; } } else From 4ce5245157418217b3f7e41724df3b8cf3dd3272 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Mar 2024 11:03:39 +0000 Subject: [PATCH 063/117] Automatic style fix --- tests/integration/test_disk_types/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index 5047cdc605e..a53d073d30b 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -50,6 +50,8 @@ def test_different_types(cluster): assert ( fields[encrypted_col_ix] == "0" ), f"{fields[name_col_ix]} expected to be non-encrypted!" + + def test_different_types(cluster): node = cluster.instances["node"] response = TSV.toMat(node.query("SELECT * FROM system.disks FORMAT TSVWithNames")) From c1cefe18744d06bebd67b6425ae5afc9cb922e78 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 12 Mar 2024 11:41:36 +0000 Subject: [PATCH 064/117] Fix keeper reconfig for standalone binary --- src/Coordination/Standalone/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 264cf118501..75b81187973 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -374,7 +374,7 @@ void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::Abstr if (!shared->keeper_dispatcher) return; - shared->keeper_dispatcher->updateConfiguration(getConfigRef(), getMacros()); + shared->keeper_dispatcher->updateConfiguration(config_, getMacros()); } std::shared_ptr Context::getZooKeeper() const From d431276045f6600af92562abf2b6387bd37d068c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 12 Mar 2024 12:43:48 +0100 Subject: [PATCH 065/117] Fix usage of session_token in S3 --- src/Storages/StorageS3.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 11da394feec..ff055508aa6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1451,7 +1451,8 @@ void StorageS3::Configuration::connect(const ContextPtr & context) auth_settings.expiration_window_seconds.value_or( context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), - }); + }, + credentials.GetSessionToken()); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) From 2e803f29f8f89eec0d9eb95b089ecad5dd6e18b7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 12 Mar 2024 12:45:02 +0100 Subject: [PATCH 066/117] Fix KeeperSnapshotManagerS3 --- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 80345db2524..796506a07db 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -121,7 +121,8 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auth_settings.use_insecure_imds_request.value_or(false), auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), auth_settings.no_sign_request.value_or(false), - }); + }, + credentials.GetSessionToken()); auto new_client = std::make_shared(std::move(new_uri), std::move(auth_settings), std::move(client)); From 612ff3c5bc1f4878f869bd6e49cc1e60e58e3afd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 12:35:33 +0100 Subject: [PATCH 067/117] Make every style-checker runner types scaling-out very quickly --- tests/ci/autoscale_runners_lambda/app.py | 8 ++++++-- tests/ci/autoscale_runners_lambda/test_autoscale.py | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 1aa2e424320..a36c508482c 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -51,7 +51,7 @@ class Queue: label: str -def get_scales() -> Tuple[int, int]: +def get_scales(runner_type: str) -> Tuple[int, int]: "returns the multipliers for scaling down and up ASG by types" # Scaling down is quicker on the lack of running jobs than scaling up on # queue @@ -63,8 +63,12 @@ def get_scales() -> Tuple[int, int]: # 10. I am trying 7 now. # 7 still looks a bit slow, so I try 6 # Let's have it the same as the other ASG + # + # All type of style-checkers should be added very quickly to not block the workflows # UPDATE THE COMMENT ON CHANGES scale_up = 3 + if "style" in runner_type: + scale_up = 1 return scale_down, scale_up @@ -95,7 +99,7 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - scale_down, scale_up = get_scales() + scale_down, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state effective_capacity = max( diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 21a407276f9..75f178ac394 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -80,7 +80,7 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("increase-1", 1, 13, 20, [Queue("queued", 23, "increase-1")], 17), TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 33, "style-checker")], 20 + "style-checker", 1, 13, 20, [Queue("queued", 19, "style-checker")], 19 ), TestCase("increase-2", 1, 13, 20, [Queue("queued", 18, "increase-2")], 15), TestCase("increase-3", 1, 13, 20, [Queue("queued", 183, "increase-3")], 20), From 50aab0b69f75c027b0b6dd53ff747492485eed53 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 14:06:17 +0100 Subject: [PATCH 068/117] Fix annoying typo --- tests/ci/autoscale_runners_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index a36c508482c..6c3d71708e9 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -142,7 +142,7 @@ def set_capacity( logging.info( "The ASG %s capacity will be increased to %s, current capacity=%s, " - "effective capacity=%sm maximum capacity=%s, running jobs=%s, queue size=%s", + "effective capacity=%s, maximum capacity=%s, running jobs=%s, queue size=%s", asg["AutoScalingGroupName"], desired_capacity, effective_capacity, From c011901d01709dff91e8afe53121af7388842f33 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 12 Mar 2024 14:28:28 +0100 Subject: [PATCH 069/117] Fix coverage --- src/Functions/coverage.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp index f4cac26df78..a1a43d0cf58 100644 --- a/src/Functions/coverage.cpp +++ b/src/Functions/coverage.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include From b5489ac9620311b160b8c5fa23c02afcc7271114 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 12 Mar 2024 14:05:39 +0000 Subject: [PATCH 070/117] Print out all queries to have better insights --- ...materialized_views_ignore_errors.reference | 26 +++++++++++++++++-- ..._logs_materialized_views_ignore_errors.sql | 8 +++--- 2 files changed, 29 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference index 029f80b46b0..596a047c104 100644 --- a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.reference @@ -1,2 +1,24 @@ -11 queryfinish OK -11 querystart OK +"-- Attach MV to system.query_log and check that writing query_log will not fail\n\nset log_queries=1;","querystart","OK" +"-- Attach MV to system.query_log and check that writing query_log will not fail\n\nset log_queries=1;","queryfinish","OK" +"drop table if exists log_proxy_02572;","querystart","OK" +"drop table if exists log_proxy_02572;","queryfinish","OK" +"drop table if exists push_to_logs_proxy_mv_02572;","querystart","OK" +"drop table if exists push_to_logs_proxy_mv_02572;","queryfinish","OK" +"-- create log tables\nsystem flush logs;","querystart","OK" +"-- create log tables\nsystem flush logs;","queryfinish","OK" +"create table log_proxy_02572 as system.query_log engine=Distributed('test_shard_localhost', currentDatabase(), 'receiver_02572');","querystart","OK" +"create table log_proxy_02572 as system.query_log engine=Distributed('test_shard_localhost', currentDatabase(), 'receiver_02572');","queryfinish","OK" +"create materialized view push_to_logs_proxy_mv_02572 to log_proxy_02572 as select * from system.query_log;","querystart","OK" +"create materialized view push_to_logs_proxy_mv_02572 to log_proxy_02572 as select * from system.query_log;","queryfinish","OK" +"select 1 format Null;","querystart","OK" +"select 1 format Null;","queryfinish","OK" +"system flush logs;","querystart","OK" +"system flush logs;","queryfinish","OK" +"system flush logs;","querystart","OK" +"system flush logs;","queryfinish","OK" +"drop table log_proxy_02572;","querystart","OK" +"drop table log_proxy_02572;","queryfinish","OK" +"drop table push_to_logs_proxy_mv_02572;","querystart","OK" +"drop table push_to_logs_proxy_mv_02572;","queryfinish","OK" +"set log_queries=0;","querystart","OK" +"set log_queries=0;","queryfinish","OK" diff --git a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql index a7a74190821..2381639fba0 100644 --- a/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql +++ b/tests/queries/0_stateless/02572_system_logs_materialized_views_ignore_errors.sql @@ -21,10 +21,12 @@ system flush logs; drop table log_proxy_02572; drop table push_to_logs_proxy_mv_02572; +set log_queries=0; + system flush logs; -- lower() to pass through clickhouse-test "exception" check -select count(), lower(type::String), errorCodeToName(exception_code) +select replaceAll(query, '\n', '\\n'), lower(type::String), errorCodeToName(exception_code) from system.query_log where current_database = currentDatabase() - group by 2, 3 - order by 2; + order by event_time_microseconds + format CSV; From 0b588480f5165af38675911ebba043fe410465db Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Mar 2024 14:34:15 +0000 Subject: [PATCH 071/117] Fix lazy execution in dictGetOrDefault for RangeHashedDictionary --- src/Functions/FunctionsExternalDictionaries.h | 7 ++-- .../03009_range_dict_get_or_default.reference | 1 + .../03009_range_dict_get_or_default.sql | 34 +++++++++++++++++++ 3 files changed, 40 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03009_range_dict_get_or_default.reference create mode 100644 tests/queries/0_stateless/03009_range_dict_get_or_default.sql diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 261c728e9e1..011772baab9 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -324,12 +324,15 @@ public: String getName() const override { return name; } bool isVariadic() const override { return true; } - bool isShortCircuit(ShortCircuitSettings & settings, size_t /*number_of_arguments*/) const override + bool isShortCircuit(ShortCircuitSettings & settings, size_t number_of_arguments) const override { if constexpr (dictionary_get_function_type != DictionaryGetFunctionType::getOrDefault) return false; - settings.arguments_with_disabled_lazy_execution.insert({0, 1, 2}); + /// We execute lazily only last argument with default expression. + for (size_t i = 0; i != number_of_arguments - 1; ++i) + settings.arguments_with_disabled_lazy_execution.insert(i); + settings.enable_lazy_execution_for_common_descendants_of_arguments = false; settings.force_enable_lazy_execution = false; return true; diff --git a/tests/queries/0_stateless/03009_range_dict_get_or_default.reference b/tests/queries/0_stateless/03009_range_dict_get_or_default.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03009_range_dict_get_or_default.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03009_range_dict_get_or_default.sql b/tests/queries/0_stateless/03009_range_dict_get_or_default.sql new file mode 100644 index 00000000000..1f4b4073b9f --- /dev/null +++ b/tests/queries/0_stateless/03009_range_dict_get_or_default.sql @@ -0,0 +1,34 @@ +DROP DICTIONARY IF EXISTS range_dictionary; +DROP TABLE IF EXISTS range_dictionary_nullable_source_table; + + +CREATE TABLE range_dictionary_nullable_source_table +( + key UInt64, + start_date Date, + end_date Date, + value Nullable(UInt64) +) +ENGINE = TinyLog; + +INSERT INTO range_dictionary_nullable_source_table VALUES (0, toDate('2019-05-05'), toDate('2019-05-20'), 0), (1, toDate('2019-05-05'), toDate('2019-05-20'), NULL); + +CREATE DICTIONARY range_dictionary +( + key UInt64, + start_date Date, + end_date Date, + value Nullable(UInt64) DEFAULT NULL +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_dictionary_nullable_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED()) +RANGE(MIN start_date MAX end_date); + +SELECT dictGetOrDefault('range_dictionary', 'value', toUInt64(2), toDate(toLowCardinality(materialize('2019-05-15'))), 2); + + +DROP DICTIONARY IF EXISTS range_dictionary; +DROP TABLE IF EXISTS range_dictionary_nullable_source_table; + From 94f15dd5e058fc9f68bf3644c895908bba849db6 Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Tue, 12 Mar 2024 16:48:43 +0200 Subject: [PATCH 072/117] Fix `has()` function with `Nullable` column Previous implementation didn't check for `null_map` when the given argument was not `Null`. This commit adds the missing check. Fixes #60214 --- src/Functions/array/arrayIndex.h | 9 +++++++-- tests/queries/0_stateless/00662_has_nullable.reference | 2 ++ tests/queries/0_stateless/00662_has_nullable.sql | 7 +++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/00662_has_nullable.reference create mode 100644 tests/queries/0_stateless/00662_has_nullable.sql diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index 3b19f0b486a..cd537763b4a 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -1007,8 +1007,13 @@ private: if (!(*null_map)[row]) continue; } - else if (!applyVisitor(FieldVisitorAccurateEquals(), arr[i], value)) - continue; + else + { + if (null_map && (*null_map)[row]) + continue; + if (!applyVisitor(FieldVisitorAccurateEquals(), arr[i], value)) + continue; + } ConcreteAction::apply(data[row], i); diff --git a/tests/queries/0_stateless/00662_has_nullable.reference b/tests/queries/0_stateless/00662_has_nullable.reference new file mode 100644 index 00000000000..97da99d655e --- /dev/null +++ b/tests/queries/0_stateless/00662_has_nullable.reference @@ -0,0 +1,2 @@ +1 1 +\N 0 diff --git a/tests/queries/0_stateless/00662_has_nullable.sql b/tests/queries/0_stateless/00662_has_nullable.sql new file mode 100644 index 00000000000..0d395871d9f --- /dev/null +++ b/tests/queries/0_stateless/00662_has_nullable.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS 00662_has_nullable; +CREATE TABLE 00662_has_nullable(a Nullable(UInt64)) ENGINE = Memory; + +INSERT INTO 00662_has_nullable VALUES (1), (Null); +SELECT a, has([0, 1], a) FROM 00662_has_nullable; + +DROP TABLE 00662_has_nullable; From 01d0fba0853930de42d4c122c9f067bd91f9af4b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 12 Mar 2024 14:48:59 +0000 Subject: [PATCH 073/117] init --- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index bcedba7346d..3c84d086d85 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2552,7 +2552,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads. if (max_streams > 1 && !is_sync_remote) - max_streams = static_cast(max_streams * settings.max_streams_to_max_threads_ratio); + { + if (auto streams_with_ratio = max_streams * settings.max_streams_to_max_threads_ratio; streams_with_ratio > sizeof(size_t)) + max_streams = static_cast(streams_with_ratio); + else + throw Exception(ErrorCodes::INCORRECT_DATA, "Exceeded limit for `max_streams_to_max_threads_ratio`. Make sure that `max_streams * max_streams_to_max_threads_ratio` not exceeds {}, current value: {}", sizeof(size_t), streams_with_ratio); + } auto & prewhere_info = analysis_result.prewhere_info; From 5576c170517c10757611868444b4f2aa7d9cf41c Mon Sep 17 00:00:00 2001 From: Peter Date: Tue, 12 Mar 2024 22:57:06 +0800 Subject: [PATCH 074/117] Using straightforward approach to import GPG key --- docs/en/getting-started/install.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 234420de374..f3bacf7d092 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -78,8 +78,8 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun #### Setup the Debian repository ``` bash -sudo apt-get install -y apt-transport-https ca-certificates dirmngr -sudo gpg --no-default-keyring --keyring /usr/share/keyrings/clickhouse-keyring.gpg --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 +sudo apt-get install -y apt-transport-https ca-certificates dirmngr curl gpg +curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | sudo gpg --dearmor -o /usr/share/keyrings/clickhouse-keyring.gpg echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list From 24f4cda4845698bc2e014087a46222d135541f36 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 12 Mar 2024 15:34:07 +0000 Subject: [PATCH 075/117] impl --- .../02908_many_requests_to_system_replicas.sh | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh index d3eed891ab9..2c57545e603 100755 --- a/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh +++ b/tests/queries/0_stateless/02908_many_requests_to_system_replicas.sh @@ -12,22 +12,34 @@ CONCURRENCY=200 echo "Creating $NUM_TABLES tables" +function get_done_or_die_trying() +{ + # Sometimes curl produces errors like 'Recv failure: Connection reset by peer' and fails test, let's add a little bit of retries + for _ in $(seq 1 10) + do + curl "$CLICKHOUSE_URL" --silent --fail --show-error --data "$1" &>/dev/null && return + done + + echo "Cannot successfully make request" + exit 1 +} + function init_table() { set -e i=$1 - curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r1_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r1') ORDER BY tuple()" 2>&1 - curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r2_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r2') ORDER BY tuple()" 2>&1 - curl $CLICKHOUSE_URL --silent --fail --show-error --data "CREATE TABLE test_02908_r3_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r3') ORDER BY tuple()" 2>&1 + get_done_or_die_trying "CREATE TABLE test_02908_r1_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r1') ORDER BY tuple()" + get_done_or_die_trying "CREATE TABLE test_02908_r2_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r2') ORDER BY tuple()" + get_done_or_die_trying "CREATE TABLE test_02908_r3_$i (a UInt64) ENGINE=ReplicatedMergeTree('/02908/{database}/test_$i', 'r3') ORDER BY tuple()" - curl $CLICKHOUSE_URL --silent --fail --show-error --data "INSERT INTO test_02908_r1_$i SELECT rand64() FROM numbers(5);" 2>&1 + get_done_or_die_trying "INSERT INTO test_02908_r1_$i SELECT rand64() FROM numbers(5);" } export init_table; -for i in `seq 1 $NUM_TABLES`; +for i in $(seq 1 $NUM_TABLES) do - init_table $i & + init_table "$i" & done wait; @@ -35,15 +47,15 @@ wait; echo "Making $CONCURRENCY requests to system.replicas" -for i in `seq 1 $CONCURRENCY`; +for i in $(seq 1 $CONCURRENCY) do - curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT * FROM system.replicas WHERE database=currentDatabase() FORMAT Null;" 2>&1 || echo "query $i failed" & + curl "$CLICKHOUSE_URL" --silent --fail --show-error --data "SELECT * FROM system.replicas WHERE database=currentDatabase() FORMAT Null;" 2>&1 || echo "query $i failed" & done echo "Query system.replicas while waiting for other concurrent requests to finish" # lost_part_count column is read from ZooKeeper -curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT sum(lost_part_count) FROM system.replicas WHERE database=currentDatabase();" 2>&1; +curl "$CLICKHOUSE_URL" --silent --fail --show-error --data "SELECT sum(lost_part_count) FROM system.replicas WHERE database=currentDatabase();" 2>&1; # is_leader column is filled without ZooKeeper -curl $CLICKHOUSE_URL --silent --fail --show-error --data "SELECT sum(is_leader) FROM system.replicas WHERE database=currentDatabase();" 2>&1; +curl "$CLICKHOUSE_URL" --silent --fail --show-error --data "SELECT sum(is_leader) FROM system.replicas WHERE database=currentDatabase();" 2>&1; wait; From 07ee777c8d354526cf89c3647863916a0855de49 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Mar 2024 15:39:04 +0000 Subject: [PATCH 076/117] Restart CI From 858ad2d68860d7993280b90c0f17ae3a0e84d712 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 12 Mar 2024 16:39:10 +0100 Subject: [PATCH 077/117] beautify and change max_value --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3c84d086d85..a314492c5b0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2553,10 +2553,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads. if (max_streams > 1 && !is_sync_remote) { - if (auto streams_with_ratio = max_streams * settings.max_streams_to_max_threads_ratio; streams_with_ratio > sizeof(size_t)) + if (auto streams_with_ratio = max_streams * settings.max_streams_to_max_threads_ratio; streams_with_ratio < SIZE_MAX) max_streams = static_cast(streams_with_ratio); else - throw Exception(ErrorCodes::INCORRECT_DATA, "Exceeded limit for `max_streams_to_max_threads_ratio`. Make sure that `max_streams * max_streams_to_max_threads_ratio` not exceeds {}, current value: {}", sizeof(size_t), streams_with_ratio); + throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Exceeded limit for `max_streams` with `max_streams_to_max_threads_ratio`. Make sure that `max_streams * max_streams_to_max_threads_ratio` not exceeds {}, current value: {}", SIZE_MAX, streams_with_ratio); } auto & prewhere_info = analysis_result.prewhere_info; From 2fbc11302148f3374e5ce69ed50adebbce9cbae4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 12 Mar 2024 17:00:44 +0100 Subject: [PATCH 078/117] Adjust the packages in installation docs --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index f3bacf7d092..ca689ef7995 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -78,7 +78,7 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun #### Setup the Debian repository ``` bash -sudo apt-get install -y apt-transport-https ca-certificates dirmngr curl gpg +sudo apt-get install -y apt-transport-https ca-certificates curl gnupg curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | sudo gpg --dearmor -o /usr/share/keyrings/clickhouse-keyring.gpg echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \ From e1ea0671b132097182279831560f1bafc07c95ce Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 12 Mar 2024 17:27:25 +0100 Subject: [PATCH 079/117] Disable test 02998_primary_key_skip_columns.sql in sanitizer builds as it can be slow --- tests/queries/0_stateless/02998_primary_key_skip_columns.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql index b2dadcc5e7c..b567155ab1f 100644 --- a/tests/queries/0_stateless/02998_primary_key_skip_columns.sql +++ b/tests/queries/0_stateless/02998_primary_key_skip_columns.sql @@ -1,3 +1,5 @@ +-- Tags: no-asan, no-tsan, no-msan, no-ubsan + DROP TABLE IF EXISTS test; CREATE TABLE test (a UInt64, b UInt64, c UInt64) ENGINE = MergeTree ORDER BY (a, b, c) SETTINGS index_granularity = 1, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns = 1; From 53442f49140c9c8b391f5b3a76a9bf4ab45dbed0 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 12 Mar 2024 13:48:47 -0300 Subject: [PATCH 080/117] Cleans up markdown. --- docs/en/sql-reference/aggregate-functions/reference/varpop.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index 5f18bdc30f6..76472f62789 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -27,7 +27,7 @@ Returns an integer of type `Float64`. **Implementation details** -This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable` function](#varPopStable). +This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable` function](#varPopStable). **Example** From b3cfb8a2cba2f779e63cb35aec49fd263a24eb19 Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 12 Mar 2024 13:57:03 -0300 Subject: [PATCH 081/117] Adds sleep function docs. --- .../functions/other-functions.md | 92 ++++++++++++++++++- 1 file changed, 89 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 739b688a0d2..de8ac67a936 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -300,11 +300,97 @@ The argument is internally still evaluated. Useful e.g. for benchmarks. ## sleep(seconds) -Sleeps ‘seconds’ seconds for each data block. The sleep time can be specified as integer or as floating-point number. +Used to introduce a delay or pause in the execution of a query. It is primarily used for testing and debugging purposes. -## sleepEachRow(seconds) +**Syntax** -Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as integer or as floating-point number. +```sql +sleep(seconds) +``` + +**Arguments** + +- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. + +**Returned value** + +This function does not return any value. + +**Example** + +```sql +SELECT sleep(2); +``` + +This function does not return any value. However, if you run the function with `clickhouse client` you will see something similar to: + +```response +SELECT sleep(2) + +Query id: 8aa9943e-a686-45e1-8317-6e8e3a5596ac + +┌─sleep(2)─┐ +│ 0 │ +└──────────┘ + +1 row in set. Elapsed: 2.012 sec. +``` + +This query will pause for 2 seconds before completing. During this time, no results will be returned, and the query will appear to be hanging or unresponsive. + +**Implementation details** + +The `sleep()` function is generally not used in production environments, as it can negatively impact query performance and system responsiveness. However, it can be useful in the following scenarios: + +1. **Testing**: When testing or benchmarking ClickHouse, you may want to simulate delays or introduce pauses to observe how the system behaves under certain conditions. +2. **Debugging**: If you need to examine the state of the system or the execution of a query at a specific point in time, you can use `sleep()` to introduce a pause, allowing you to inspect or collect relevant information. +3. **Simulation**: In some cases, you may want to simulate real-world scenarios where delays or pauses occur, such as network latency or external system dependencies. + +It's important to use the `sleep()` function judiciously and only when necessary, as it can potentially impact the overall performance and responsiveness of your ClickHouse system. + +## sleepEachRow + +Pauses the execution of a query for a specified number of seconds for each row in the result set. + +**Syntax** + +```sql +sleepEachRow(seconds) +``` + +**Arguments** + +- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution for each row in the result set. It can be a floating-point value to specify fractional seconds. + +**Returned value** + +This function returns the same input values as it receives, without modifying them. + +**Example** + +```sql +SELECT number, sleepEachRow(0.5) FROM system.numbers LIMIT 5; +``` + +```response +┌─number─┬─sleepEachRow(0.5)─┐ +│ 0 │ 0 │ +│ 1 │ 0 │ +│ 2 │ 0 │ +│ 3 │ 0 │ +│ 4 │ 0 │ +└────────┴───────────────────┘ +``` + +But the output will be delayed, with a 0.5-second pause between each row. + +The `sleepEachRow()` function is primarily used for testing and debugging purposes, similar to the `sleep()` function. It allows you to simulate delays or introduce pauses in the processing of each row, which can be useful in scenarios such as: + +1. **Testing**: When testing or benchmarking ClickHouse's performance under specific conditions, you can use `sleepEachRow()` to simulate delays or introduce pauses for each row processed. +2. **Debugging**: If you need to examine the state of the system or the execution of a query for each row processed, you can use `sleepEachRow()` to introduce pauses, allowing you to inspect or collect relevant information. +3. **Simulation**: In some cases, you may want to simulate real-world scenarios where delays or pauses occur for each row processed, such as when dealing with external systems or network latencies. + +Like the [`sleep()` function](#sleep), it's important to use `sleepEachRow()` judiciously and only when necessary, as it can significantly impact the overall performance and responsiveness of your ClickHouse system, especially when dealing with large result sets. ## currentDatabase() From 2847def5176509bdbefe0809c2abbe31baf540af Mon Sep 17 00:00:00 2001 From: johnnymatthews <9611008+johnnymatthews@users.noreply.github.com> Date: Tue, 12 Mar 2024 13:57:34 -0300 Subject: [PATCH 082/117] Formats markdown with prettier. --- .../functions/other-functions.md | 408 +++++++++--------- 1 file changed, 204 insertions(+), 204 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index de8ac67a936..df443eec0de 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -17,7 +17,7 @@ Returns a named value from the [macros](../../operations/server-configuration-pa **Syntax** -``` sql +```sql getMacro(name); ``` @@ -35,7 +35,7 @@ Type: [String](../../sql-reference/data-types/string.md). Example `` section in the server configuration file: -``` xml +```xml Value @@ -43,13 +43,13 @@ Example `` section in the server configuration file: Query: -``` sql +```sql SELECT getMacro('test'); ``` Result: -``` text +```text ┌─getMacro('test')─┐ │ Value │ └──────────────────┘ @@ -57,12 +57,12 @@ Result: The same value can be retrieved as follows: -``` sql +```sql SELECT * FROM system.macros WHERE macro = 'test'; ``` -``` text +```text ┌─macro─┬─substitution─┐ │ test │ Value │ └───────┴──────────────┘ @@ -74,7 +74,7 @@ Returns the fully qualified domain name of the ClickHouse server. **Syntax** -``` sql +```sql fqdn(); ``` @@ -88,13 +88,13 @@ Type: `String`. **Example** -``` sql +```sql SELECT FQDN(); ``` Result: -``` text +```text ┌─FQDN()──────────────────────────┐ │ clickhouse.ru-central1.internal │ └─────────────────────────────────┘ @@ -104,7 +104,7 @@ Result: Extracts the tail of a string following its last slash or backslash. This function if often used to extract the filename from a path. -``` sql +```sql basename(expr) ``` @@ -123,13 +123,13 @@ A string that contains: Query: -``` sql +```sql SELECT 'some/long/path/to/file' AS a, basename(a) ``` Result: -``` text +```text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ @@ -137,13 +137,13 @@ Result: Query: -``` sql +```sql SELECT 'some\\long\\path\\to\\file' AS a, basename(a) ``` Result: -``` text +```text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ @@ -151,13 +151,13 @@ Result: Query: -``` sql +```sql SELECT 'some-file-name' AS a, basename(a) ``` Result: -``` text +```text ┌─a──────────────┬─basename('some-file-name')─┐ │ some-file-name │ some-file-name │ └────────────────┴────────────────────────────┘ @@ -170,11 +170,11 @@ This function is used by the system to implement Pretty formats. `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. -``` sql +```sql SELECT visibleWidth(NULL) ``` -``` text +```text ┌─visibleWidth(NULL)─┐ │ 4 │ └────────────────────┘ @@ -256,7 +256,7 @@ SELECT key, byteSize(u8) AS `byteSize(UInt8)`, byteSize(u16) AS `byteSize(UInt16 Result: -``` text +```text Row 1: ────── key: 1 @@ -401,7 +401,7 @@ Useful in table engine parameters of `CREATE TABLE` queries where you need to sp Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. -``` sql +```sql SELECT currentUser(); ``` @@ -416,13 +416,13 @@ Type: `String`. **Example** -``` sql +```sql SELECT currentUser(); ``` Result: -``` text +```text ┌─currentUser()─┐ │ default │ └───────────────┘ @@ -438,7 +438,7 @@ This function is mostly intended for development, debugging and demonstration. **Syntax** -``` sql +```sql isConstant(x) ``` @@ -457,13 +457,13 @@ Type: [UInt8](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT isConstant(x + 1) FROM (SELECT 43 AS x) ``` Result: -``` text +```text ┌─isConstant(plus(x, 1))─┐ │ 1 │ └────────────────────────┘ @@ -471,13 +471,13 @@ Result: Query: -``` sql +```sql WITH 3.14 AS pi SELECT isConstant(cos(pi)) ``` Result: -``` text +```text ┌─isConstant(cos(pi))─┐ │ 1 │ └─────────────────────┘ @@ -485,13 +485,13 @@ Result: Query: -``` sql +```sql SELECT isConstant(number) FROM numbers(1) ``` Result: -``` text +```text ┌─isConstant(number)─┐ │ 0 │ └────────────────────┘ @@ -511,7 +511,7 @@ Checks whether a floating point value is finite. **Syntax** -``` sql +```sql ifNotFinite(x,y) ``` @@ -565,7 +565,7 @@ The band is drawn with accuracy to one eighth of a symbol. Example: -``` sql +```sql SELECT toHour(EventTime) AS h, count() AS c, @@ -575,7 +575,7 @@ GROUP BY h ORDER BY h ASC ``` -``` text +```text ┌──h─┬──────c─┬─bar────────────────┐ │ 0 │ 292907 │ █████████▋ │ │ 1 │ 180563 │ ██████ │ @@ -633,7 +633,7 @@ For example, the first argument could have type `Int64`, while the second argume Example: -``` sql +```sql SELECT transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other') AS title, count() AS c @@ -643,7 +643,7 @@ GROUP BY title ORDER BY c DESC ``` -``` text +```text ┌─title─────┬──────c─┐ │ Yandex │ 498635 │ │ Google │ 229872 │ @@ -657,7 +657,7 @@ Similar to the other variation but has no ‘default’ argument. In case no mat Example: -``` sql +```sql SELECT transform(domain(Referer), ['yandex.ru', 'google.ru', 'vkontakte.ru'], ['www.yandex', 'example.com', 'vk.com']) AS s, count() AS c @@ -667,7 +667,7 @@ ORDER BY count() DESC LIMIT 10 ``` -``` text +```text ┌─s──────────────┬───────c─┐ │ │ 2906259 │ │ www.yandex │ 867767 │ @@ -687,13 +687,13 @@ Given a size (number of bytes), this function returns a readable, rounded size w Example: -``` sql +```sql SELECT arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, formatReadableDecimalSize(filesize_bytes) AS filesize ``` -``` text +```text ┌─filesize_bytes─┬─filesize───┐ │ 1 │ 1.00 B │ │ 1024 │ 1.02 KB │ @@ -708,7 +708,7 @@ Given a size (number of bytes), this function returns a readable, rounded size w Example: -``` sql +```sql SELECT arrayJoin([1, 1024, 1024*1024, 192851925]) AS filesize_bytes, formatReadableSize(filesize_bytes) AS filesize @@ -716,7 +716,7 @@ SELECT Alias: `FORMAT_BYTES`. -``` text +```text ┌─filesize_bytes─┬─filesize───┐ │ 1 │ 1.00 B │ │ 1024 │ 1.00 KiB │ @@ -731,13 +731,13 @@ Given a number, this function returns a rounded number with suffix (thousand, mi Example: -``` sql +```sql SELECT arrayJoin([1024, 1234 * 1000, (4567 * 1000) * 1000, 98765432101234]) AS number, formatReadableQuantity(number) AS number_for_humans ``` -``` text +```text ┌─────────number─┬─number_for_humans─┐ │ 1024 │ 1.02 thousand │ │ 1234000 │ 1.23 million │ @@ -752,7 +752,7 @@ Given a time interval (delta) in seconds, this function returns a time delta wit **Syntax** -``` sql +```sql formatReadableTimeDelta(column[, maximum_unit, minimum_unit]) ``` @@ -760,21 +760,22 @@ formatReadableTimeDelta(column[, maximum_unit, minimum_unit]) - `column` — A column with a numeric time delta. - `maximum_unit` — Optional. Maximum unit to show. - * Acceptable values: `nanoseconds`, `microseconds`, `milliseconds`, `seconds`, `minutes`, `hours`, `days`, `months`, `years`. - * Default value: `years`. + - Acceptable values: `nanoseconds`, `microseconds`, `milliseconds`, `seconds`, `minutes`, `hours`, `days`, `months`, `years`. + - Default value: `years`. - `minimum_unit` — Optional. Minimum unit to show. All smaller units are truncated. - * Acceptable values: `nanoseconds`, `microseconds`, `milliseconds`, `seconds`, `minutes`, `hours`, `days`, `months`, `years`. - * If explicitly specified value is bigger than `maximum_unit`, an exception will be thrown. - * Default value: `seconds` if `maximum_unit` is `seconds` or bigger, `nanoseconds` otherwise. + - Acceptable values: `nanoseconds`, `microseconds`, `milliseconds`, `seconds`, `minutes`, `hours`, `days`, `months`, `years`. + - If explicitly specified value is bigger than `maximum_unit`, an exception will be thrown. + - Default value: `seconds` if `maximum_unit` is `seconds` or bigger, `nanoseconds` otherwise. **Example** -``` sql + +```sql SELECT arrayJoin([100, 12345, 432546534]) AS elapsed, formatReadableTimeDelta(elapsed) AS time_delta ``` -``` text +```text ┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐ │ 100 │ 1 minute and 40 seconds │ │ 12345 │ 3 hours, 25 minutes and 45 seconds │ @@ -782,13 +783,13 @@ SELECT └────────────┴─────────────────────────────────────────────────────────────────┘ ``` -``` sql +```sql SELECT arrayJoin([100, 12345, 432546534]) AS elapsed, formatReadableTimeDelta(elapsed, 'minutes') AS time_delta ``` -``` text +```text ┌────elapsed─┬─time_delta ─────────────────────────────────────────────────────┐ │ 100 │ 1 minute and 40 seconds │ │ 12345 │ 205 minutes and 45 seconds │ @@ -824,7 +825,6 @@ parseTimeDelta(timestr) - `timestr` — A sequence of numbers followed by something resembling a time unit. - **Returned value** - A floating-point number with the number of seconds. @@ -936,7 +936,7 @@ The window function that provides access to a row at a specified offset before o **Syntax** -``` sql +```sql neighbor(column, offset[, default_value]) ``` @@ -966,13 +966,13 @@ Type: type of data blocks affected or default value type. Query: -``` sql +```sql SELECT number, neighbor(number, 2) FROM system.numbers LIMIT 10; ``` Result: -``` text +```text ┌─number─┬─neighbor(number, 2)─┐ │ 0 │ 2 │ │ 1 │ 3 │ @@ -989,13 +989,13 @@ Result: Query: -``` sql +```sql SELECT number, neighbor(number, 2, 999) FROM system.numbers LIMIT 10; ``` Result: -``` text +```text ┌─number─┬─neighbor(number, 2, 999)─┐ │ 0 │ 2 │ │ 1 │ 3 │ @@ -1014,7 +1014,7 @@ This function can be used to compute year-over-year metric value: Query: -``` sql +```sql WITH toDate('2018-01-01') AS start_date SELECT toStartOfMonth(start_date + (number * 32)) AS month, @@ -1026,7 +1026,7 @@ FROM numbers(16) Result: -``` text +```text ┌──────month─┬─money─┬─prev_year─┬─year_over_year─┐ │ 2018-01-01 │ 32 │ 0 │ 0 │ │ 2018-02-01 │ 63 │ 0 │ 0 │ @@ -1063,7 +1063,7 @@ To prevent that you can create a subquery with [ORDER BY](../../sql-reference/st Example: -``` sql +```sql SELECT EventID, EventTime, @@ -1080,7 +1080,7 @@ FROM ) ``` -``` text +```text ┌─EventID─┬───────────EventTime─┬─delta─┐ │ 1106 │ 2016-11-24 00:00:04 │ 0 │ │ 1107 │ 2016-11-24 00:00:05 │ 1 │ @@ -1092,7 +1092,7 @@ FROM Please note that the block size affects the result. The internal state of `runningDifference` state is reset for each new block. -``` sql +```sql SELECT number, runningDifference(number + 1) AS diff @@ -1100,7 +1100,7 @@ FROM numbers(100000) WHERE diff != 1 ``` -``` text +```text ┌─number─┬─diff─┐ │ 0 │ 0 │ └────────┴──────┘ @@ -1109,7 +1109,7 @@ WHERE diff != 1 └────────┴──────┘ ``` -``` sql +```sql set max_block_size=100000 -- default value is 65536! SELECT @@ -1119,7 +1119,7 @@ FROM numbers(100000) WHERE diff != 1 ``` -``` text +```text ┌─number─┬─diff─┐ │ 0 │ 0 │ └────────┴──────┘ @@ -1135,21 +1135,20 @@ Calculates the number of concurrent events. Each event has a start time and an end time. The start time is included in the event, while the end time is excluded. Columns with a start time and an end time must be of the same data type. The function calculates the total number of active (concurrent) events for each event start time. - :::tip Events must be ordered by the start time in ascending order. If this requirement is violated the function raises an exception. Every data block is processed separately. If events from different data blocks overlap then they can not be processed correctly. ::: **Syntax** -``` sql +```sql runningConcurrency(start, end) ``` **Arguments** - `start` — A column with the start time of events. [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), or [DateTime64](../../sql-reference/data-types/datetime64.md). -- `end` — A column with the end time of events. [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), or [DateTime64](../../sql-reference/data-types/datetime64.md). +- `end` — A column with the end time of events. [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md), or [DateTime64](../../sql-reference/data-types/datetime64.md). **Returned values** @@ -1161,7 +1160,7 @@ Type: [UInt32](../../sql-reference/data-types/int-uint.md) Consider the table: -``` text +```text ┌──────start─┬────────end─┐ │ 2021-03-03 │ 2021-03-11 │ │ 2021-03-06 │ 2021-03-12 │ @@ -1172,13 +1171,13 @@ Consider the table: Query: -``` sql +```sql SELECT start, runningConcurrency(start, end) FROM example_table; ``` Result: -``` text +```text ┌──────start─┬─runningConcurrency(start, end)─┐ │ 2021-03-03 │ 1 │ │ 2021-03-06 │ 2 │ @@ -1204,7 +1203,7 @@ Given a MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexa Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md). An exception is thrown if the type is not `Enum`. -``` sql +```sql getSizeOfEnumType(value) ``` @@ -1218,11 +1217,11 @@ getSizeOfEnumType(value) **Example** -``` sql +```sql SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x ``` -``` text +```text ┌─x─┐ │ 2 │ └───┘ @@ -1232,7 +1231,7 @@ SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x Returns the size on disk without considering compression. -``` sql +```sql blockSerializedSize(value[, value[, ...]]) ``` @@ -1248,13 +1247,13 @@ blockSerializedSize(value[, value[, ...]]) Query: -``` sql +```sql SELECT blockSerializedSize(maxState(1)) as x ``` Result: -``` text +```text ┌─x─┐ │ 2 │ └───┘ @@ -1264,7 +1263,7 @@ Result: Returns the internal name of the data type that represents the value. -``` sql +```sql toColumnTypeName(value) ``` @@ -1280,13 +1279,13 @@ toColumnTypeName(value) Difference between `toTypeName ' and ' toColumnTypeName`: -``` sql +```sql SELECT toTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` Result: -``` text +```text ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime │ └─────────────────────────────────────────────────────┘ @@ -1294,13 +1293,13 @@ Result: Query: -``` sql +```sql SELECT toColumnTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` Result: -``` text +```text ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ Const(UInt32) │ └───────────────────────────────────────────────────────────┘ @@ -1312,7 +1311,7 @@ The example shows that the `DateTime` data type is internally stored as `Const(U Outputs a detailed description of data structures in RAM -``` sql +```sql dumpColumnStructure(value) ``` @@ -1326,11 +1325,11 @@ dumpColumnStructure(value) **Example** -``` sql +```sql SELECT dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime')) ``` -``` text +```text ┌─dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime, Const(size = 1, UInt32(size = 1)) │ └──────────────────────────────────────────────────────────────┘ @@ -1342,7 +1341,7 @@ Returns the default value for the given data type. Does not include default values for custom columns set by the user. -``` sql +```sql defaultValueOfArgumentType(expression) ``` @@ -1360,13 +1359,13 @@ defaultValueOfArgumentType(expression) Query: -``` sql +```sql SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) ``` Result: -``` text +```text ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ │ 0 │ └─────────────────────────────────────────────┘ @@ -1374,13 +1373,13 @@ Result: Query: -``` sql +```sql SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ``` Result: -``` text +```text ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ │ ᴺᵁᴸᴸ │ └───────────────────────────────────────────────────────┘ @@ -1392,7 +1391,7 @@ Returns the default value for the given type name. Does not include default values for custom columns set by the user. -``` sql +```sql defaultValueOfTypeName(type) ``` @@ -1410,13 +1409,13 @@ defaultValueOfTypeName(type) Query: -``` sql +```sql SELECT defaultValueOfTypeName('Int8') ``` Result: -``` text +```text ┌─defaultValueOfTypeName('Int8')─┐ │ 0 │ └────────────────────────────────┘ @@ -1424,13 +1423,13 @@ Result: Query: -``` sql +```sql SELECT defaultValueOfTypeName('Nullable(Int8)') ``` Result: -``` text +```text ┌─defaultValueOfTypeName('Nullable(Int8)')─┐ │ ᴺᵁᴸᴸ │ └──────────────────────────────────────────┘ @@ -1542,7 +1541,7 @@ Creates an array with a single value. Used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). -``` sql +```sql SELECT replicate(x, arr); ``` @@ -1561,13 +1560,13 @@ Type: `Array`. Query: -``` sql +```sql SELECT replicate(1, ['a', 'b', 'c']) ``` Result: -``` text +```text ┌─replicate(1, ['a', 'b', 'c'])─┐ │ [1,1,1] │ └───────────────────────────────┘ @@ -1579,7 +1578,7 @@ Returns the amount of free space in the filesystem hosting the database persiste **Syntax** -``` sql +```sql filesystemAvailable() ``` @@ -1593,13 +1592,13 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT formatReadableSize(filesystemAvailable()) AS "Available space"; ``` Result: -``` text +```text ┌─Available space─┐ │ 30.75 GiB │ └─────────────────┘ @@ -1611,7 +1610,7 @@ Returns the total amount of the free space on the filesystem hosting the databas **Syntax** -``` sql +```sql filesystemFree() ``` @@ -1625,13 +1624,13 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT formatReadableSize(filesystemFree()) AS "Free space"; ``` Result: -``` text +```text ┌─Free space─┐ │ 32.39 GiB │ └────────────┘ @@ -1643,7 +1642,7 @@ Returns the capacity of the filesystem in bytes. Needs the [path](../../operatio **Syntax** -``` sql +```sql filesystemCapacity() ``` @@ -1657,13 +1656,13 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT formatReadableSize(filesystemCapacity()) AS "Capacity"; ``` Result: -``` text +```text ┌─Capacity──┐ │ 39.32 GiB │ └───────────┘ @@ -1675,7 +1674,7 @@ Calculates the result of an aggregate function based on a single value. This fun **Syntax** -``` sql +```sql initializeAggregation (aggregate_function, arg1, arg2, ..., argN) ``` @@ -1697,6 +1696,7 @@ Query: ```sql SELECT uniqMerge(state) FROM (SELECT initializeAggregation('uniqState', number % 3) AS state FROM numbers(10000)); ``` + Result: ```text @@ -1749,7 +1749,7 @@ Given a state of aggregate function, this function returns the result of aggrega **Syntax** -``` sql +```sql finalizeAggregation(state) ``` @@ -1854,7 +1854,7 @@ The state is reset for each new block of data. **Syntax** -``` sql +```sql runningAccumulate(agg_state[, grouping]); ``` @@ -1875,13 +1875,13 @@ Consider how you can use `runningAccumulate` to find the cumulative sum of numbe Query: -``` sql +```sql SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); ``` Result: -``` text +```text ┌─k─┬─res─┐ │ 0 │ 0 │ │ 1 │ 1 │ @@ -1909,7 +1909,7 @@ The following example shows the `groupping` parameter usage: Query: -``` sql +```sql SELECT grouping, item, @@ -1928,7 +1928,7 @@ FROM Result: -``` text +```text ┌─grouping─┬─item─┬─res─┐ │ 0 │ 0 │ 0 │ │ 0 │ 1 │ 1 │ @@ -1960,7 +1960,7 @@ Only supports tables created with the `ENGINE = Join(ANY, LEFT, )` st **Syntax** -``` sql +```sql joinGet(join_storage_table_name, `value_column`, join_keys) ``` @@ -1982,13 +1982,13 @@ More info about `join_use_nulls` in [Join operation](../../engines/table-engines Input table: -``` sql +```sql CREATE DATABASE db_test CREATE TABLE db_test.id_val(`id` UInt32, `val` UInt32) ENGINE = Join(ANY, LEFT, id) SETTINGS join_use_nulls = 1 INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) ``` -``` text +```text ┌─id─┬─val─┐ │ 4 │ 13 │ │ 2 │ 12 │ @@ -1998,13 +1998,13 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) Query: -``` sql +```sql SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 ``` Result: -``` text +```text ┌─joinGet(db_test.id_val, 'val', toUInt32(number))─┐ │ 0 │ │ 11 │ @@ -2022,7 +2022,7 @@ This function is not available in ClickHouse Cloud. Evaluate an external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learning. Accepts a path to a catboost model and model arguments (features). Returns Float64. -``` sql +```sql SELECT feat1, ..., feat_n, catboostEvaluate('/path/to/model.bin', feat_1, ..., feat_n) AS prediction FROM data_table ``` @@ -2035,7 +2035,7 @@ Before evaluating catboost models, the `libcatboostmodel.` library mus Next, specify the path to `libcatboostmodel.` in the clickhouse configuration: -``` xml +```xml ... /path/to/libcatboostmodel.so @@ -2048,7 +2048,7 @@ At the first execution of `catboostEvaluate()`, the server starts the library br communicate using a HTTP interface. By default, port `9012` is used. A different port can be specified as follows - this is useful if port `9012` is already assigned to a different service. -``` xml +```xml 9019 @@ -2072,13 +2072,13 @@ To use the `error_code` argument, configuration parameter `allow_custom_error_co **Example** -``` sql +```sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); ``` Result: -``` text +```text ↙ Progress: 0.00 rows, 0.00 B (0.00 rows/s., 0.00 B/s.) Received exception from server (version 19.14.1): Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. ``` @@ -2089,7 +2089,7 @@ Returns its argument. Intended for debugging and testing. Allows to cancel using **Syntax** -``` sql +```sql identity(x) ``` @@ -2097,13 +2097,13 @@ identity(x) Query: -``` sql +```sql SELECT identity(42); ``` Result: -``` text +```text ┌─identity(42)─┐ │ 42 │ └──────────────┘ @@ -2150,7 +2150,7 @@ Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is **Syntax** -``` sql +```sql isDecimalOverflow(d, [p]) ``` @@ -2168,7 +2168,7 @@ isDecimalOverflow(d, [p]) Query: -``` sql +```sql SELECT isDecimalOverflow(toDecimal32(1000000000, 0), 9), isDecimalOverflow(toDecimal32(1000000000, 0)), isDecimalOverflow(toDecimal32(-1000000000, 0), 9), @@ -2177,7 +2177,7 @@ SELECT isDecimalOverflow(toDecimal32(1000000000, 0), 9), Result: -``` text +```text 1 1 1 1 ``` @@ -2187,7 +2187,7 @@ Returns number of decimal digits need to represent a value. **Syntax** -``` sql +```sql countDigits(x) ``` @@ -2209,7 +2209,7 @@ For `Decimal` values takes into account their scales: calculates result over und Query: -``` sql +```sql SELECT countDigits(toDecimal32(1, 9)), countDigits(toDecimal32(-1, 9)), countDigits(toDecimal64(1, 18)), countDigits(toDecimal64(-1, 18)), countDigits(toDecimal128(1, 38)), countDigits(toDecimal128(-1, 38)); @@ -2217,7 +2217,7 @@ SELECT countDigits(toDecimal32(1, 9)), countDigits(toDecimal32(-1, 9)), Result: -``` text +```text 10 10 19 19 39 39 ``` @@ -2229,13 +2229,13 @@ Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md) **Syntax** -``` sql +```sql errorCodeToName(1) ``` Result: -``` text +```text UNSUPPORTED_METHOD ``` @@ -2246,7 +2246,7 @@ If executed in the context of a distributed table, this function generates a nor **Syntax** -``` sql +```sql tcpPort() ``` @@ -2264,13 +2264,13 @@ Type: [UInt16](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT tcpPort(); ``` Result: -``` text +```text ┌─tcpPort()─┐ │ 9000 │ └───────────┘ @@ -2288,7 +2288,7 @@ The command [SET PROFILE](../../sql-reference/statements/set.md#query-set) could **Syntax** -``` sql +```sql currentProfiles() ``` @@ -2300,11 +2300,11 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## enabledProfiles - Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). +Returns settings profiles, assigned to the current user both explicitly and implicitly. Explicitly assigned profiles are the same as returned by the [currentProfiles](#current-profiles) function. Implicitly assigned profiles include parent profiles of other assigned profiles, profiles assigned via granted roles, profiles assigned via their own settings, and the main default profile (see the `default_profile` section in the main server configuration file). **Syntax** -``` sql +```sql enabledProfiles() ``` @@ -2320,7 +2320,7 @@ Returns all the profiles specified at the current user's definition (see [CREATE **Syntax** -``` sql +```sql defaultProfiles() ``` @@ -2336,7 +2336,7 @@ Returns the roles assigned to the current user. The roles can be changed by the **Syntax** -``` sql +```sql currentRoles() ``` @@ -2352,7 +2352,7 @@ Returns the names of the current roles and the roles, granted to some of the cur **Syntax** -``` sql +```sql enabledRoles() ``` @@ -2368,7 +2368,7 @@ Returns the roles which are enabled by default for the current user when he logs **Syntax** -``` sql +```sql defaultRoles() ``` @@ -2384,7 +2384,7 @@ Returns the server port number. When the port is not used by the server, throws **Syntax** -``` sql +```sql getServerPort(port_name) ``` @@ -2392,16 +2392,16 @@ getServerPort(port_name) - `port_name` — The name of the server port. [String](../../sql-reference/data-types/string.md#string). Possible values: - - 'tcp_port' - - 'tcp_port_secure' - - 'http_port' - - 'https_port' - - 'interserver_http_port' - - 'interserver_https_port' - - 'mysql_port' - - 'postgresql_port' - - 'grpc_port' - - 'prometheus.port' + - 'tcp_port' + - 'tcp_port_secure' + - 'http_port' + - 'https_port' + - 'interserver_http_port' + - 'interserver_https_port' + - 'mysql_port' + - 'postgresql_port' + - 'grpc_port' + - 'prometheus.port' **Returned value** @@ -2413,13 +2413,13 @@ Type: [UInt16](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT getServerPort('tcp_port'); ``` Result: -``` text +```text ┌─getServerPort('tcp_port')─┐ │ 9000 │ └───────────────────────────┘ @@ -2433,7 +2433,7 @@ In contrast to [initialQueryID](#initial-query-id) function, `queryID` can retur **Syntax** -``` sql +```sql queryID() ``` @@ -2447,7 +2447,7 @@ Type: [String](../../sql-reference/data-types/string.md) Query: -``` sql +```sql CREATE TABLE tmp (str String) ENGINE = Log; INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); @@ -2455,7 +2455,7 @@ SELECT count(DISTINCT t) FROM (SELECT queryID() AS t FROM remote('127.0.0.{1..3} Result: -``` text +```text ┌─count()─┐ │ 3 │ └─────────┘ @@ -2469,7 +2469,7 @@ In contrast to [queryID](#query-id) function, `initialQueryID` returns the same **Syntax** -``` sql +```sql initialQueryID() ``` @@ -2483,7 +2483,7 @@ Type: [String](../../sql-reference/data-types/string.md) Query: -``` sql +```sql CREATE TABLE tmp (str String) ENGINE = Log; INSERT INTO tmp (*) VALUES ('a'); SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0.{1..3}', currentDatabase(), 'tmp') GROUP BY queryID()); @@ -2491,7 +2491,7 @@ SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0 Result: -``` text +```text ┌─count()─┐ │ 1 │ └─────────┘ @@ -2504,7 +2504,7 @@ If a query is not distributed then constant value `0` is returned. **Syntax** -``` sql +```sql shardNum() ``` @@ -2520,7 +2520,7 @@ In the following example a configuration with two shards is used. The query is e Query: -``` sql +```sql CREATE TABLE shard_num_example (dummy UInt8) ENGINE=Distributed(test_cluster_two_shards_localhost, system, one, dummy); SELECT dummy, shardNum(), shardCount() FROM shard_num_example; @@ -2528,7 +2528,7 @@ SELECT dummy, shardNum(), shardCount() FROM shard_num_example; Result: -``` text +```text ┌─dummy─┬─shardNum()─┬─shardCount()─┐ │ 0 │ 2 │ 2 │ │ 0 │ 1 │ 2 │ @@ -2546,7 +2546,7 @@ If a query is not distributed then constant value `0` is returned. **Syntax** -``` sql +```sql shardCount() ``` @@ -2566,7 +2566,7 @@ Returns a string with the current OS kernel version. **Syntax** -``` sql +```sql getOSKernelVersion() ``` @@ -2584,13 +2584,13 @@ Type: [String](../../sql-reference/data-types/string.md). Query: -``` sql +```sql SELECT getOSKernelVersion(); ``` Result: -``` text +```text ┌─getOSKernelVersion()────┐ │ Linux 4.15.0-55-generic │ └─────────────────────────┘ @@ -2602,7 +2602,7 @@ Returns the uptime of the current ZooKeeper session in seconds. **Syntax** -``` sql +```sql zookeeperSessionUptime() ``` @@ -2620,13 +2620,13 @@ Type: [UInt32](../../sql-reference/data-types/int-uint.md). Query: -``` sql +```sql SELECT zookeeperSessionUptime(); ``` Result: -``` text +```text ┌─zookeeperSessionUptime()─┐ │ 286 │ └──────────────────────────┘ @@ -2638,7 +2638,7 @@ Generates random table structure in a format `column1_name column1_type, column2 **Syntax** -``` sql +```sql generateRandomStructure([number_of_columns, seed]) ``` @@ -2659,13 +2659,13 @@ Type: [String](../../sql-reference/data-types/string.md). Query: -``` sql +```sql SELECT generateRandomStructure() ``` Result: -``` text +```text ┌─generateRandomStructure()─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ │ c1 Decimal32(5), c2 Date, c3 Tuple(LowCardinality(String), Int128, UInt64, UInt16, UInt8, IPv6), c4 Array(UInt128), c5 UInt32, c6 IPv4, c7 Decimal256(64), c8 Decimal128(3), c9 UInt256, c10 UInt64, c11 DateTime │ └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ @@ -2673,13 +2673,13 @@ Result: Query: -``` sql +```sql SELECT generateRandomStructure(1) ``` Result: -``` text +```text ┌─generateRandomStructure(1)─┐ │ c1 Map(UInt256, UInt16) │ └────────────────────────────┘ @@ -2687,13 +2687,13 @@ Result: Query: -``` sql +```sql SELECT generateRandomStructure(NULL, 33) ``` Result: -``` text +```text ┌─generateRandomStructure(NULL, 33)─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ │ c1 DateTime, c2 Enum8('c2V0' = 0, 'c2V1' = 1, 'c2V2' = 2, 'c2V3' = 3), c3 LowCardinality(Nullable(FixedString(30))), c4 Int16, c5 Enum8('c5V0' = 0, 'c5V1' = 1, 'c5V2' = 2, 'c5V3' = 3), c6 Nullable(UInt8), c7 String, c8 Nested(e1 IPv4, e2 UInt8, e3 UInt16, e4 UInt16, e5 Int32, e6 Map(Date, Decimal256(70))) │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ @@ -2709,7 +2709,7 @@ Converts ClickHouse table structure to CapnProto schema. **Syntax** -``` sql +```sql structureToCapnProtoSchema(structure) ``` @@ -2720,7 +2720,7 @@ structureToCapnProtoSchema(structure) **Returned value** -- CapnProto schema +- CapnProto schema Type: [String](../../sql-reference/data-types/string.md). @@ -2728,13 +2728,13 @@ Type: [String](../../sql-reference/data-types/string.md). Query: -``` sql +```sql SELECT structureToCapnProtoSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB ``` Result: -``` text +```text @0xf96402dd754d0eb7; struct Message @@ -2747,13 +2747,13 @@ struct Message Query: -``` sql +```sql SELECT structureToCapnProtoSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB ``` Result: -``` text +```text @0xd1c8320fecad2b7f; struct Message @@ -2788,13 +2788,13 @@ struct Message Query: -``` sql +```sql SELECT structureToCapnProtoSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB ``` Result: -``` text +```text @0x96ab2d4ab133c6e1; struct Root @@ -2810,7 +2810,7 @@ Converts ClickHouse table structure to Protobuf schema. **Syntax** -``` sql +```sql structureToProtobufSchema(structure) ``` @@ -2829,13 +2829,13 @@ Type: [String](../../sql-reference/data-types/string.md). Query: -``` sql +```sql SELECT structureToProtobufSchema('column1 String, column2 UInt32, column3 Array(String)') FORMAT RawBLOB ``` Result: -``` text +```text syntax = "proto3"; message Message @@ -2848,13 +2848,13 @@ message Message Query: -``` sql +```sql SELECT structureToProtobufSchema('column1 Nullable(String), column2 Tuple(element1 UInt32, element2 Array(String)), column3 Map(String, String)') FORMAT RawBLOB ``` Result: -``` text +```text syntax = "proto3"; message Message @@ -2872,13 +2872,13 @@ message Message Query: -``` sql +```sql SELECT structureToProtobufSchema('column1 String, column2 UInt32', 'Root') FORMAT RawBLOB ``` Result: -``` text +```text syntax = "proto3"; message Root @@ -3050,7 +3050,7 @@ Calculates minimum required sample size for an A/B test comparing conversions (p **Syntax** -``` sql +```sql minSampleSizeConversion(baseline, mde, power, alpha) ``` @@ -3075,13 +3075,13 @@ A named [Tuple](../data-types/tuple.md) with 3 elements: The following query calculates the required sample size for an A/B test with baseline conversion of 25%, MDE of 3%, significance level of 5%, and the desired statistical power of 80%: -``` sql +```sql SELECT minSampleSizeConversion(0.25, 0.03, 0.80, 0.05) AS sample_size; ``` Result: -``` text +```text ┌─sample_size───────────────────┐ │ (3396.077603219163,0.22,0.28) │ └───────────────────────────────┘ @@ -3093,7 +3093,7 @@ Calculates minimum required sample size for an A/B test comparing means of a con **Syntax** -``` sql +```sql minSampleSizeContinous(baseline, sigma, mde, power, alpha) ``` @@ -3105,7 +3105,7 @@ Uses the formula described in [this article](https://towardsdatascience.com/requ - `baseline` — Baseline value of a metric. [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). - `sigma` — Baseline standard deviation of a metric. [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). -- `mde` — Minimum detectable effect (MDE) as percentage of the baseline value (e.g. for a baseline value 112.25 the MDE 0.03 means an expected change to 112.25 ± 112.25*0.03). [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). +- `mde` — Minimum detectable effect (MDE) as percentage of the baseline value (e.g. for a baseline value 112.25 the MDE 0.03 means an expected change to 112.25 ± 112.25\*0.03). [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). - `power` — Required statistical power of a test (1 - probability of Type II error). [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). - `alpha` — Required significance level of a test (probability of Type I error). [Integer](../data-types/int-uint.md) or [Float](../data-types/float.md). @@ -3121,13 +3121,13 @@ A named [Tuple](../data-types/tuple.md) with 3 elements: The following query calculates the required sample size for an A/B test on a metric with baseline value of 112.25, standard deviation of 21.1, MDE of 3%, significance level of 5%, and the desired statistical power of 80%: -``` sql +```sql SELECT minSampleSizeContinous(112.25, 21.1, 0.03, 0.80, 0.05) AS sample_size; ``` Result: -``` text +```text ┌─sample_size───────────────────────────┐ │ (616.2931945826209,108.8825,115.6175) │ └───────────────────────────────────────┘ From 8abb85948c4b401a61cdf6dc6cf33b7ac9df2279 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 12 Mar 2024 17:03:11 +0000 Subject: [PATCH 083/117] Fix flaky test and add separate to show unexpected behaviour --- .../01603_insert_select_too_many_parts.sql | 3 ++- ...t_select_too_many_parts_multithread.reference | 1 + ..._insert_select_too_many_parts_multithread.sql | 16 ++++++++++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.reference create mode 100644 tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql diff --git a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql index a56b680e212..eea52282cf4 100644 --- a/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql +++ b/tests/queries/0_stateless/01603_insert_select_too_many_parts.sql @@ -5,7 +5,8 @@ SYSTEM STOP MERGES too_many_parts; SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; -- exception is not thrown if threshold is exceeded when multi-block INSERT is already started. -INSERT INTO too_many_parts SELECT * FROM numbers(10); +-- Single thread is used as different threads check it separately https://github.com/ClickHouse/ClickHouse/issues/61158 +INSERT INTO too_many_parts SELECT * FROM numbers(10) SETTINGS max_insert_threads=1; SELECT count() FROM too_many_parts; -- exception is thrown if threshold is exceeded on new INSERT. diff --git a/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.reference b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.reference new file mode 100644 index 00000000000..29d6383b52c --- /dev/null +++ b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.reference @@ -0,0 +1 @@ +100 diff --git a/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql new file mode 100644 index 00000000000..00cf262add5 --- /dev/null +++ b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql @@ -0,0 +1,16 @@ +# Tags: disabled +# TODO: Fix parts_to_throw_insert logic for parallel MergeTreeSink onStart calls +DROP TABLE IF EXISTS too_many_parts; + +CREATE TABLE too_many_parts (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS parts_to_delay_insert = 5, parts_to_throw_insert = 5; + +SYSTEM STOP MERGES too_many_parts; +SET max_block_size = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0, max_threads=100, max_insert_threads=100; + +-- exception is not thrown if threshold is exceeded when multi-block INSERT is already started. +INSERT INTO too_many_parts SELECT * FROM numbers_mt(100); +SELECT count() FROM too_many_parts; + +INSERT INTO too_many_parts SELECT * FROM numbers_mt(10); -- { serverError 252 } + +DROP TABLE too_many_parts; From f3f1f611a45500f82c581e5ec34810e77a02cfdc Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Mar 2024 17:26:42 +0000 Subject: [PATCH 084/117] Better --- base/base/defines.h | 10 ++++++++-- src/Parsers/ASTQueryWithTableAndOutput.h | 1 + 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 1f02748633d..3308315b27f 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -108,16 +108,22 @@ { [[noreturn]] void abortOnFailedAssertion(const String & description); } - #define chassert(x) do { static_cast(x) ? void(0) : ::DB::abortOnFailedAssertion(#x); } while (0) + #define chassert_1(x, ...) do { static_cast(x) ? void(0) : ::DB::abortOnFailedAssertion(#x); } while (0) + #define chassert_2(x, comment, ...) do { static_cast(x) ? void(0) : ::DB::abortOnFailedAssertion(comment); } while (0) #define UNREACHABLE() abort() // clang-format off #else /// Here sizeof() trick is used to suppress unused warning for result, /// since simple "(void)x" will evaluate the expression, while /// "sizeof(!(x))" will not. - #define chassert(x) (void)sizeof(!(x)) + #define chassert_1(x, ...) (void)sizeof(!(x)) + #define chassert_2(x, comment, ...) (void)sizeof(!(x)) #define UNREACHABLE() __builtin_unreachable() #endif + #define CHASSERT_IMPL(_1,_2, N,...) N(_1, _2) + #define CHASSERT_IMPL_(tuple) CHASSERT_IMPL tuple + #define chassert(...) CHASSERT_IMPL_((__VA_ARGS__, chassert_2, chassert_1)) + #endif /// Macros for Clang Thread Safety Analysis (TSA). They can be safely ignored by other compilers. diff --git a/src/Parsers/ASTQueryWithTableAndOutput.h b/src/Parsers/ASTQueryWithTableAndOutput.h index 1b8621fb63b..358291d9aa8 100644 --- a/src/Parsers/ASTQueryWithTableAndOutput.h +++ b/src/Parsers/ASTQueryWithTableAndOutput.h @@ -61,6 +61,7 @@ protected: settings.ostr << '.'; } + chassert(table != nullptr, "Table is empty for the ASTQueryWithTableAndOutputImpl."); table->formatImpl(settings, state, frame); } }; From ca282ab4510c3f06f6433de8349b28ce3527f436 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Mar 2024 17:33:09 +0000 Subject: [PATCH 085/117] Better --- base/base/defines.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/base/defines.h b/base/base/defines.h index 3308315b27f..627c50c27d2 100644 --- a/base/base/defines.h +++ b/base/base/defines.h @@ -120,9 +120,9 @@ #define chassert_2(x, comment, ...) (void)sizeof(!(x)) #define UNREACHABLE() __builtin_unreachable() #endif - #define CHASSERT_IMPL(_1,_2, N,...) N(_1, _2) - #define CHASSERT_IMPL_(tuple) CHASSERT_IMPL tuple - #define chassert(...) CHASSERT_IMPL_((__VA_ARGS__, chassert_2, chassert_1)) + #define CHASSERT_DISPATCH(_1,_2, N,...) N(_1, _2) + #define CHASSERT_INVOKE(tuple) CHASSERT_DISPATCH tuple + #define chassert(...) CHASSERT_INVOKE((__VA_ARGS__, chassert_2, chassert_1)) #endif From ddd2d92d0fbd31dad7e091ce1d828eb65fc34338 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 12 Mar 2024 18:40:20 +0100 Subject: [PATCH 086/117] Update curl to curl with data race fix --- contrib/curl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/curl b/contrib/curl index 5ce164e0e92..1a05e833f8f 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit 5ce164e0e9290c96eb7d502173426c0a135ec008 +Subproject commit 1a05e833f8f7140628b27882b10525fd9ec4b873 From 296f7a1da2e5f5297fd26eb878071e4e60ac21fa Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Mar 2024 19:02:58 +0100 Subject: [PATCH 087/117] Fix --- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 4 ++++ src/IO/ReadWriteBufferFromHTTP.cpp | 2 +- src/IO/ReadWriteBufferFromHTTP.h | 3 +++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 4adb92cf5c8..7e942a6cf6f 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -86,6 +86,10 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lockhasNotFoundURL()) + return {}; + auto [it, inserted] = files.add(path, FileData::createDirectoryInfo(true)); if (!inserted) { diff --git a/src/IO/ReadWriteBufferFromHTTP.cpp b/src/IO/ReadWriteBufferFromHTTP.cpp index bcbec97537a..fdc8ef04d2e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.cpp +++ b/src/IO/ReadWriteBufferFromHTTP.cpp @@ -449,6 +449,7 @@ bool ReadWriteBufferFromHTTP::nextImpl() if (http_skip_not_found_url && e.getHTTPStatus() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND) { next_result = false; + has_not_found_url = true; return; } @@ -740,4 +741,3 @@ ReadWriteBufferFromHTTP::HTTPFileInfo ReadWriteBufferFromHTTP::parseFileInfo(con } } - diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index d0d792430c0..1b7437ea0c6 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -79,6 +79,7 @@ private: const bool use_external_buffer; const bool http_skip_not_found_url; + bool has_not_found_url = false; std::function out_stream_callback; @@ -183,6 +184,8 @@ public: std::optional tryGetLastModificationTime(); + bool hasNotFoundURL() const { return has_not_found_url; } + HTTPFileInfo getFileInfo(); static HTTPFileInfo parseFileInfo(const Poco::Net::HTTPResponse & response, size_t requested_range_begin); }; From c947484fe0d788dd46384b90987b59694f0a0b77 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Mar 2024 19:36:10 +0100 Subject: [PATCH 088/117] Fxi again --- tests/integration/test_disk_types/test.py | 15 ++++++++++++--- .../test_endpoint_macro_substitution/test.py | 17 ++++++++++++----- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index a53d073d30b..b9b8ef2010d 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -87,19 +87,28 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] for name, disk_type in list(disk_types.items()): - if disk_type != "S3": + if disk_type == "Local": assert ( node.query( "SELECT name FROM system.disks WHERE type='" + disk_type + "'" ) == name + "\n" ) - else: + elif disk_type == "S3": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + "SELECT name FROM system.disks WHERE object_storage_type='" + disk_type + "' ORDER BY name" ) == "disk_encrypted\ndisk_s3\n" ) + else: + assert ( + node.query( + "SELECT name FROM system.disks WHERE object_storage_type='" + + disk_type + + "'" + ) + == name + "\n" + ) diff --git a/tests/integration/test_endpoint_macro_substitution/test.py b/tests/integration/test_endpoint_macro_substitution/test.py index e161d8e82ff..ee72fb9b492 100644 --- a/tests/integration/test_endpoint_macro_substitution/test.py +++ b/tests/integration/test_endpoint_macro_substitution/test.py @@ -67,22 +67,29 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] - fs = HdfsClient(hosts=cluster.hdfs_ip) - for name, disk_type in list(disk_types.items()): - if disk_type != "S3": + if disk_type == "Local": assert ( node.query( "SELECT name FROM system.disks WHERE type='" + disk_type + "'" ) == name + "\n" ) - else: + elif disk_type == "S3": assert ( node.query( - "SELECT name FROM system.disks WHERE type='" + "SELECT name FROM system.disks WHERE object_storage_type='" + disk_type + "' ORDER BY name" ) == "disk_encrypted\ndisk_s3\n" ) + else: + assert ( + node.query( + "SELECT name FROM system.disks WHERE object_storage_type='" + + disk_type + + "'" + ) + == name + "\n" + ) From ec80cc475ed8c0c3126aceb5db35ff20b9295451 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 12 Mar 2024 19:58:23 +0100 Subject: [PATCH 089/117] Fix formatting of overlay database --- src/Databases/DatabasesOverlay.cpp | 4 +++- .../0_stateless/03009_format_show_database.reference | 2 ++ tests/queries/0_stateless/03009_format_show_database.sh | 8 ++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03009_format_show_database.reference create mode 100755 tests/queries/0_stateless/03009_format_show_database.sh diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 8cea3441698..c8705254e73 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -149,7 +149,9 @@ ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr */ ASTPtr DatabasesOverlay::getCreateDatabaseQuery() const { - return std::make_shared(); + auto query = std::make_shared(); + query->setDatabase(getDatabaseName()); + return query; } String DatabasesOverlay::getTableDataPath(const String & table_name) const diff --git a/tests/queries/0_stateless/03009_format_show_database.reference b/tests/queries/0_stateless/03009_format_show_database.reference new file mode 100644 index 00000000000..83cfd4c1a68 --- /dev/null +++ b/tests/queries/0_stateless/03009_format_show_database.reference @@ -0,0 +1,2 @@ +CREATE DATABASE default +UNKNOWN_DATABASE diff --git a/tests/queries/0_stateless/03009_format_show_database.sh b/tests/queries/0_stateless/03009_format_show_database.sh new file mode 100755 index 00000000000..7f33ad7b1e1 --- /dev/null +++ b/tests/queries/0_stateless/03009_format_show_database.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "show database default" +$CLICKHOUSE_LOCAL -q "show database default2" 2>&1 | grep -o 'UNKNOWN_DATABASE' From 15484bedadf62d05fdcaea842ce9d86bdafe57f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 12 Mar 2024 21:03:26 +0100 Subject: [PATCH 090/117] Fix clone for ASTQueryWithTableAndOutputImpl derived classes --- src/Parsers/TablePropertiesQueriesASTs.h | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/Parsers/TablePropertiesQueriesASTs.h b/src/Parsers/TablePropertiesQueriesASTs.h index e8e4bd8adb3..81ad975aa37 100644 --- a/src/Parsers/TablePropertiesQueriesASTs.h +++ b/src/Parsers/TablePropertiesQueriesASTs.h @@ -85,6 +85,15 @@ using ASTShowCreateDictionaryQuery = ASTQueryWithTableAndOutputImpl { +public: + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + res->children.clear(); + cloneTableOptions(*res); + return res; + } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { @@ -98,6 +107,15 @@ protected: class ASTShowCreateDatabaseQuery : public ASTQueryWithTableAndOutputImpl { +public: + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + res->children.clear(); + cloneTableOptions(*res); + return res; + } + protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { From 460692ac74d68a38bad1b4786b3bbabd377189e6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 13 Mar 2024 00:36:23 -0400 Subject: [PATCH 091/117] Update docs/en/sql-reference/functions/other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index df443eec0de..288905c83da 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -360,7 +360,7 @@ sleepEachRow(seconds) **Arguments** -- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution for each row in the result set. It can be a floating-point value to specify fractional seconds. +- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution for each row in the result set to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. **Returned value** From 87846b2c98205f95db69d99ede102e600615c54a Mon Sep 17 00:00:00 2001 From: Mikhail Koviazin Date: Wed, 13 Mar 2024 09:14:55 +0200 Subject: [PATCH 092/117] Added more tests --- .../0_stateless/00662_has_nullable.reference | 12 +++++++ .../0_stateless/00662_has_nullable.sql | 32 +++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/tests/queries/0_stateless/00662_has_nullable.reference b/tests/queries/0_stateless/00662_has_nullable.reference index 97da99d655e..1ac93f25a39 100644 --- a/tests/queries/0_stateless/00662_has_nullable.reference +++ b/tests/queries/0_stateless/00662_has_nullable.reference @@ -1,2 +1,14 @@ +Nullable(UInt64), non-null array 1 1 \N 0 +Non-nullable UInt64, nullable array +0 0 +1 1 +2 1 +Nullable(UInt64), nullable array +0 0 +\N 1 +1 1 +All NULLs +0 0 +\N 1 diff --git a/tests/queries/0_stateless/00662_has_nullable.sql b/tests/queries/0_stateless/00662_has_nullable.sql index 0d395871d9f..3fd3bd46baa 100644 --- a/tests/queries/0_stateless/00662_has_nullable.sql +++ b/tests/queries/0_stateless/00662_has_nullable.sql @@ -1,7 +1,39 @@ DROP TABLE IF EXISTS 00662_has_nullable; + +SELECT 'Nullable(UInt64), non-null array'; CREATE TABLE 00662_has_nullable(a Nullable(UInt64)) ENGINE = Memory; INSERT INTO 00662_has_nullable VALUES (1), (Null); SELECT a, has([0, 1], a) FROM 00662_has_nullable; DROP TABLE 00662_has_nullable; + +-------------------------------------------------------------------------------- + +SELECT 'Non-nullable UInt64, nullable array'; +CREATE TABLE 00662_has_nullable(a UInt64) ENGINE = Memory; + +INSERT INTO 00662_has_nullable VALUES (0), (1), (2); +SELECT a, has([NULL, 1, 2], a) FROM 00662_has_nullable; + +DROP TABLE 00662_has_nullable; + +-------------------------------------------------------------------------------- + +SELECT 'Nullable(UInt64), nullable array'; +CREATE TABLE 00662_has_nullable(a Nullable(UInt64)) ENGINE = Memory; + +INSERT INTO 00662_has_nullable VALUES (0), (Null), (1); +SELECT a, has([NULL, 1, 2], a) FROM 00662_has_nullable; + +DROP TABLE 00662_has_nullable; + +-------------------------------------------------------------------------------- + +SELECT 'All NULLs'; +CREATE TABLE 00662_has_nullable(a Nullable(UInt64)) ENGINE = Memory; + +INSERT INTO 00662_has_nullable VALUES (0), (Null); +SELECT a, has([NULL, NULL], a) FROM 00662_has_nullable; + +DROP TABLE 00662_has_nullable; From c0bfafa203091abbe2f5af6d1253249fc432ad64 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 13 Mar 2024 10:48:02 +0000 Subject: [PATCH 093/117] fix tags --- .../02997_insert_select_too_many_parts_multithread.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql index 00cf262add5..2dfc8094115 100644 --- a/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql +++ b/tests/queries/0_stateless/02997_insert_select_too_many_parts_multithread.sql @@ -1,5 +1,5 @@ -# Tags: disabled -# TODO: Fix parts_to_throw_insert logic for parallel MergeTreeSink onStart calls +-- Tags: disabled +-- TODO: Fix parts_to_throw_insert logic for parallel MergeTreeSink onStart calls DROP TABLE IF EXISTS too_many_parts; CREATE TABLE too_many_parts (x UInt64) ENGINE = MergeTree ORDER BY tuple() SETTINGS parts_to_delay_insert = 5, parts_to_throw_insert = 5; From a704ea510a2c180d46ba89031915624619d3b74c Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 13 Mar 2024 12:03:42 +0100 Subject: [PATCH 094/117] Fix `test_placement_info` (#61057) * Fix test * Fix last minute changes * Automatic style fix --------- Co-authored-by: robot-clickhouse --- tests/integration/helpers/mock_servers.py | 2 +- .../configs/imds_bootstrap.xml | 9 +++++++++ tests/integration/test_placement_info/test.py | 18 ++++++++++-------- 3 files changed, 20 insertions(+), 9 deletions(-) create mode 100644 tests/integration/test_placement_info/configs/imds_bootstrap.xml diff --git a/tests/integration/helpers/mock_servers.py b/tests/integration/helpers/mock_servers.py index e4655ffeeaf..f2181d85e12 100644 --- a/tests/integration/helpers/mock_servers.py +++ b/tests/integration/helpers/mock_servers.py @@ -33,7 +33,7 @@ def start_mock_servers(cluster, script_dir, mocks, timeout=100): cluster.exec_in_container( container_id, - ["python", server_name, str(port)], + ["python3", server_name, str(port)], detach=True, ) diff --git a/tests/integration/test_placement_info/configs/imds_bootstrap.xml b/tests/integration/test_placement_info/configs/imds_bootstrap.xml new file mode 100644 index 00000000000..5b2a77e0663 --- /dev/null +++ b/tests/integration/test_placement_info/configs/imds_bootstrap.xml @@ -0,0 +1,9 @@ + + + 1 + + + 0 + ci-placeholder + + diff --git a/tests/integration/test_placement_info/test.py b/tests/integration/test_placement_info/test.py index 1b93a3eae0b..32fd2fa57d7 100644 --- a/tests/integration/test_placement_info/test.py +++ b/tests/integration/test_placement_info/test.py @@ -2,16 +2,14 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.mock_servers import start_mock_servers import os -import time -METADATA_SERVER_HOSTNAME = "resolver" +METADATA_SERVER_HOSTNAME = "node_imds" METADATA_SERVER_PORT = 8080 cluster = ClickHouseCluster(__file__) node_imds = cluster.add_instance( "node_imds", - with_minio=True, - main_configs=["configs/imds.xml"], + main_configs=["configs/imds_bootstrap.xml"], env_variables={ "AWS_EC2_METADATA_SERVICE_ENDPOINT": f"http://{METADATA_SERVER_HOSTNAME}:{METADATA_SERVER_PORT}", }, @@ -32,10 +30,10 @@ node_missing_value = cluster.add_instance( ) -def start_metadata_server(): +def start_metadata_server(started_cluster): script_dir = os.path.join(os.path.dirname(__file__), "metadata_servers") start_mock_servers( - cluster, + started_cluster, script_dir, [ ( @@ -51,13 +49,17 @@ def start_metadata_server(): def start_cluster(): try: cluster.start() - start_metadata_server() - yield + start_metadata_server(cluster) + yield cluster finally: cluster.shutdown() def test_placement_info_from_imds(): + with open(os.path.join(os.path.dirname(__file__), "configs/imds.xml"), "r") as f: + node_imds.replace_config( + "/etc/clickhouse-server/config.d/imds_bootstrap.xml", f.read() + ) node_imds.stop_clickhouse(kill=True) node_imds.start_clickhouse() From 0353121dccb87403ff08334137fd9cecbb8953f1 Mon Sep 17 00:00:00 2001 From: peter279k Date: Wed, 13 Mar 2024 19:17:27 +0800 Subject: [PATCH 095/117] Improve related NULL functions usage --- docs/en/sql-reference/functions/functions-for-nulls.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 91c04cfded3..e73d6c899e7 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -14,7 +14,7 @@ Returns whether the argument is [NULL](../../sql-reference/syntax.md#null). isNull(x) ``` -Alias: `ISNULL`. +Alias: `IS NULL`. **Arguments** @@ -58,6 +58,8 @@ Returns whether the argument is not [NULL](../../sql-reference/syntax.md#null-li isNotNull(x) ``` +Alias: `IS NOT NULL`. + **Arguments:** - `x` — A value of non-compound data type. @@ -100,6 +102,8 @@ Returns whether the argument is 0 (zero) or [NULL](../../sql-reference/syntax.md isZeroOrNull(x) ``` +Alias: `x = 0 OR x IS NULL`. + **Arguments:** - `x` — A value of non-compound data type. From 3931351ec4806769048d2638f54323f1ae89e056 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Mar 2024 12:36:47 +0100 Subject: [PATCH 096/117] Pass timeout through setting --- src/Core/Settings.h | 1 + src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 5 +++-- src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp | 10 +++++++--- src/Disks/IO/CachedOnDiskWriteBufferFromFile.h | 15 +++++++++------ src/IO/ReadSettings.h | 1 + src/IO/WriteSettings.h | 1 + src/Interpreters/Cache/FileCache.cpp | 5 +++-- src/Interpreters/Cache/FileCache.h | 3 ++- src/Interpreters/Cache/FileCache_fwd.h | 1 - src/Interpreters/Cache/FileSegment.cpp | 4 ++-- src/Interpreters/Cache/FileSegment.h | 2 +- src/Interpreters/Cache/Metadata.cpp | 6 +++++- .../Cache/WriteBufferToFileSegment.cpp | 8 +++++++- src/Interpreters/Cache/WriteBufferToFileSegment.h | 2 ++ src/Interpreters/Context.cpp | 2 ++ 15 files changed, 46 insertions(+), 20 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d70a6cf51c5..7ba335099e6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -778,6 +778,7 @@ class IColumn; M(UInt64, filesystem_cache_max_download_size, (128UL * 1024 * 1024 * 1024), "Max remote filesystem cache size that can be downloaded by a single query", 0) \ M(Bool, throw_on_error_from_cache_on_write_operations, false, "Ignore error from cache when caching on write operations (INSERT, merges)", 0) \ M(UInt64, filesystem_cache_segments_batch_size, 20, "Limit on size of a single batch of file segments that a read buffer can request from cache. Too low value will lead to excessive requests to cache, too large may slow down eviction from cache", 0) \ + M(UInt64, filesystem_cache_reserve_space_wait_lock_timeout_milliseconds, 1000, "Wait time to lock cache for sapce reservation in filesystem cache", 0) \ \ M(Bool, use_page_cache_for_disks_without_file_cache, false, "Use userspace page cache for remote disks that don't have filesystem cache enabled.", 0) \ M(Bool, read_from_page_cache_if_exists_otherwise_bypass_cache, false, "Use userspace page cache in passive mode, similar to read_from_filesystem_cache_if_exists_otherwise_bypass_cache.", 0) \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 47ee5858562..1e108b481ee 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -637,7 +637,8 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); - bool continue_predownload = file_segment.reserve(current_predownload_size); + bool continue_predownload = file_segment.reserve( + current_predownload_size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); if (continue_predownload) { LOG_TEST(log, "Left to predownload: {}, buffer size: {}", bytes_to_predownload, current_impl_buffer_size); @@ -992,7 +993,7 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() { chassert(file_offset_of_buffer_end + size - 1 <= file_segment.range().right); - bool success = file_segment.reserve(size); + bool success = file_segment.reserve(size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); if (success) { chassert(file_segment.getCurrentWriteOffset() == static_cast(implementation_buffer->getPosition())); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index faed55de713..f4e309f461e 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -26,16 +26,18 @@ FileSegmentRangeWriter::FileSegmentRangeWriter( FileCache * cache_, const FileSegment::Key & key_, const FileCacheUserInfo & user_, + size_t reserve_space_lock_wait_timeout_milliseconds_, std::shared_ptr cache_log_, const String & query_id_, const String & source_path_) : cache(cache_) , key(key_) + , user(user_) + , reserve_space_lock_wait_timeout_milliseconds(reserve_space_lock_wait_timeout_milliseconds_) , log(getLogger("FileSegmentRangeWriter")) , cache_log(cache_log_) , query_id(query_id_) , source_path(source_path_) - , user(user_) { } @@ -89,7 +91,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset size_t size_to_write = std::min(available_size, size); - bool reserved = file_segment->reserve(size_to_write); + bool reserved = file_segment->reserve(size_to_write, reserve_space_lock_wait_timeout_milliseconds); if (!reserved) { appendFilesystemCacheLog(*file_segment); @@ -211,6 +213,7 @@ CachedOnDiskWriteBufferFromFile::CachedOnDiskWriteBufferFromFile( , key(key_) , query_id(query_id_) , user(user_) + , reserve_space_lock_wait_timeout_milliseconds(settings_.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds) , throw_on_error_from_cache(settings_.throw_on_error_from_cache) , cache_log(!query_id_.empty() && settings_.enable_filesystem_cache_log ? cache_log_ : nullptr) { @@ -251,7 +254,8 @@ void CachedOnDiskWriteBufferFromFile::cacheData(char * data, size_t size, bool t if (!cache_writer) { - cache_writer = std::make_unique(cache.get(), key, user, cache_log, query_id, source_path); + cache_writer = std::make_unique( + cache.get(), key, user, reserve_space_lock_wait_timeout_milliseconds, cache_log, query_id, source_path); } Stopwatch watch(CLOCK_MONOTONIC); diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 59e0c76ca3d..ad4f6b5916d 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -30,6 +30,7 @@ public: FileCache * cache_, const FileSegment::Key & key_, const FileCacheUserInfo & user_, + size_t reserve_space_lock_wait_timeout_milliseconds_, std::shared_ptr cache_log_, const String & query_id_, const String & source_path_); @@ -52,13 +53,14 @@ private: void completeFileSegment(); FileCache * cache; - FileSegment::Key key; + const FileSegment::Key key; + const FileCacheUserInfo user; + const size_t reserve_space_lock_wait_timeout_milliseconds; LoggerPtr log; std::shared_ptr cache_log; - String query_id; - String source_path; - FileCacheUserInfo user; + const String query_id; + const String source_path; FileSegmentsHolderPtr file_segments; @@ -99,11 +101,12 @@ private: String source_path; FileCacheKey key; - size_t current_download_offset = 0; const String query_id; const FileCacheUserInfo user; + const size_t reserve_space_lock_wait_timeout_milliseconds; + const bool throw_on_error_from_cache; - bool throw_on_error_from_cache; + size_t current_download_offset = 0; bool cache_in_error_state_or_disabled = false; std::unique_ptr cache_writer; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index c0a63bf51b1..6a0cac35878 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -100,6 +100,7 @@ struct ReadSettings bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false; bool enable_filesystem_cache_log = false; size_t filesystem_cache_segments_batch_size = 20; + size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; bool use_page_cache_for_disks_without_file_cache = false; bool read_from_page_cache_if_exists_otherwise_bypass_cache = false; diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index fcadf34f021..7d36677b468 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -20,6 +20,7 @@ struct WriteSettings bool enable_filesystem_cache_on_write_operations = false; bool enable_filesystem_cache_log = false; bool throw_on_error_from_cache = false; + size_t filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = 1000; bool s3_allow_parallel_part_upload = true; diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 5650b9ce44e..ea40ffcfa3c 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -777,12 +777,13 @@ bool FileCache::tryReserve( FileSegment & file_segment, const size_t size, FileCacheReserveStat & reserve_stat, - const UserInfo & user) + const UserInfo & user, + size_t lock_wait_timeout_milliseconds) { ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheReserveMicroseconds); assertInitialized(); - auto cache_lock = tryLockCache(std::chrono::milliseconds(FILECACHE_TRY_RESERVE_LOCK_TIMEOUT_MILLISECONDS)); + auto cache_lock = tryLockCache(std::chrono::milliseconds(lock_wait_timeout_milliseconds)); if (!cache_lock) { ProfileEvents::increment(ProfileEvents::FilesystemCacheFailToReserveSpaceBecauseOfLockContention); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 7434b2ac78a..007c4fd9483 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -161,7 +161,8 @@ public: FileSegment & file_segment, size_t size, FileCacheReserveStat & stat, - const UserInfo & user); + const UserInfo & user, + size_t lock_wait_timeout_milliseconds); std::vector getFileSegmentInfos(const UserID & user_id); diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index eaed279e7fd..06261b19db7 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -12,7 +12,6 @@ static constexpr int FILECACHE_DEFAULT_LOAD_METADATA_THREADS = 16; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; -static constexpr size_t FILECACHE_TRY_RESERVE_LOCK_TIMEOUT_MILLISECONDS = 1000; /// 1 sec. class FileCache; using FileCachePtr = std::shared_ptr; diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 6b2d4a4bec8..9ec2b090dc7 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -497,7 +497,7 @@ LockedKeyPtr FileSegment::lockKeyMetadata(bool assert_exists) const return metadata->tryLock(); } -bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve_stat) +bool FileSegment::reserve(size_t size_to_reserve, size_t lock_wait_timeout_milliseconds, FileCacheReserveStat * reserve_stat) { if (!size_to_reserve) throw Exception(ErrorCodes::LOGICAL_ERROR, "Zero space reservation is not allowed"); @@ -549,7 +549,7 @@ bool FileSegment::reserve(size_t size_to_reserve, FileCacheReserveStat * reserve if (!reserve_stat) reserve_stat = &dummy_stat; - bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user); + bool reserved = cache->tryReserve(*this, size_to_reserve, *reserve_stat, getKeyMetadata()->user, lock_wait_timeout_milliseconds); if (!reserved) setDownloadFailedUnlocked(lockFileSegment()); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index ea97a6b0157..c34ee064345 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -199,7 +199,7 @@ public: /// Try to reserve exactly `size` bytes (in addition to the getDownloadedSize() bytes already downloaded). /// Returns true if reservation was successful, false otherwise. - bool reserve(size_t size_to_reserve, FileCacheReserveStat * reserve_stat = nullptr); + bool reserve(size_t size_to_reserve, size_t lock_wait_timeout_milliseconds, FileCacheReserveStat * reserve_stat = nullptr); /// Write data into reserved space. void write(const char * from, size_t size, size_t offset); diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 727f2762cca..b79605622b6 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -693,6 +694,9 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalset(memory->data(), memory->size()); } + const auto reserve_space_lock_wait_timeout_milliseconds = + Context::getGlobalContextInstance()->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; + size_t offset = file_segment.getCurrentWriteOffset(); if (offset != static_cast(reader->getPosition())) reader->seek(offset, SEEK_SET); @@ -701,7 +705,7 @@ void CacheMetadata::downloadImpl(FileSegment & file_segment, std::optionalavailable(); - if (!file_segment.reserve(size)) + if (!file_segment.reserve(size, reserve_space_lock_wait_timeout_milliseconds)) { LOG_TEST( log, "Failed to reserve space during background download " diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 7cd4e2d6e8d..759135722dc 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -32,6 +33,11 @@ WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment , file_segment(&segment_holder_->front()) , segment_holder(std::move(segment_holder_)) { + auto query_context = CurrentThread::getQueryContext(); + if (query_context) + reserve_space_lock_wait_timeout_milliseconds = query_context->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; + else + reserve_space_lock_wait_timeout_milliseconds = Context::getGlobalContextInstance()->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; } /// If it throws an exception, the file segment will be incomplete, so you should not use it in the future. @@ -49,7 +55,7 @@ void WriteBufferToFileSegment::nextImpl() FileCacheReserveStat reserve_stat; /// In case of an error, we don't need to finalize the file segment /// because it will be deleted soon and completed in the holder's destructor. - bool ok = file_segment->reserve(bytes_to_write, &reserve_stat); + bool ok = file_segment->reserve(bytes_to_write, reserve_space_lock_wait_timeout_milliseconds, &reserve_stat); if (!ok) { diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.h b/src/Interpreters/Cache/WriteBufferToFileSegment.h index feb33472513..bff340d79b3 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.h +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.h @@ -28,6 +28,8 @@ private: /// Empty if file_segment is not owned by this WriteBufferToFileSegment FileSegmentsHolderPtr segment_holder; + + size_t reserve_space_lock_wait_timeout_milliseconds; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d658fbe9920..6a0657a842c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5166,6 +5166,7 @@ ReadSettings Context::getReadSettings() const res.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache; res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log; res.filesystem_cache_segments_batch_size = settings.filesystem_cache_segments_batch_size; + res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; res.filesystem_cache_max_download_size = settings.filesystem_cache_max_download_size; res.skip_download_if_exceeds_query_cache = settings.skip_download_if_exceeds_query_cache; @@ -5214,6 +5215,7 @@ WriteSettings Context::getWriteSettings() const res.enable_filesystem_cache_on_write_operations = settings.enable_filesystem_cache_on_write_operations; res.enable_filesystem_cache_log = settings.enable_filesystem_cache_log; res.throw_on_error_from_cache = settings.throw_on_error_from_cache_on_write_operations; + res.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds = settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; res.s3_allow_parallel_part_upload = settings.s3_allow_parallel_part_upload; From 9da03607282b8fc338be4356322c67229686fd39 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 13 Mar 2024 13:10:48 +0100 Subject: [PATCH 097/117] Reload CI From 457578627103b0eb0028f51b739c3911de278bf4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Mar 2024 12:38:17 +0000 Subject: [PATCH 098/117] Fix 01952_optimize_distributed_group_by_sharding_key with analyzer. --- ...istributed_group_by_sharding_key.reference | 78 +++++++++++++++++++ ...mize_distributed_group_by_sharding_key.sql | 14 ++++ 2 files changed, 92 insertions(+) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index ddfa6929d69..212dd348edb 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -71,3 +71,81 @@ Expression (Projection) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) ReadFromSystemNumbers ReadFromRemote (Read from remote replica) +set allow_experimental_analyzer = 1; +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Project names) + Distinct (DISTINCT) + Union + Distinct (Preliminary DISTINCT) + Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +Union + Expression (Project names) + Distinct (DISTINCT) + Distinct (Preliminary DISTINCT) + Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +Expression (Project names) + LimitBy + Union + Expression (Before LIMIT BY) + LimitBy + Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))) + ReadFromSystemNumbers + Expression + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +Union + Expression (Project names) + LimitBy + Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Project names) + Distinct (DISTINCT) + Sorting (Merge sorted streams for ORDER BY, without aggregation) + Union + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Distinct (Preliminary DISTINCT) + Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Project names) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) + Union + Distinct (DISTINCT) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + Distinct (Preliminary DISTINCT) + Expression ((Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +Expression (Project names) + LimitBy + Expression (Before LIMIT BY) + Sorting (Merge sorted streams for ORDER BY, without aggregation) + Union + LimitBy + Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) [lifted up part])) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Project names) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) + Union + LimitBy + Expression ((Before LIMIT BY + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers))))))) [lifted up part])) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + Change column names to column identifiers)))))))) + ReadFromSystemNumbers + ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql index 74b55b95315..adf55a9dd7f 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.sql @@ -4,6 +4,8 @@ set optimize_skip_unused_shards=1; set optimize_distributed_group_by_sharding_key=1; set prefer_localhost_replica=1; +set allow_experimental_analyzer = 0; + -- { echo } explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized @@ -14,3 +16,15 @@ explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized + +set allow_experimental_analyzer = 1; + +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized + +explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized From 4954bde599dd1bdcdf56957e17b0b9a661aa17f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:38:35 +0100 Subject: [PATCH 099/117] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Johnny <9611008+johnnymatthews@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f9c3f91a12b..a9b7cc9566d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -615,7 +615,9 @@ Assumes that the string contains valid UTF-8 encoded text. If this assumption is **Example** ```sql -SELECT 'database' AS string, substringUTF8(string, 5), substringUTF8(string, 5, 1) +SELECT 'Täglich grüßt das Murmeltier.' AS str, + substringUTF8(str, 9), + substringUTF8(str, 9, 5) ``` ```response From b4953f35b4a8ca3eca816557c080ff612062b482 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:39:03 +0100 Subject: [PATCH 100/117] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Johnny <9611008+johnnymatthews@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a9b7cc9566d..25a0c7e38d8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -686,7 +686,7 @@ Assumes that the string contains valid UTF-8 encoded text. If this assumption is **Example** ```sql -SELECT substringIndexUTF8('www.clickhouse.com', '.', 2) +SELECT substringIndexUTF8('www.straßen-in-europa.de', '.', 2) ``` ```response From 6ca4fc26f4bca8f787b3a575d5496ffd75ee0c55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:39:14 +0100 Subject: [PATCH 101/117] Update docs/en/sql-reference/functions/string-functions.md Co-authored-by: Johnny <9611008+johnnymatthews@users.noreply.github.com> --- docs/en/sql-reference/functions/string-functions.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 25a0c7e38d8..01a583e1713 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -621,9 +621,7 @@ SELECT 'Täglich grüßt das Murmeltier.' AS str, ``` ```response -┌─string───┬─substringUTF8('database', 5)─┬─substringUTF8('database', 5, 1)─┐ -│ database │ base │ b │ -└──────────┴──────────────────────────────┴─────────────────────────────────┘ +Täglich grüßt das Murmeltier. grüßt das Murmeltier. grüßt ``` ## substringIndex From 1e536251a20a0fdbac08b0a99e420a8e74886bcd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:41:56 +0100 Subject: [PATCH 102/117] Update string-functions.md --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 01a583e1713..b4e2adbed3c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -688,7 +688,7 @@ SELECT substringIndexUTF8('www.straßen-in-europa.de', '.', 2) ``` ```response -www.clickhouse +www.straßen-in-europa ``` ## appendTrailingCharIfAbsent From e5e632ec3362d2106adca2e02ae2a4ea1862ee3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Mar 2024 12:43:10 +0000 Subject: [PATCH 103/117] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index dbd216ea7be..42aa579658e 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -9,7 +9,6 @@ 01747_join_view_filter_dictionary 01761_cast_to_enum_nullable 01925_join_materialized_columns -01952_optimize_distributed_group_by_sharding_key 02354_annoy # Check after constants refactoring 02901_parallel_replicas_rollup From d8c5008280aaf19bd481d436099afd89019a81c4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:07:14 +0000 Subject: [PATCH 104/117] Follow up to #61258 --- .../functions/other-functions.md | 6 ++--- src/Functions/sleep.h | 25 ++++--------------- 2 files changed, 8 insertions(+), 23 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 288905c83da..e7fca31483a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -298,7 +298,7 @@ Full columns and constants are represented differently in memory. Functions usua Accepts any arguments, including `NULL` and does nothing. Always returns 0. The argument is internally still evaluated. Useful e.g. for benchmarks. -## sleep(seconds) +## sleep Used to introduce a delay or pause in the execution of a query. It is primarily used for testing and debugging purposes. @@ -310,7 +310,7 @@ sleep(seconds) **Arguments** -- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. +- `seconds`: [UInt*](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md) The number of seconds to pause the query execution to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. **Returned value** @@ -360,7 +360,7 @@ sleepEachRow(seconds) **Arguments** -- `seconds`: [Int](../../sql-reference/data-types/int-uint.md) The number of seconds to pause the query execution for each row in the result set to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. +- `seconds`: [UInt*](../../sql-reference/data-types/int-uint.md) or [Float*](../../sql-reference/data-types/float.md) The number of seconds to pause the query execution for each row in the result set to a maximum of 3 seconds. It can be a floating-point value to specify fractional seconds. **Returned value** diff --git a/src/Functions/sleep.h b/src/Functions/sleep.h index 73d58ca6b5b..84f08dd5440 100644 --- a/src/Functions/sleep.h +++ b/src/Functions/sleep.h @@ -62,32 +62,17 @@ public: { } - /// Get the name of the function. - String getName() const override - { - return name; - } - - /// Do not sleep during query analysis. - bool isSuitableForConstantFolding() const override - { - return false; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - + String getName() const override { return name; } + bool isSuitableForConstantFolding() const override { return false; } /// Do not sleep during query analysis. + size_t getNumberOfArguments() const override { return 1; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { WhichDataType which(arguments[0]); - if (!which.isFloat() - && !which.isNativeUInt()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}, expected Float64", + if (!which.isFloat() && !which.isNativeUInt()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}, expected UInt* or Float*", arguments[0]->getName(), getName()); return std::make_shared(); From e6af636a549f808730c87ab69a6b76531d3dbc95 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 13 Mar 2024 14:07:49 +0100 Subject: [PATCH 105/117] fix data race in poco tcp server --- base/poco/Net/src/TCPServerDispatcher.cpp | 4 +++- src/Common/tests/gtest_connection_pool.cpp | 11 +++-------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/base/poco/Net/src/TCPServerDispatcher.cpp b/base/poco/Net/src/TCPServerDispatcher.cpp index 20a1ffe1b4f..7f9f9a20ee7 100644 --- a/base/poco/Net/src/TCPServerDispatcher.cpp +++ b/base/poco/Net/src/TCPServerDispatcher.cpp @@ -93,7 +93,7 @@ void TCPServerDispatcher::release() void TCPServerDispatcher::run() { - AutoPtr guard(this, true); // ensure object stays alive + AutoPtr guard(this); // ensure object stays alive int idleTime = (int) _pParams->getThreadIdleTime().totalMilliseconds(); @@ -149,11 +149,13 @@ void TCPServerDispatcher::enqueue(const StreamSocket& socket) { try { + this->duplicate(); _threadPool.startWithPriority(_pParams->getThreadPriority(), *this, threadName); ++_currentThreads; } catch (Poco::Exception& exc) { + this->release(); ++_refusedConnections; std::cerr << "Got exception while starting thread for connection. Error code: " << exc.code() << ", message: '" << exc.displayText() << "'" << std::endl; diff --git a/src/Common/tests/gtest_connection_pool.cpp b/src/Common/tests/gtest_connection_pool.cpp index c271cc0e2ec..dcc3c11fd52 100644 --- a/src/Common/tests/gtest_connection_pool.cpp +++ b/src/Common/tests/gtest_connection_pool.cpp @@ -123,17 +123,15 @@ protected: std::string getServerUrl() const { - return "http://" + server_data.socket->address().toString(); + return "http://" + server_data.server->socket().address().toString(); } void startServer() { server_data.reset(); - server_data.params = new Poco::Net::HTTPServerParams(); - server_data.socket = std::make_unique(server_data.port); server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive); server_data.server = std::make_unique( - server_data.handler_factory, *server_data.socket, server_data.params); + server_data.handler_factory, server_data.port); server_data.server->start(); } @@ -155,8 +153,7 @@ protected: { // just some port to avoid collisions with others tests UInt16 port = 9871; - Poco::Net::HTTPServerParams::Ptr params; - std::unique_ptr socket; + HTTPRequestHandlerFactory::Ptr handler_factory; std::unique_ptr server; @@ -171,8 +168,6 @@ protected: server = nullptr; handler_factory = nullptr; - socket = nullptr; - params = nullptr; } ~ServerData() { From fefee44540bd029eb2d354706f61a1d96ed0e272 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Mar 2024 14:27:47 +0100 Subject: [PATCH 106/117] Update settings changes history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index e680c02671a..d7b0669f64f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,7 @@ static std::map sett {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, + {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From 9bb71291d50d29dd0c401580402adc12290224bb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Mar 2024 14:42:57 +0100 Subject: [PATCH 107/117] Fix unit test --- src/Interpreters/tests/gtest_filecache.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/tests/gtest_filecache.cpp b/src/Interpreters/tests/gtest_filecache.cpp index b596ccb0285..2679d1b8d18 100644 --- a/src/Interpreters/tests/gtest_filecache.cpp +++ b/src/Interpreters/tests/gtest_filecache.cpp @@ -245,7 +245,7 @@ void download(FileSegment & file_segment) ASSERT_EQ(file_segment.state(), State::DOWNLOADING); ASSERT_EQ(file_segment.getDownloadedSize(), 0); - ASSERT_TRUE(file_segment.reserve(file_segment.range().size())); + ASSERT_TRUE(file_segment.reserve(file_segment.range().size(), 1000)); download(cache_base_path, file_segment); ASSERT_EQ(file_segment.state(), State::DOWNLOADING); @@ -257,7 +257,7 @@ void assertDownloadFails(FileSegment & file_segment) { ASSERT_EQ(file_segment.getOrSetDownloader(), FileSegment::getCallerId()); ASSERT_EQ(file_segment.getDownloadedSize(), 0); - ASSERT_FALSE(file_segment.reserve(file_segment.range().size())); + ASSERT_FALSE(file_segment.reserve(file_segment.range().size(), 1000)); file_segment.complete(); } @@ -956,7 +956,7 @@ TEST_F(FileCacheTest, temporaryData) for (auto & segment : *some_data_holder) { ASSERT_TRUE(segment->getOrSetDownloader() == DB::FileSegment::getCallerId()); - ASSERT_TRUE(segment->reserve(segment->range().size())); + ASSERT_TRUE(segment->reserve(segment->range().size(), 1000)); download(*segment); segment->complete(); } From 4f2be003521b00c9a9087e17fcffdf08cabcd5f1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:55:32 +0000 Subject: [PATCH 108/117] Some fixups --- docs/en/sql-reference/functions/functions-for-nulls.md | 10 +++++----- docs/en/sql-reference/operators/index.md | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index e73d6c899e7..61da9a191a1 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -10,11 +10,13 @@ sidebar_label: Nullable Returns whether the argument is [NULL](../../sql-reference/syntax.md#null). +See also operator [`IS NULL`](../operators/index.md#is_null). + ``` sql isNull(x) ``` -Alias: `IS NULL`. +Alias: `ISNULL` **Arguments** @@ -54,12 +56,12 @@ Result: Returns whether the argument is not [NULL](../../sql-reference/syntax.md#null-literal). +See also operator [`IS NOT NULL`](../operators/index.md#is_not_null). + ``` sql isNotNull(x) ``` -Alias: `IS NOT NULL`. - **Arguments:** - `x` — A value of non-compound data type. @@ -102,8 +104,6 @@ Returns whether the argument is 0 (zero) or [NULL](../../sql-reference/syntax.md isZeroOrNull(x) ``` -Alias: `x = 0 OR x IS NULL`. - **Arguments:** - `x` — A value of non-compound data type. diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 120e464e009..31bf43e8b35 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -353,7 +353,7 @@ For efficiency, the `and` and `or` functions accept any number of arguments. The ClickHouse supports the `IS NULL` and `IS NOT NULL` operators. -### IS NULL +### IS NULL {#is_null} - For [Nullable](../../sql-reference/data-types/nullable.md) type values, the `IS NULL` operator returns: - `1`, if the value is `NULL`. @@ -374,7 +374,7 @@ SELECT x+100 FROM t_null WHERE y IS NULL └──────────────┘ ``` -### IS NOT NULL +### IS NOT NULL {#is_not_null} - For [Nullable](../../sql-reference/data-types/nullable.md) type values, the `IS NOT NULL` operator returns: - `0`, if the value is `NULL`. From 6bbf9eb5400206c326a4e453a38612c8beb6ef89 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 13 Mar 2024 13:57:55 +0000 Subject: [PATCH 109/117] Fixup fixups --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 61da9a191a1..4dfbf4262ed 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -16,7 +16,7 @@ See also operator [`IS NULL`](../operators/index.md#is_null). isNull(x) ``` -Alias: `ISNULL` +Alias: `ISNULL`. **Arguments** From cb28c84a93709ab12fc32171bf880c0e911ec0d5 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 13 Mar 2024 16:25:58 +0100 Subject: [PATCH 110/117] Fix `forget_partition` test (#61237) --- ...et_partition.sql => 02995_forget_partition.sh} | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) rename tests/queries/0_stateless/{02995_forget_partition.sql => 02995_forget_partition.sh} (63%) mode change 100644 => 100755 diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sh old mode 100644 new mode 100755 similarity index 63% rename from tests/queries/0_stateless/02995_forget_partition.sql rename to tests/queries/0_stateless/02995_forget_partition.sh index 269f7932ea4..8ece8d3ddb3 --- a/tests/queries/0_stateless/02995_forget_partition.sql +++ b/tests/queries/0_stateless/02995_forget_partition.sh @@ -1,5 +1,12 @@ --- Tags: zookeeper, no-replicated-database +#!/usr/bin/env bash +# Tags: zookeeper, no-replicated-database +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ drop table if exists forget_partition; create table forget_partition @@ -16,7 +23,12 @@ insert into forget_partition select number, '2024-01-01' + interval number day, alter table forget_partition drop partition '20240101'; alter table forget_partition drop partition '20240102'; +""" +# DROP PARTITION do not wait for a part to be removed from memory due to possible concurrent SELECTs, so we have to do wait manually here +while [[ $(${CLICKHOUSE_CLIENT} -q "select count() from system.parts where database=currentDatabase() and table='forget_partition' and partition='20240101'") != 0 ]]; do sleep 0.1; done + +${CLICKHOUSE_CLIENT} --multiline --multiquery -q """ set allow_unrestricted_reads_from_keeper=1; select '---before---'; @@ -31,3 +43,4 @@ select '---after---'; select name from system.zookeeper where path = '/test/02995/' || currentDatabase() || '/rmt/block_numbers' order by name; drop table forget_partition; +""" From aecc135f5dd76c55b99205af170beafc06b9ee62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 13 Mar 2024 17:03:18 +0100 Subject: [PATCH 111/117] Add more details about how NULLs are processed in aggregations --- docs/en/sql-reference/aggregate-functions/index.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 5d2229fbcce..e97db436271 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -18,6 +18,10 @@ ClickHouse also supports: During aggregation, all `NULL`s are skipped. If the aggregation has several parameters it will ignore any row in which one or more of the parameters are NULL. +There are a few exceptions to this rule: + - Both [`first_value`](../../sql-reference/aggregate-functions/reference/first_value.md) and [`last_value`](../../sql-reference/aggregate-functions/reference/last_value.md) support modifiers that respect NULLs (`first_value(b) ignore nulls`). + - [`count`](../../sql-reference/aggregate-functions/reference/count.md) without parameters (`count()`) or with constant ones (`count(1)`) will count NULL rows too. With a column as parameter, it will count only not null values. + **Examples:** Consider this table: From 5b15ec6ae19fe1caa3800dfa333b61570cfc92b4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Mar 2024 17:10:55 +0100 Subject: [PATCH 112/117] Move test from stateless to integration --- .../config.d/storage_conf.xml | 4 + .../integration/test_filesystem_cache/test.py | 75 +++++++++++++++++++ ...810_system_sync_filesystem_cache.reference | 3 - .../02810_system_sync_filesystem_cache.sh | 69 ----------------- 4 files changed, 79 insertions(+), 72 deletions(-) delete mode 100644 tests/queries/0_stateless/02810_system_sync_filesystem_cache.reference delete mode 100755 tests/queries/0_stateless/02810_system_sync_filesystem_cache.sh diff --git a/tests/integration/test_filesystem_cache/config.d/storage_conf.xml b/tests/integration/test_filesystem_cache/config.d/storage_conf.xml index b614815b34f..a8e4f9f8a99 100644 --- a/tests/integration/test_filesystem_cache/config.d/storage_conf.xml +++ b/tests/integration/test_filesystem_cache/config.d/storage_conf.xml @@ -7,4 +7,8 @@ + + system + filesystem_cache_log
+
diff --git a/tests/integration/test_filesystem_cache/test.py b/tests/integration/test_filesystem_cache/test.py index c44d817c57c..dfab462732a 100644 --- a/tests/integration/test_filesystem_cache/test.py +++ b/tests/integration/test_filesystem_cache/test.py @@ -426,3 +426,78 @@ def test_force_filesystem_cache_on_merges(cluster): test(node, True) node = cluster.instances["node"] test(node, False) + + +def test_system_sync_filesystem_cache(cluster): + node = cluster.instances["node"] + node.query( + """ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (a Int32, b String) +ENGINE = MergeTree() ORDER BY tuple() +SETTINGS disk = disk(type = cache, + max_size = '100Ki', + path = "test_system_sync_filesystem_cache", + delayed_cleanup_interval_ms = 10000000, disk = hdd_blob), + min_bytes_for_wide_part = 10485760; + +INSERT INTO test SELECT 1, 'test'; + """ + ) + + query_id = "system_sync_filesystem_cache_1" + node.query( + "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1", + query_id=query_id, + ) + + key, offset = ( + node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT key, offset FROM system.filesystem_cache_log WHERE query_id = '{query_id}' ORDER BY size DESC LIMIT 1; + """ + ) + .strip() + .split("\t") + ) + + cache_path = node.query( + f"SELECT cache_path FROM system.filesystem_cache WHERE key = '{key}' and file_segment_range_begin = {offset}" + ) + + node.exec_in_container(["bash", "-c", f"rm {cache_path}"]) + + assert key in node.query("SYSTEM SYNC FILESYSTEM CACHE") + + node.query("SELECT * FROM test FORMAT Null") + assert key not in node.query("SYSTEM SYNC FILESYSTEM CACHE") + + query_id = "system_sync_filesystem_cache_2" + node.query( + "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1", + query_id=query_id, + ) + + key, offset = ( + node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT key, offset FROM system.filesystem_cache_log WHERE query_id = '{query_id}' ORDER BY size DESC LIMIT 1; + """ + ) + .strip() + .split("\t") + ) + cache_path = node.query( + f"SELECT cache_path FROM system.filesystem_cache WHERE key = '{key}' and file_segment_range_begin = {offset}" + ) + + node.exec_in_container(["bash", "-c", f"echo -n 'fff' > {cache_path}"]) + + assert key in node.query("SYSTEM SYNC FILESYSTEM CACHE") + + node.query("SELECT * FROM test FORMAT Null") + + assert key not in node.query("SYSTEM SYNC FILESYSTEM CACHE") diff --git a/tests/queries/0_stateless/02810_system_sync_filesystem_cache.reference b/tests/queries/0_stateless/02810_system_sync_filesystem_cache.reference deleted file mode 100644 index 7614df8ec46..00000000000 --- a/tests/queries/0_stateless/02810_system_sync_filesystem_cache.reference +++ /dev/null @@ -1,3 +0,0 @@ -ok -ok -ok diff --git a/tests/queries/0_stateless/02810_system_sync_filesystem_cache.sh b/tests/queries/0_stateless/02810_system_sync_filesystem_cache.sh deleted file mode 100755 index c88ba4d5a74..00000000000 --- a/tests/queries/0_stateless/02810_system_sync_filesystem_cache.sh +++ /dev/null @@ -1,69 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings - -# set -x - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - - -$CLICKHOUSE_CLIENT -nm --query """ -DROP TABLE IF EXISTS test; - -CREATE TABLE test (a Int32, b String) -ENGINE = MergeTree() ORDER BY tuple() -SETTINGS disk = disk(type = cache, max_size = '100Ki', path = ${CLICKHOUSE_TEST_UNIQUE_NAME}, delayed_cleanup_interval_ms = 10000000, disk = s3_disk), min_bytes_for_wide_part = 10485760; - -INSERT INTO test SELECT 1, 'test'; -""" - -query_id=$RANDOM - -$CLICKHOUSE_CLIENT --query_id "$query_id" --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1" - -${CLICKHOUSE_CLIENT} -q "system flush logs" - -key=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; -""") - -offset=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT offset FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; -""") - -path=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT cache_path FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset; -""") - -rm $path - -$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null SETTINGS enable_filesystem_cache_log = 1" 2>&1 | grep -F -e "No such file or directory" > /dev/null && echo "ok" || echo "fail" - -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=fatal/g') - -$CLICKHOUSE_CLIENT --query "SYSTEM SYNC FILESYSTEM CACHE" 2>&1 | grep -q "$key" && echo 'ok' || echo 'fail' - -$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" - -key=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT key FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; -""") - -offset=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT offset FROM system.filesystem_cache_log WHERE query_id = '$query_id' ORDER BY size DESC LIMIT 1; -""") - -path=$($CLICKHOUSE_CLIENT -nm --query """ -SELECT cache_path FROM system.filesystem_cache WHERE key = '$key' AND file_segment_range_begin = $offset; -""") - -echo -n 'fff' > $path - -#cat $path - -$CLICKHOUSE_CLIENT --query "SYSTEM SYNC FILESYSTEM CACHE" 2>&1 | grep -q "$key" && echo 'ok' || echo 'fail' - -$CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" - -$CLICKHOUSE_CLIENT --query "SYSTEM SYNC FILESYSTEM CACHE" From 4670f055649f5f8f216acd42947881038dedbdbd Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Mar 2024 17:18:13 +0000 Subject: [PATCH 113/117] Fix test test_input_format_parallel_parsing_memory_tracking --- .../test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index c95bbfda708..a89cb619350 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -41,7 +41,7 @@ def test_memory_tracking_total(): [ "bash", "-c", - "clickhouse local -q \"SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), ' ') from numbers(10000)\" > data.json", + "clickhouse local -q \"SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), ' ') from numbers(10000)\" > data.jsonl", ] ) @@ -56,7 +56,7 @@ def test_memory_tracking_total(): "--show-error", "--data-binary", "@data.json", - "http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV", + "http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20JSONEachRow", ] ) == "" From 6e6a67a2fb37ef22807d22aac1a8958bc618dc4d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Mar 2024 19:13:39 +0100 Subject: [PATCH 114/117] Fix unit test --- .../Cache/WriteBufferToFileSegment.cpp | 19 ++++++++++++++----- .../Cache/WriteBufferToFileSegment.h | 2 +- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp index 759135722dc..51914c0a14e 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.cpp +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.cpp @@ -19,9 +19,22 @@ namespace ErrorCodes extern const int NOT_ENOUGH_SPACE; } +namespace +{ + size_t getCacheLockWaitTimeout() + { + auto query_context = CurrentThread::getQueryContext(); + if (query_context) + return query_context->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; + else + return Context::getGlobalContextInstance()->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; + } +} + WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegment * file_segment_) : WriteBufferFromFileDecorator(std::make_unique(file_segment_->getPath())) , file_segment(file_segment_) + , reserve_space_lock_wait_timeout_milliseconds(getCacheLockWaitTimeout()) { } @@ -32,12 +45,8 @@ WriteBufferToFileSegment::WriteBufferToFileSegment(FileSegmentsHolderPtr segment : throw Exception(ErrorCodes::LOGICAL_ERROR, "WriteBufferToFileSegment can be created only from single segment")) , file_segment(&segment_holder_->front()) , segment_holder(std::move(segment_holder_)) + , reserve_space_lock_wait_timeout_milliseconds(getCacheLockWaitTimeout()) { - auto query_context = CurrentThread::getQueryContext(); - if (query_context) - reserve_space_lock_wait_timeout_milliseconds = query_context->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; - else - reserve_space_lock_wait_timeout_milliseconds = Context::getGlobalContextInstance()->getReadSettings().filesystem_cache_reserve_space_wait_lock_timeout_milliseconds; } /// If it throws an exception, the file segment will be incomplete, so you should not use it in the future. diff --git a/src/Interpreters/Cache/WriteBufferToFileSegment.h b/src/Interpreters/Cache/WriteBufferToFileSegment.h index bff340d79b3..822488ceb48 100644 --- a/src/Interpreters/Cache/WriteBufferToFileSegment.h +++ b/src/Interpreters/Cache/WriteBufferToFileSegment.h @@ -29,7 +29,7 @@ private: /// Empty if file_segment is not owned by this WriteBufferToFileSegment FileSegmentsHolderPtr segment_holder; - size_t reserve_space_lock_wait_timeout_milliseconds; + const size_t reserve_space_lock_wait_timeout_milliseconds; }; From 89de338676a0233dac4563d9f4ba7e4a16b54135 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 13 Mar 2024 17:33:39 +0100 Subject: [PATCH 115/117] Better --- .../aggregate-functions/index.md | 53 +++++++++++++++++-- 1 file changed, 49 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index e97db436271..96bf0c5d93b 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -16,11 +16,9 @@ ClickHouse also supports: ## NULL Processing -During aggregation, all `NULL`s are skipped. If the aggregation has several parameters it will ignore any row in which one or more of the parameters are NULL. +During aggregation, all `NULL` arguments are skipped. If the aggregation has several arguments it will ignore any row in which one or more of them are NULL. -There are a few exceptions to this rule: - - Both [`first_value`](../../sql-reference/aggregate-functions/reference/first_value.md) and [`last_value`](../../sql-reference/aggregate-functions/reference/last_value.md) support modifiers that respect NULLs (`first_value(b) ignore nulls`). - - [`count`](../../sql-reference/aggregate-functions/reference/count.md) without parameters (`count()`) or with constant ones (`count(1)`) will count NULL rows too. With a column as parameter, it will count only not null values. +There is an exception to this rule, which are the functions [`first_value`](../../sql-reference/aggregate-functions/reference/first_value.md), [`last_value`](../../sql-reference/aggregate-functions/reference/last_value.md) and their aliases when followed by the modifier `RESPECT NULLS`: `FIRST_VALUE(b) RESPECT NULLS`. **Examples:** @@ -89,3 +87,50 @@ FROM t_null_big; │ [2,2,3] │ [2,NULL,2,3,NULL] │ └───────────────┴───────────────────────────────────────┘ ``` + +Note that aggregations are skipped when the columns are used as arguments to an aggregated function. For example [`count`](../../sql-reference/aggregate-functions/reference/count.md) without parameters (`count()`) or with constant ones (`count(1)`) will count all rows in the block (independently of the value of the GROUP BY column as it's not an argument), while `count(column)` will only return the number of rows where column is not NULL. + +```sql +SELECT + v, + count(1), + count(v) +FROM +( + SELECT if(number < 10, NULL, number % 3) AS v + FROM numbers(15) +) +GROUP BY v + +┌────v─┬─count()─┬─count(v)─┐ +│ ᴺᵁᴸᴸ │ 10 │ 0 │ +│ 0 │ 1 │ 1 │ +│ 1 │ 2 │ 2 │ +│ 2 │ 2 │ 2 │ +└──────┴─────────┴──────────┘ +``` + +And here is an example of of first_value with `RESPECT NULLS` where we can see that NULL inputs are respected and it will return the first value read, whether it's NULL or not: + +```sql +SELECT + col || '_' || ((col + 1) * 5 - 1) as range, + first_value(odd_or_null) as first, + first_value(odd_or_null) IGNORE NULLS as first_ignore_null, + first_value(odd_or_null) RESPECT NULLS as first_respect_nulls +FROM +( + SELECT + intDiv(number, 5) AS col, + if(number % 2 == 0, NULL, number) as odd_or_null + FROM numbers(15) +) +GROUP BY col +ORDER BY col + +┌─range─┬─first─┬─first_ignore_null─┬─first_respect_nulls─┐ +│ 0_4 │ 1 │ 1 │ ᴺᵁᴸᴸ │ +│ 1_9 │ 5 │ 5 │ 5 │ +│ 2_14 │ 11 │ 11 │ ᴺᵁᴸᴸ │ +└───────┴───────┴───────────────────┴─────────────────────┘ +``` From ff976520389c277cfcb034466aed514fe96acc6d Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 14 Mar 2024 09:53:18 +0000 Subject: [PATCH 116/117] fix spell check --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index c7eb213bff2..1706d44bc8a 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2687,7 +2687,9 @@ userver utils uuid varPop +varPopStable varSamp +varSampStable variadic variantElement variantType From c6f0b2a5a458648c179456a6843aae8c9c3fb53a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 14 Mar 2024 11:57:03 +0100 Subject: [PATCH 117/117] Revert "Fix usage of session_token in S3 engine" --- src/Coordination/KeeperSnapshotManagerS3.cpp | 3 +-- src/Storages/StorageS3.cpp | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 796506a07db..80345db2524 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -121,8 +121,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auth_settings.use_insecure_imds_request.value_or(false), auth_settings.expiration_window_seconds.value_or(S3::DEFAULT_EXPIRATION_WINDOW_SECONDS), auth_settings.no_sign_request.value_or(false), - }, - credentials.GetSessionToken()); + }); auto new_client = std::make_shared(std::move(new_uri), std::move(auth_settings), std::move(client)); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ff055508aa6..11da394feec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1451,8 +1451,7 @@ void StorageS3::Configuration::connect(const ContextPtr & context) auth_settings.expiration_window_seconds.value_or( context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), - }, - credentials.GetSessionToken()); + }); } void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection)