From 01c42994815e81b2de424eed596b7d5ec2bce27a Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Sep 2022 22:34:31 +0200 Subject: [PATCH 001/116] Support single disk instead of storage policy --- src/Disks/StoragePolicy.cpp | 38 ++++++-- src/Disks/StoragePolicy.h | 12 +++ src/Interpreters/Context.cpp | 25 ++++++ src/Interpreters/Context.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 14 ++- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/registerStorageMergeTree.cpp | 2 - src/Storages/StorageMergeTree.cpp | 17 ++-- .../test_disk_configuration/__init__.py | 0 .../config.d/storage_configuration.xml | 25 ++++++ .../test_disk_configuration/test.py | 87 +++++++++++++++++++ 11 files changed, 209 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_disk_configuration/__init__.py create mode 100644 tests/integration/test_disk_configuration/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_disk_configuration/test.py diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 1c8c522a568..60f315c4957 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -400,11 +400,16 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti { std::shared_ptr result = std::make_shared(config, config_prefix, disks); + std::lock_guard lock(mutex); + /// First pass, check. for (const auto & [name, policy] : policies) { + if (name.starts_with(TMP_STORAGE_POLICY_PREFIX)) + continue; + if (!result->policies.contains(name)) - throw Exception("Storage policy " + backQuote(name) + " is missing in new configuration", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage policy {} is missing in new configuration", backQuote(name)); policy->checkCompatibleWith(result->policies[name]); } @@ -412,20 +417,43 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti /// Second pass, load. for (const auto & [name, policy] : policies) { - result->policies[name] = std::make_shared(policy, config, config_prefix + "." + name, disks); + /// Do not reload from config temporary storage policy, because it is not present in config. + if (name.starts_with(TMP_STORAGE_POLICY_PREFIX)) + result->policies[name] = policy; + else + result->policies[name] = std::make_shared(policy, config, config_prefix + "." + name, disks); } return result; } - -StoragePolicyPtr StoragePolicySelector::get(const String & name) const +StoragePolicyPtr StoragePolicySelector::tryGet(const String & name) const { + std::lock_guard lock(mutex); + auto it = policies.find(name); if (it == policies.end()) - throw Exception("Unknown storage policy " + backQuote(name), ErrorCodes::UNKNOWN_POLICY); + return nullptr; return it->second; } +StoragePolicyPtr StoragePolicySelector::get(const String & name) const +{ + auto policy = tryGet(name); + if (!policy) + throw Exception("Unknown storage policy " + backQuote(name), ErrorCodes::UNKNOWN_POLICY); + + return policy; +} + +void StoragePolicySelector::add(StoragePolicyPtr storage_policy) +{ + std::lock_guard lock(mutex); + + auto [_, inserted] = policies.emplace(storage_policy->getName(), storage_policy); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "StoragePolicy is already present in StoragePolicySelector"); +} + } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index b0e92bdad67..6e91d1ade31 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -92,6 +92,7 @@ public: bool hasAnyVolumeWithDisabledMerges() const override; bool containsVolume(const String & volume_name) const override; + private: Volumes volumes; const String name; @@ -118,6 +119,8 @@ using StoragePoliciesMap = std::map; class StoragePolicySelector { public: + static constexpr auto TMP_STORAGE_POLICY_PREFIX = "__"; + StoragePolicySelector(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); StoragePolicySelectorPtr updateFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) const; @@ -125,11 +128,20 @@ public: /// Policy by name StoragePolicyPtr get(const String & name) const; + StoragePolicyPtr tryGet(const String & name) const; + /// All policies const StoragePoliciesMap & getPoliciesMap() const { return policies; } + /// Add storage policy to StoragePolicySelector. + /// Used when storage policy needs to be created on the fly, not being present in config file. + /// Done by getOrSetStoragePolicyForSingleDisk. + void add(StoragePolicyPtr storage_policy); + private: StoragePoliciesMap policies; + + mutable std::mutex mutex; }; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 321a46baff6..d30e0c21075 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -2662,6 +2663,30 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } +StoragePolicyPtr Context::getOrSetStoragePolicyForSingleDisk(const String & name) const +{ + std::lock_guard lock(shared->storage_policies_mutex); + + const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + name; + auto storage_policy_selector = getStoragePolicySelector(lock); + StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); + + if (!storage_policy) + { + auto disk_selector = getDiskSelector(lock); + auto disk = disk_selector->get(name); + auto volume = std::make_shared("_volume_" + name, disk); + + static const auto move_factor_for_single_disk_volume = 0.0; + storage_policy = std::make_shared(storage_policy_name, Volumes{volume}, move_factor_for_single_disk_volume); + const_cast(storage_policy_selector.get())->add(storage_policy); + } + /// Note: it is important to put storage policy into disk selector (and not recreate it on each call) + /// because in some places there are checks that storage policy pointers are the same from different tables. + /// (We can assume that tables with the same `disk` setting are on the same storage policy). + + return storage_policy; +} DisksMap Context::getDisksMap() const { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a9984e32c1b..6e5bd0b9945 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -918,6 +918,8 @@ public: /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; + StoragePolicyPtr getOrSetStoragePolicyForSingleDisk(const String & name) const; + /// Get the server uptime in seconds. double getUptimeSeconds() const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ce4f1dc884d..32fdfbdf461 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -234,6 +234,10 @@ MergeTreeData::MergeTreeData( context_->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); const auto settings = getSettings(); + + if (settings->disk.changed && settings->storage_policy.changed) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); + allow_nullable_key = attach || settings->allow_nullable_key; if (relative_data_path.empty()) @@ -365,7 +369,15 @@ MergeTreeData::MergeTreeData( StoragePolicyPtr MergeTreeData::getStoragePolicy() const { - return getContext()->getStoragePolicy(getSettings()->storage_policy); + const auto & settings = getSettings(); + StoragePolicyPtr storage_policy; + + if (settings->disk.changed) + storage_policy = getContext()->getOrSetStoragePolicyForSingleDisk(settings->disk); + else + storage_policy = getContext()->getStoragePolicy(settings->storage_policy); + + return storage_policy; } bool MergeTreeData::supportsFinal() const diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 8d7f057e720..9e25d96519f 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -128,6 +128,7 @@ struct Settings; M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ M(String, storage_policy, "default", "Name of storage disk policy", 0) \ + M(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \ M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4274386e393..6982521f76a 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -683,8 +683,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - auto storage_policy = args.getContext()->getStoragePolicy(storage_settings->storage_policy); - return std::make_shared( zookeeper_path, replica_name, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e4062734352..143130fe87c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1632,13 +1632,18 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const auto dest_table_storage = std::dynamic_pointer_cast(dest_table); if (!dest_table_storage) - throw Exception("Table " + getStorageID().getNameForLogs() + " supports movePartitionToTable only for MergeTree family of table engines." - " Got " + dest_table->getName(), ErrorCodes::NOT_IMPLEMENTED); + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Table {} supports movePartitionToTable only for MergeTree family of table engines. Got {}", + getStorageID().getNameForLogs(), dest_table->getName()); + if (dest_table_storage->getStoragePolicy() != this->getStoragePolicy()) - throw Exception("Destination table " + dest_table_storage->getStorageID().getNameForLogs() + - " should have the same storage policy of source table " + getStorageID().getNameForLogs() + ". " + - getStorageID().getNameForLogs() + ": " + this->getStoragePolicy()->getName() + ", " + - dest_table_storage->getStorageID().getNameForLogs() + ": " + dest_table_storage->getStoragePolicy()->getName(), ErrorCodes::UNKNOWN_POLICY); + throw Exception( + ErrorCodes::UNKNOWN_POLICY, + "Destination table {} should have the same storage policy of source table {}. {} : {}, {} : {}", + dest_table_storage->getStorageID().getNameForLogs(), getStorageID().getNameForLogs(), + getStorageID().getNameForLogs(), this->getStoragePolicy()->getName(), + dest_table_storage->getStorageID().getNameForLogs(), dest_table_storage->getStoragePolicy()->getName()); auto dest_metadata_snapshot = dest_table->getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr(); diff --git a/tests/integration/test_disk_configuration/__init__.py b/tests/integration/test_disk_configuration/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disk_configuration/configs/config.d/storage_configuration.xml b/tests/integration/test_disk_configuration/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..f9e273a6d44 --- /dev/null +++ b/tests/integration/test_disk_configuration/configs/config.d/storage_configuration.xml @@ -0,0 +1,25 @@ + + + + + local + /disk_local/ + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ disk_local +
+
+
+
+
+
diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py new file mode 100644 index 00000000000..4b7d60fbe99 --- /dev/null +++ b/tests/integration/test_disk_configuration/test.py @@ -0,0 +1,87 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +TABLE_NAME = "test" + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/storage_configuration.xml", + ], + with_zookeeper=True, + stay_alive=True, + with_minio=True, + ) + + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_merge_tree_disk_setting(start_cluster): + node1 = cluster.instances["node1"] + + assert ( + "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time" + in node1.query_and_get_error( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS disk = 'disk_local', storage_policy = 's3'; + """ + ) + ) + + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS disk = 's3'; + """ + ) + + minio = cluster.minio_client + count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + + node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) > count + + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}_2; + CREATE TABLE {TABLE_NAME}_2 (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS disk = 's3'; + """ + ) + + node1.query(f"INSERT INTO {TABLE_NAME}_2 SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_2")) == 100 + + assert "__s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'").strip() + assert "__s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}_2'").strip() + + node1.query("SYSTEM RELOAD CONFIG") + assert not node1.contains_in_log("An error has occurred while reloading storage policies, storage policies were not applied") + assert "['s3']" in node1.query("SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'").strip() + + node1.restart_clickhouse() + + assert "_s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'").strip() + assert "['s3']" in node1.query("SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'").strip() + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From d538b5f3b84d85b90ee77beccb24ae427278abae Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Sep 2022 14:12:06 +0200 Subject: [PATCH 002/116] Allow custom disk configuration --- src/Common/SettingsChanges.h | 8 +- .../getDictionaryConfigurationFromAST.cpp | 2 +- src/Disks/DiskSelector.cpp | 15 +- src/Disks/DiskSelector.h | 7 +- src/Disks/IDisk.h | 6 + .../registerDiskAzureBlobStorage.cpp | 2 +- .../DiskObjectStorageCommon.cpp | 8 +- .../ObjectStorages/DiskObjectStorageCommon.h | 2 +- .../ObjectStorages/HDFS/registerDiskHDFS.cpp | 3 +- .../ObjectStorages/S3/registerDiskS3.cpp | 2 +- src/Disks/StoragePolicy.cpp | 6 - src/Disks/StoragePolicy.h | 2 - src/Disks/getDiskConfigurationFromAST.cpp | 93 ++++++++ src/Disks/getDiskConfigurationFromAST.h | 28 +++ src/Interpreters/Context.cpp | 20 +- src/Interpreters/Context.h | 9 +- src/Parsers/ASTSetQuery.cpp | 10 +- src/Parsers/ParserSetQuery.cpp | 17 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 +- .../MergeTree/registerStorageMergeTree.cpp | 66 +++--- src/Storages/createDiskFromDiskAST.cpp | 51 ++++ src/Storages/createDiskFromDiskAST.h | 23 ++ .../configs/config.d/remote_servers.xml | 16 ++ .../test_disk_configuration/test.py | 217 +++++++++++++++++- ...54_create_table_with_custom_disk.reference | 11 + .../02454_create_table_with_custom_disk.sql | 22 ++ 26 files changed, 589 insertions(+), 69 deletions(-) create mode 100644 src/Disks/getDiskConfigurationFromAST.cpp create mode 100644 src/Disks/getDiskConfigurationFromAST.h create mode 100644 src/Storages/createDiskFromDiskAST.cpp create mode 100644 src/Storages/createDiskFromDiskAST.h create mode 100644 tests/integration/test_disk_configuration/configs/config.d/remote_servers.xml create mode 100644 tests/queries/0_stateless/02454_create_table_with_custom_disk.reference create mode 100644 tests/queries/0_stateless/02454_create_table_with_custom_disk.sql diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 67cb69f77bf..10516cfabd4 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -13,11 +14,16 @@ struct SettingChange String name; Field value; + /// A setting value which cannot be put in Field. + ASTPtr value_ast; + SettingChange() = default; + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} + SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } }; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index e19495a27a3..ef63ca34675 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -506,7 +506,7 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & [name, value] : settings->changes) + for (const auto & [name, value, _] : settings->changes) { AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index d1a53afc376..ea5f27edfa6 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -121,15 +121,22 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } -DiskPtr DiskSelector::get(const String & name) const +DiskPtr DiskSelector::tryGet(const String & name) const { assertInitialized(); auto it = disks.find(name); if (it == disks.end()) - throw Exception("Unknown disk " + name, ErrorCodes::UNKNOWN_DISK); + return nullptr; return it->second; } +DiskPtr DiskSelector::get(const String & name) const +{ + auto disk = tryGet(name); + if (!disk) + throw Exception(ErrorCodes::UNKNOWN_DISK, "Unknown disk {}", name); + return disk; +} const DisksMap & DiskSelector::getDisksMap() const { @@ -141,7 +148,9 @@ const DisksMap & DiskSelector::getDisksMap() const void DiskSelector::addToDiskMap(const String & name, DiskPtr disk) { assertInitialized(); - disks.emplace(name, disk); + auto [_, inserted] = disks.emplace(name, disk); + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk with name `{}` is already in disks map", name); } diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index a760b3819ca..eda16d6b2e1 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -18,6 +18,8 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: + static constexpr auto TMP_DISK_PREFIX = "__tmp_"; + DiskSelector() = default; DiskSelector(const DiskSelector & from) = default; @@ -26,12 +28,13 @@ public: DiskSelectorPtr updateFromConfig( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context - ) const; + ContextPtr context) const; /// Get disk by name DiskPtr get(const String & name) const; + DiskPtr tryGet(const String & name) const; + /// Get all disks with names const DisksMap & getDisksMap() const; diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index de7b9181533..866290fbade 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -376,6 +376,11 @@ public: virtual bool supportsChmod() const { return false; } virtual void chmod(const String & /*path*/, mode_t /*mode*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support chmod"); } + /// Was disk created to be used without storage configuration? + bool isCustomDisk() const { return is_custom_disk; } + + void markDiskAsCustom() { is_custom_disk = true; } + protected: friend class DiskDecorator; @@ -389,6 +394,7 @@ protected: private: std::shared_ptr executor; + bool is_custom_disk = false; }; using Disks = std::vector; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index e6615316391..4b8dcafce46 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -72,7 +72,7 @@ void registerDiskAzureBlobStorage(DiskFactory & factory) ContextPtr context, const DisksMap & /*map*/) { - auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); ObjectStoragePtr azure_object_storage = std::make_unique( name, diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp index 3f69e79c645..d9c59b296f4 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp @@ -11,19 +11,19 @@ static String getDiskMetadataPath( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context) + const String & root_disks_path) { - return config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/"); + return config.getString(config_prefix + ".metadata_path", fs::path(root_disks_path) / "disks" / name / ""); } std::pair prepareForLocalMetadata( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context) + const String & root_disks_path) { /// where the metadata files are stored locally - auto metadata_path = getDiskMetadataPath(name, config, config_prefix, context); + auto metadata_path = getDiskMetadataPath(name, config, config_prefix, root_disks_path); fs::create_directories(metadata_path); auto metadata_disk = std::make_shared(name + "-metadata", metadata_path, 0); return std::make_pair(metadata_path, metadata_disk); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCommon.h b/src/Disks/ObjectStorages/DiskObjectStorageCommon.h index 0bdbe0dfd36..aa70c9f7a47 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCommon.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageCommon.h @@ -16,7 +16,7 @@ std::pair prepareForLocalMetadata( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context); + const String & root_disks_path); bool isFileWithPersistentCache(const String & path); diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index a9189e0101b..66115a7c329 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -38,7 +39,7 @@ void registerDiskHDFS(DiskFactory & factory) /// FIXME Cache currently unsupported :( ObjectStoragePtr hdfs_storage = std::make_unique(uri, std::move(settings), config); - auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context_); + auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context_->getPath()); auto metadata_storage = std::make_shared(metadata_disk, uri); uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16); diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 12e4df42863..72e6aac9d0e 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -122,7 +122,7 @@ void registerDiskS3(DiskFactory & factory) if (uri.key.back() != '/') throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 path must ends with '/', but '{}' doesn't.", uri.key); - auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); auto metadata_storage = std::make_shared(metadata_disk, uri.key); S3Capabilities s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 60f315c4957..3850e071f5e 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -400,8 +400,6 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti { std::shared_ptr result = std::make_shared(config, config_prefix, disks); - std::lock_guard lock(mutex); - /// First pass, check. for (const auto & [name, policy] : policies) { @@ -429,8 +427,6 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti StoragePolicyPtr StoragePolicySelector::tryGet(const String & name) const { - std::lock_guard lock(mutex); - auto it = policies.find(name); if (it == policies.end()) return nullptr; @@ -449,8 +445,6 @@ StoragePolicyPtr StoragePolicySelector::get(const String & name) const void StoragePolicySelector::add(StoragePolicyPtr storage_policy) { - std::lock_guard lock(mutex); - auto [_, inserted] = policies.emplace(storage_policy->getName(), storage_policy); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "StoragePolicy is already present in StoragePolicySelector"); diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 6e91d1ade31..014fa3ec251 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -140,8 +140,6 @@ public: private: StoragePoliciesMap policies; - - mutable std::mutex mutex; }; } diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp new file mode 100644 index 00000000000..628defe56ef --- /dev/null +++ b/src/Disks/getDiskConfigurationFromAST.cpp @@ -0,0 +1,93 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +[[noreturn]] static void throwBadConfiguration(const std::string & message = "") +{ + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect configuration{}. Example of expected configuration: `(type=s3 ...`)`", + message.empty() ? "" : ": " + message); +} + +Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +{ + if (disk_args.empty()) + throwBadConfiguration("expected non-empty list of arguments"); + + Poco::AutoPtr xml_document(new Poco::XML::Document()); + Poco::AutoPtr root(xml_document->createElement("disk")); + xml_document->appendChild(root); + Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); + root->appendChild(disk_configuration); + + for (const auto & arg : disk_args) + { + const auto * setting_function = arg->as(); + if (!setting_function || setting_function->name != "equals") + throwBadConfiguration("expected configuration arguments as key=value pairs"); + + const auto * function_args_expr = assert_cast(setting_function->arguments.get()); + if (!function_args_expr) + throwBadConfiguration("expected a list of key=value arguments"); + + auto function_args = function_args_expr->children; + if (function_args.empty()) + throwBadConfiguration("expected a non-empty list of key=value arguments"); + + auto * key_identifier = function_args[0]->as(); + if (!key_identifier) + throwBadConfiguration("expected the key (key=value) to be identifier"); + + const std::string & key = key_identifier->name(); + Poco::AutoPtr key_element(xml_document->createElement(key)); + disk_configuration->appendChild(key_element); + + if (!function_args[1]->as() && !function_args[1]->as()) + throwBadConfiguration("expected values to be literals or identifiers"); + + auto value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); + Poco::AutoPtr value_element(xml_document->createTextNode(convertFieldToString(value->as()->value))); + key_element->appendChild(value_element); + } + + return xml_document; +} + +DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +{ + auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); + + Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); + conf->load(xml_document); + + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ss.exceptions(std::ios::failbit); + conf->save(ss); + LOG_TEST(&Poco::Logger::get("getDiskConfigurationFromAST"), "Received disk configuration: {}", ss.str()); + + return conf; +} + +} diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h new file mode 100644 index 00000000000..85b7efef33e --- /dev/null +++ b/src/Disks/getDiskConfigurationFromAST.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using DiskConfigurationPtr = Poco::AutoPtr; + +/** + * Transform a list of pairs ( key1=value1, key2=value2, ... ), where keys and values are ASTLiteral or ASTIdentifier + * into + * + * value1 + * value2 + * ... + * + * + * Used in case disk configuration is passed via AST when creating + * a disk object on-the-fky without any configuration file. + */ +DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); + +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d30e0c21075..e095716b416 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2654,6 +2654,19 @@ DiskPtr Context::getDisk(const String & name) const return disk_selector->get(name); } +DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const +{ + std::lock_guard lock(shared->storage_policies_mutex); + + auto disk_selector = getDiskSelector(lock); + + auto disk = disk_selector->tryGet(name); + if (!disk) + const_cast(disk_selector.get())->addToDiskMap(name, creator(getDisksMap(lock))); + + return disk; +} + StoragePolicyPtr Context::getStoragePolicy(const String & name) const { std::lock_guard lock(shared->storage_policies_mutex); @@ -2663,7 +2676,7 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } -StoragePolicyPtr Context::getOrSetStoragePolicyForSingleDisk(const String & name) const +StoragePolicyPtr Context::getOrCreateStoragePolicyForSingleDisk(const String & name) const { std::lock_guard lock(shared->storage_policies_mutex); @@ -2691,6 +2704,11 @@ StoragePolicyPtr Context::getOrSetStoragePolicyForSingleDisk(const String & name DisksMap Context::getDisksMap() const { std::lock_guard lock(shared->storage_policies_mutex); + return getDisksMap(lock); +} + +DisksMap Context::getDisksMap(std::lock_guard & lock) const +{ return getDiskSelector(lock)->getDisksMap(); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6e5bd0b9945..e6711bcb04d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -909,16 +909,17 @@ public: /// Provides storage disks DiskPtr getDisk(const String & name) const; + using DiskCreator = std::function; + DiskPtr getOrCreateDisk(const String & name, DiskCreator creator) const; StoragePoliciesMap getPoliciesMap() const; DisksMap getDisksMap() const; void updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config); - /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; - StoragePolicyPtr getOrSetStoragePolicyForSingleDisk(const String & name) const; + StoragePolicyPtr getOrCreateStoragePolicyForSingleDisk(const String & name) const; /// Get the server uptime in seconds. double getUptimeSeconds() const; @@ -1047,7 +1048,9 @@ private: StoragePolicySelectorPtr getStoragePolicySelector(std::lock_guard & lock) const; - DiskSelectorPtr getDiskSelector(std::lock_guard & /* lock */) const; + DiskSelectorPtr getDiskSelector(std::lock_guard & lock) const; + + DisksMap getDisksMap(std::lock_guard & lock) const; }; } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index f6b3609b349..c9eb4fe43a7 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -19,7 +19,7 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const } } -void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const +void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked stacked) const { if (is_standalone) format.ostr << (format.hilite ? hilite_keyword : "") << "SET " << (format.hilite ? hilite_none : ""); @@ -30,7 +30,13 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma format.ostr << ", "; formatSettingName(it->name, format.ostr); - format.ostr << " = " << applyVisitor(FieldVisitorToString(), it->value); + if (it->value_ast) + { + format.ostr << " = "; + it->value_ast->formatImpl(format, state, stacked); + } + else + format.ostr << " = " << applyVisitor(FieldVisitorToString(), it->value); } } diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 20de785ac1b..92f5b55eab1 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -4,6 +4,8 @@ #include #include +#include +#include #include #include @@ -93,11 +95,14 @@ protected: bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected) { ParserCompoundIdentifier name_p; - ParserLiteralOrMap value_p; + ParserLiteralOrMap literal_or_map_p; ParserToken s_eq(TokenType::Equals); + ParserSetQuery set_p(true); + ParserFunction function_p; ASTPtr name; ASTPtr value; + ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -109,7 +114,15 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p value = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); - else if (!value_p.parse(pos, value, expected)) + /// for SETTINGS disk=disk(type='s3', path='', ...) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + { + tryGetIdentifierNameInto(name, change.name); + change.value_ast = function_ast; + + return true; + } + else if (!literal_or_map_p.parse(pos, value, expected)) return false; tryGetIdentifierNameInto(name, change.name); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 32fdfbdf461..676a84012e5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -90,6 +90,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -369,13 +370,15 @@ MergeTreeData::MergeTreeData( StoragePolicyPtr MergeTreeData::getStoragePolicy() const { - const auto & settings = getSettings(); + auto settings = getSettings(); + const auto & context = getContext(); + StoragePolicyPtr storage_policy; if (settings->disk.changed) - storage_policy = getContext()->getOrSetStoragePolicyForSingleDisk(settings->disk); + storage_policy = context->getOrCreateStoragePolicyForSingleDisk(settings->disk); else - storage_policy = getContext()->getStoragePolicy(settings->storage_policy); + storage_policy = context->getStoragePolicy(settings->storage_policy); return storage_policy; } @@ -1271,7 +1274,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) for (const auto & [disk_name, disk] : getContext()->getDisksMap()) { - if (disk->isBroken()) + if (disk->isBroken() || disk->isCustomDisk()) continue; if (!defined_disk_names.contains(disk_name) @@ -2367,7 +2370,6 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context if (!settings.allow_non_metadata_alters) { - auto mutation_commands = commands.getMutationCommands(new_metadata, settings.materialize_ttl_after_modify, getContext()); if (!mutation_commands.empty()) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6982521f76a..0f71ecf0923 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -250,6 +251,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) } ASTs & engine_args = args.engine_args; + auto context = args.getContext(); size_t arg_num = 0; size_t arg_cnt = engine_args.size(); @@ -359,7 +361,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { /// Try use default values if arguments are not specified. /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. - const auto & config = args.getContext()->getConfigRef(); + const auto & config = context->getConfigRef(); zookeeper_path = StorageReplicatedMergeTree::getDefaultZooKeeperPath(config); /// TODO maybe use hostname if {replica} is not defined? replica_name = StorageReplicatedMergeTree::getDefaultReplicaName(config); @@ -395,10 +397,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// We did unfold it in previous versions to make moving table from Atomic to Ordinary database work correctly, /// but now it's not allowed (and it was the only reason to unfold {uuid} macro). info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = args.getContext()->getMacros()->expand(zookeeper_path, info); + zookeeper_path = context->getMacros()->expand(zookeeper_path, info); info.level = 0; - replica_name = args.getContext()->getMacros()->expand(replica_name, info); + replica_name = context->getMacros()->expand(replica_name, info); } ast_zk_path->value = zookeeper_path; @@ -416,11 +418,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) } if (!allow_uuid_macro) info.table_id.uuid = UUIDHelpers::Nil; - zookeeper_path = args.getContext()->getMacros()->expand(zookeeper_path, info); + zookeeper_path = context->getMacros()->expand(zookeeper_path, info); info.level = 0; info.table_id.uuid = UUIDHelpers::Nil; - replica_name = args.getContext()->getMacros()->expand(replica_name, info); + replica_name = context->getMacros()->expand(replica_name, info); /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation @@ -482,7 +484,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) throw Exception(error_msg, ErrorCodes::BAD_ARGUMENTS); --arg_cnt; - setGraphitePatternsFromConfig(args.getContext(), graphite_config_name, merging_params.graphite_params); + setGraphitePatternsFromConfig(context, graphite_config_name, merging_params.graphite_params); } else if (merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) { @@ -510,7 +512,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) ColumnsDescription columns; if (args.columns.empty() && replicated) - columns = getColumnsDescriptionFromZookeeper(zookeeper_path, args.getContext()); + columns = getColumnsDescriptionFromZookeeper(zookeeper_path, context); else columns = args.columns; @@ -519,9 +521,9 @@ static StoragePtr create(const StorageFactory::Arguments & args) std::unique_ptr storage_settings; if (replicated) - storage_settings = std::make_unique(args.getContext()->getReplicatedMergeTreeSettings()); + storage_settings = std::make_unique(context->getReplicatedMergeTreeSettings()); else - storage_settings = std::make_unique(args.getContext()->getMergeTreeSettings()); + storage_settings = std::make_unique(context->getMergeTreeSettings()); if (is_extended_storage_def) { @@ -532,7 +534,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// Partition key may be undefined, but despite this we store it's empty /// value in partition_key structure. MergeTree checks this case and use /// single default partition with name "all". - metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, args.getContext()); + metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, context); /// PRIMARY KEY without ORDER BY is allowed and considered as ORDER BY. if (!args.storage_def->order_by && args.storage_def->primary_key) @@ -550,16 +552,16 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// before storage creation. After that storage will just copy this /// column if sorting key will be changed. metadata.sorting_key = KeyDescription::getSortingKeyFromAST( - args.storage_def->order_by->ptr(), metadata.columns, args.getContext(), merging_param_key_arg); + args.storage_def->order_by->ptr(), metadata.columns, context, merging_param_key_arg); /// If primary key explicitly defined, than get it from AST if (args.storage_def->primary_key) { - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext()); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, context); } else /// Otherwise we don't have explicit primary key and copy it from order by { - metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.getContext()); + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, context); /// and set it's definition_ast to nullptr (so isPrimaryKeyDefined() /// will return false but hasPrimaryKey() will return true. metadata.primary_key.definition_ast = nullptr; @@ -570,25 +572,25 @@ static StoragePtr create(const StorageFactory::Arguments & args) FunctionNameNormalizer().visit(partition_key.get()); auto primary_key_asts = metadata.primary_key.expression_list_ast->children; metadata.minmax_count_projection.emplace(ProjectionDescription::getMinMaxCountProjection( - args.columns, partition_key, minmax_columns, primary_key_asts, args.getContext())); + args.columns, partition_key, minmax_columns, primary_key_asts, context)); if (args.storage_def->sample_by) - metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.getContext()); + metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, context); if (args.storage_def->ttl_table) { metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST( - args.storage_def->ttl_table->ptr(), metadata.columns, args.getContext(), metadata.primary_key); + args.storage_def->ttl_table->ptr(), metadata.columns, context, metadata.primary_key); } if (args.query.columns_list && args.query.columns_list->indices) for (auto & index : args.query.columns_list->indices->children) - metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, args.getContext())); + metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, context)); if (args.query.columns_list && args.query.columns_list->projections) for (auto & projection_ast : args.query.columns_list->projections->children) { - auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, columns, args.getContext()); + auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, columns, context); metadata.projections.add(std::move(projection)); } @@ -601,10 +603,22 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto column_ttl_asts = columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) { - auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key); + auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, context, metadata.primary_key); metadata.column_ttls_by_name[name] = new_ttl_entry; } + if (args.storage_def->settings) + { + for (auto & [_, value, value_ast] : args.storage_def->settings->changes) + { + if (isDiskFunction(value_ast)) + { + value = createDiskFromDiskAST(*value_ast->as(), context); + break; + } + } + } + storage_settings->loadFromQuery(*args.storage_def); // updates the default storage_settings with settings specified via SETTINGS arg in a query @@ -622,14 +636,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto partition_by_ast = makeASTFunction("toYYYYMM", std::make_shared(date_column_name)); - metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata.columns, args.getContext()); + metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata.columns, context); ++arg_num; /// If there is an expression for sampling if (arg_cnt - arg_num == 3) { - metadata.sampling_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.getContext()); + metadata.sampling_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, context); ++arg_num; } @@ -639,10 +653,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// before storage creation. After that storage will just copy this /// column if sorting key will be changed. metadata.sorting_key - = KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, args.getContext(), merging_param_key_arg); + = KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, context, merging_param_key_arg); /// In old syntax primary_key always equals to sorting key. - metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, args.getContext()); + metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, context); /// But it's not explicitly defined, so we evaluate definition to /// nullptr metadata.primary_key.definition_ast = nullptr; @@ -654,7 +668,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) FunctionNameNormalizer().visit(partition_key.get()); auto primary_key_asts = metadata.primary_key.expression_list_ast->children; metadata.minmax_count_projection.emplace(ProjectionDescription::getMinMaxCountProjection( - args.columns, partition_key, minmax_columns, primary_key_asts, args.getContext())); + args.columns, partition_key, minmax_columns, primary_key_asts, context)); const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) @@ -690,7 +704,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) args.table_id, args.relative_data_path, metadata, - args.getContext(), + context, date_column_name, merging_params, std::move(storage_settings), @@ -703,7 +717,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) args.relative_data_path, metadata, args.attach, - args.getContext(), + context, date_column_name, merging_params, std::move(storage_settings), diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp new file mode 100644 index 00000000000..5b39a6d9c27 --- /dev/null +++ b/src/Storages/createDiskFromDiskAST.cpp @@ -0,0 +1,51 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +bool isDiskFunction(ASTPtr ast) +{ + if (!ast) + return false; + + const auto * function = ast->as(); + return function && function->name == "disk" && function->arguments->as(); +} + +std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context) +{ + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + auto disk_setting_string = serializeAST(function, true); + auto disk_name = DiskSelector::TMP_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + LOG_TRACE(&Poco::Logger::get("createDiskFromDiskAST"), "Using disk name `{}` for custom disk {}", disk_name, disk_setting_string); + + context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(disk_name, function_args, context); + auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(); + return disk; + }); + + return disk_name; +} + +} diff --git a/src/Storages/createDiskFromDiskAST.h b/src/Storages/createDiskFromDiskAST.h new file mode 100644 index 00000000000..6047a494a6f --- /dev/null +++ b/src/Storages/createDiskFromDiskAST.h @@ -0,0 +1,23 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class ASTFunction; + +/** + * Create a DiskPtr from disk AST function like disk(), + * add it to DiskSelector by a unique (but always the same for given configuration) disk name + * and return this name. + */ +std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context); + +/* + * Is given ast has form of a disk() function. + */ +bool isDiskFunction(ASTPtr ast); + +} diff --git a/tests/integration/test_disk_configuration/configs/config.d/remote_servers.xml b/tests/integration/test_disk_configuration/configs/config.d/remote_servers.xml new file mode 100644 index 00000000000..a7a4cbe5d95 --- /dev/null +++ b/tests/integration/test_disk_configuration/configs/config.d/remote_servers.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index 4b7d60fbe99..5933ffbd703 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -14,10 +14,23 @@ def start_cluster(): "node1", main_configs=[ "configs/config.d/storage_configuration.xml", + "configs/config.d/remote_servers.xml", ], with_zookeeper=True, stay_alive=True, with_minio=True, + macros={"replica": "node1", "shard": "shard1"}, + ) + cluster.add_instance( + "node2", + main_configs=[ + "configs/config.d/storage_configuration.xml", + "configs/config.d/remote_servers.xml", + ], + with_zookeeper=True, + stay_alive=True, + with_minio=True, + macros={"replica": "node2", "shard": "shard1"}, ) cluster.start() @@ -58,7 +71,10 @@ def test_merge_tree_disk_setting(start_cluster): node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - assert len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) > count + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + > count + ) node1.query( f""" @@ -73,15 +89,202 @@ def test_merge_tree_disk_setting(start_cluster): node1.query(f"INSERT INTO {TABLE_NAME}_2 SELECT number FROM numbers(100)") assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_2")) == 100 - assert "__s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'").strip() - assert "__s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}_2'").strip() + assert ( + "__s3" + in node1.query( + f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'" + ).strip() + ) + assert ( + "__s3" + in node1.query( + f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}_2'" + ).strip() + ) node1.query("SYSTEM RELOAD CONFIG") - assert not node1.contains_in_log("An error has occurred while reloading storage policies, storage policies were not applied") - assert "['s3']" in node1.query("SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'").strip() + assert not node1.contains_in_log( + "An error has occurred while reloading storage policies, storage policies were not applied" + ) + assert ( + "['s3']" + in node1.query( + "SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'" + ).strip() + ) node1.restart_clickhouse() - assert "_s3" in node1.query(f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'").strip() - assert "['s3']" in node1.query("SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'").strip() + assert ( + "_s3" + in node1.query( + f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}'" + ).strip() + ) + assert ( + "['s3']" + in node1.query( + "SELECT disks FROM system.storage_policies WHERE policy_name = '__s3'" + ).strip() + ) assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + + node1.query(f"DROP TABLE {TABLE_NAME} SYNC") + node1.query(f"DROP TABLE {TABLE_NAME}_2 SYNC") + + for obj in list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)): + minio.remove_object(cluster.minio_bucket, obj.object_name) + + +def test_merge_tree_custom_disk_setting(start_cluster): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}; + CREATE TABLE {TABLE_NAME} (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS + disk = disk( + type=s3, + endpoint='http://minio1:9001/root/data/', + access_key_id='minio', + secret_access_key='minio123'); + """ + ) + + # Check that data was indeed created on s3 with the needed path in s3 + + minio = cluster.minio_client + count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + + node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + > count + ) + + # Check that data for the second table was created on the same disk on the same path + + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}_2; + CREATE TABLE {TABLE_NAME}_2 (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS + disk = disk( + type=s3, + endpoint='http://minio1:9001/root/data/', + access_key_id='minio', + secret_access_key='minio123'); + """ + ) + + count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + node1.query(f"INSERT INTO {TABLE_NAME}_2 SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_2")) == 100 + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + > count + ) + + # Check that data for a disk with a different path was created on the different path + + for obj in list(minio.list_objects(cluster.minio_bucket, "data2/", recursive=True)): + minio.remove_object(cluster.minio_bucket, obj.object_name) + + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}_3; + CREATE TABLE {TABLE_NAME}_3 (a Int32) + ENGINE = MergeTree() + ORDER BY tuple() + SETTINGS + disk = disk( + type=s3, + endpoint='http://minio1:9001/root/data2/', + access_key_id='minio', + secret_access_key='minio123'); + """ + ) + count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + node1.query(f"INSERT INTO {TABLE_NAME}_3 SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_3")) == 100 + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + == count + ) + assert ( + len(list(minio.list_objects(cluster.minio_bucket, "data2/", recursive=True))) + > 0 + ) + + # check DETACH ATTACH + + node1.query(f"DETACH TABLE {TABLE_NAME}") + node1.query(f"ATTACH TABLE {TABLE_NAME}") + + node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 200 + + # check after server restart the same disk path is used with the same metadata + + node1.restart_clickhouse() + + node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 + + # check replicated merge tree on cluster + + replica = "{replica}" + node1.query( + f""" + DROP TABLE IF EXISTS {TABLE_NAME}_4; + CREATE TABLE {TABLE_NAME}_4 ON CLUSTER 'cluster' (a Int32) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/tbl/', '{replica}') + ORDER BY tuple() + SETTINGS + disk = disk( + type=s3, + endpoint='http://minio1:9001/root/data2/', + access_key_id='minio', + secret_access_key='minio123'); + """ + ) + + expected = """ + SETTINGS disk = disk(type = s3, endpoint = \\'http://minio1:9001/root/data2/\\', access_key_id = \\'minio\\', secret_access_key = \\'minio123\\'), index_granularity = 8192 + """ + + assert expected.strip() in node1.query(f"SHOW CREATE TABLE {TABLE_NAME}_4").strip() + assert expected.strip() in node2.query(f"SHOW CREATE TABLE {TABLE_NAME}_4").strip() + + node1.restart_clickhouse() + node2.restart_clickhouse() + + assert expected.strip() in node1.query(f"SHOW CREATE TABLE {TABLE_NAME}_4").strip() + assert expected.strip() in node2.query(f"SHOW CREATE TABLE {TABLE_NAME}_4").strip() + + # check that disk names are the same for all replicas + + policy1 = node1.query( + f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}_4'" + ).strip() + + policy2 = node2.query( + f"SELECT storage_policy FROM system.tables WHERE name = '{TABLE_NAME}_4'" + ).strip() + + assert policy1 == policy2 + assert ( + node1.query( + f"SELECT disks FROM system.storage_policies WHERE policy_name = '{policy1}'" + ).strip() + == node2.query( + f"SELECT disks FROM system.storage_policies WHERE policy_name = '{policy2}'" + ).strip() + ) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference new file mode 100644 index 00000000000..533b744f6e9 --- /dev/null +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -0,0 +1,11 @@ +CREATE TABLE test +( + `a` Int32 +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS disk = disk(type = local, path = \'/home/ubuntu/disk_path/\') +100 +CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/home/ubuntu/disk_path/\'), index_granularity = 8192 +a Int32 +200 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql new file mode 100644 index 00000000000..6687ae4c32e --- /dev/null +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS test; + +EXPLAIN SYNTAX +CREATE TABLE test (a Int32) +ENGINE = MergeTree() order by tuple() +SETTINGS disk = disk(type=local, path='/home/ubuntu/disk_path/'); + +CREATE TABLE test (a Int32) +ENGINE = MergeTree() order by tuple() +SETTINGS disk = disk(type=local, path='/home/ubuntu/disk_path/'); + +INSERT INTO test SELECT number FROM numbers(100); +SELECT count() FROM test; + +DETACH TABLE test; +ATTACH TABLE test; + +SHOW CREATE TABLE test; +DESCRIBE TABLE test; + +INSERT INTO test SELECT number FROM numbers(100); +SELECT count() FROM test; From 7fbc77eaccb7ab2e3c7c79b95c5b6aca22fddecb Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 30 Sep 2022 17:47:58 +0200 Subject: [PATCH 003/116] Avoid incorrect warning during update of DiskSelector disks --- src/Disks/DiskSelector.cpp | 11 ++++++++--- src/Parsers/ASTSetQuery.cpp | 6 +++--- src/Storages/createDiskFromDiskAST.cpp | 5 ----- tests/integration/test_disk_configuration/test.py | 8 ++++++++ 4 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index ea5f27edfa6..30150588493 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -73,12 +73,12 @@ DiskSelectorPtr DiskSelector::updateFromConfig( std::shared_ptr result = std::make_shared(*this); constexpr auto default_disk_name = "default"; - DisksMap old_disks_minus_new_disks (result->getDisksMap()); + DisksMap old_disks_minus_new_disks(result->getDisksMap()); for (const auto & disk_name : keys) { if (!std::all_of(disk_name.begin(), disk_name.end(), isWordCharASCII)) - throw Exception("Disk name can contain only alphanumeric and '_' (" + disk_name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + throw Exception(ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG, "Disk name can contain only alphanumeric and '_' ({})", disk_name); auto disk_config_prefix = config_prefix + "." + disk_name; if (!result->getDisksMap().contains(disk_name)) @@ -106,10 +106,15 @@ DiskSelectorPtr DiskSelector::updateFromConfig( writeString("Disks ", warning); int index = 0; - for (const auto & [name, _] : old_disks_minus_new_disks) + for (const auto & [name, disk] : old_disks_minus_new_disks) { + /// Custom disks are not present in config. + if (disk->isCustomDisk()) + continue; + if (index++ > 0) writeString(", ", warning); + writeBackQuotedString(name, warning); } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index f85c8e8ff95..bc56690a26d 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -34,13 +34,13 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, first = false; formatSettingName(change.name, format.ostr); - if (it->value_ast) + if (change.value_ast) { format.ostr << " = "; - it->value_ast->formatImpl(format, state, stacked); + change.value_ast->formatImpl(format, state, stacked); } else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), it->value); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & [name, value] : query_parameters) diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp index 5b39a6d9c27..11a942beeba 100644 --- a/src/Storages/createDiskFromDiskAST.cpp +++ b/src/Storages/createDiskFromDiskAST.cpp @@ -11,11 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - bool isDiskFunction(ASTPtr ast) { if (!ast) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index 5933ffbd703..aca072623ed 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -238,6 +238,14 @@ def test_merge_tree_custom_disk_setting(start_cluster): node1.query(f"INSERT INTO {TABLE_NAME} SELECT number FROM numbers(100)") assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 + # check reload config does not wipe custom disk + + node1.query("SYSTEM RELOAD CONFIG") + assert not node1.contains_in_log( + "disappeared from configuration, this change will be applied after restart of ClickHouse" + ) + assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}")) == 300 + # check replicated merge tree on cluster replica = "{replica}" From ef0a3dfc09d8b90156db056c5742c7e34be3869d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 1 Oct 2022 13:57:52 +0200 Subject: [PATCH 004/116] Fix test --- .../0_stateless/02454_create_table_with_custom_disk.reference | 4 ++-- .../0_stateless/02454_create_table_with_custom_disk.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference index 533b744f6e9..7281966f9a4 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -4,8 +4,8 @@ CREATE TABLE test ) ENGINE = MergeTree ORDER BY tuple() -SETTINGS disk = disk(type = local, path = \'/home/ubuntu/disk_path/\') +SETTINGS disk = disk(type = local, path = \'/disk_path/\') 100 -CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/home/ubuntu/disk_path/\'), index_granularity = 8192 +CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/disk_path/\'), index_granularity = 8192 a Int32 200 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 6687ae4c32e..5909eb3ad36 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -3,11 +3,11 @@ DROP TABLE IF EXISTS test; EXPLAIN SYNTAX CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/home/ubuntu/disk_path/'); +SETTINGS disk = disk(type=local, path='/disk_path/'); CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/home/ubuntu/disk_path/'); +SETTINGS disk = disk(type=local, path='/disk_path/'); INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; From a013f41403637704d0fc3e02f45770d4f9ef8993 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 4 Oct 2022 15:25:55 +0200 Subject: [PATCH 005/116] Try fix test --- .../0_stateless/02454_create_table_with_custom_disk.reference | 4 ++-- .../0_stateless/02454_create_table_with_custom_disk.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference index 7281966f9a4..378722b5166 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.reference @@ -4,8 +4,8 @@ CREATE TABLE test ) ENGINE = MergeTree ORDER BY tuple() -SETTINGS disk = disk(type = local, path = \'/disk_path/\') +SETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\') 100 -CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/disk_path/\'), index_granularity = 8192 +CREATE TABLE default.test\n(\n `a` Int32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS disk = disk(type = local, path = \'/var/lib/clickhouse/disks/local/\'), index_granularity = 8192 a Int32 200 diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 5909eb3ad36..c80e2188aff 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -3,11 +3,11 @@ DROP TABLE IF EXISTS test; EXPLAIN SYNTAX CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/disk_path/'); +SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/disk_path/'); +SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; From ef6580854e666f178a7f51e33fcad0d1a8bfea8a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 4 Oct 2022 20:01:36 +0200 Subject: [PATCH 006/116] Update 02454_create_table_with_custom_disk.sql --- .../queries/0_stateless/02454_create_table_with_custom_disk.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index c80e2188aff..db3fe200ff3 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -1,3 +1,5 @@ +-- Tags: no-s3-storage + DROP TABLE IF EXISTS test; EXPLAIN SYNTAX From fb366506af9010309ae76ddd049b2ce721805ae7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 5 Oct 2022 12:20:13 +0200 Subject: [PATCH 007/116] Fix integration test --- src/Disks/DiskSelector.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 30150588493..d7cb1d928b7 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -118,8 +118,11 @@ DiskSelectorPtr DiskSelector::updateFromConfig( writeBackQuotedString(name, warning); } - writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); - LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); + if (index > 0) + { + writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); + LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); + } } return result; From 301c7c90efefe89573428643af9ab0d74af75e2e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Oct 2022 23:27:21 +0200 Subject: [PATCH 008/116] Add local disk path restriction --- src/Disks/getDiskConfigurationFromAST.h | 2 +- src/Storages/createDiskFromDiskAST.cpp | 32 +++++++++++++++++-- .../config.d/custom_disks_base_path.xml | 3 ++ tests/config/install.sh | 1 + .../02454_create_table_with_custom_disk.sql | 4 +++ 5 files changed, 39 insertions(+), 3 deletions(-) create mode 100644 tests/config/config.d/custom_disks_base_path.xml diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h index 85b7efef33e..1f9d7c1bfe6 100644 --- a/src/Disks/getDiskConfigurationFromAST.h +++ b/src/Disks/getDiskConfigurationFromAST.h @@ -21,7 +21,7 @@ using DiskConfigurationPtr = Poco::AutoPtr; * * * Used in case disk configuration is passed via AST when creating - * a disk object on-the-fky without any configuration file. + * a disk object on-the-fly without any configuration file. */ DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp index 11a942beeba..9abbb5c0a9b 100644 --- a/src/Storages/createDiskFromDiskAST.cpp +++ b/src/Storages/createDiskFromDiskAST.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + bool isDiskFunction(ASTPtr ast) { if (!ast) @@ -28,9 +34,13 @@ std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr conte auto disk_setting_string = serializeAST(function, true); auto disk_name = DiskSelector::TMP_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - LOG_TRACE(&Poco::Logger::get("createDiskFromDiskAST"), "Using disk name `{}` for custom disk {}", disk_name, disk_setting_string); - context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + LOG_TRACE( + &Poco::Logger::get("createDiskFromDiskAST"), + "Using disk name `{}` for custom disk {}", + disk_name, disk_setting_string); + + auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { const auto * function_args_expr = assert_cast(function.arguments.get()); const auto & function_args = function_args_expr->children; auto config = getDiskConfigurationFromAST(disk_name, function_args, context); @@ -40,6 +50,24 @@ std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr conte return disk; }); + if (!result_disk->isRemote()) + { + static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + return disk_name; } diff --git a/tests/config/config.d/custom_disks_base_path.xml b/tests/config/config.d/custom_disks_base_path.xml new file mode 100644 index 00000000000..d8da9ebc590 --- /dev/null +++ b/tests/config/config.d/custom_disks_base_path.xml @@ -0,0 +1,3 @@ + + /var/lib/clickhouse/disks/ + diff --git a/tests/config/install.sh b/tests/config/install.sh index 9a172f0be15..019e55f99f6 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -51,6 +51,7 @@ ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/custom_disks_base_path.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index db3fe200ff3..6fdaf8814d9 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -11,6 +11,10 @@ CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); +CREATE TABLE test (a Int32) +ENGINE = MergeTree() order by tuple() +SETTINGS disk = disk(type=local, path='/local/'); -- { serverError BAD_ARGUMENTS } + INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; From bed9f7594af57f335e805dafd63195e13c9115d9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 7 Nov 2022 16:58:03 +0100 Subject: [PATCH 009/116] Better code around SettingsChanges --- programs/client/Client.cpp | 2 +- src/Access/SettingsConstraints.cpp | 16 ++++---- src/Access/SettingsConstraints.h | 2 +- src/Backups/BackupSettings.cpp | 6 +-- src/Backups/RestoreSettings.cpp | 7 ++-- src/Client/ClientBase.cpp | 4 +- src/Common/SettingsChanges.cpp | 26 ++++++++++--- src/Common/SettingsChanges.h | 38 ++++++++++++++----- src/Core/BaseSettings.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 4 +- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 11 +++--- .../getDictionaryConfigurationFromAST.cpp | 6 +-- src/Interpreters/Context.cpp | 7 ++-- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 22 ++++++----- src/Parsers/ASTBackupQuery.cpp | 2 +- src/Parsers/ASTDictionary.cpp | 2 +- src/Parsers/ASTSetQuery.cpp | 14 +++---- .../InsertQuerySettingsPushDownVisitor.cpp | 2 +- .../InsertQuerySettingsPushDownVisitor.h | 2 +- src/Parsers/ParserBackupQuery.cpp | 2 +- src/Parsers/ParserSetQuery.cpp | 14 +++---- src/Parsers/ParserSetQuery.h | 2 +- ...QueryWithOutputSettingsPushDownVisitor.cpp | 4 +- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Storages/AlterCommands.cpp | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 12 +++--- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 10 ++--- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 9 +++-- .../MaterializedPostgreSQLConsumer.cpp | 8 ++-- .../MaterializedPostgreSQLConsumer.h | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageJoin.cpp | 36 +++++++++--------- src/Storages/StorageLogSettings.cpp | 4 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- .../System/StorageSystemDDLWorkerQueue.cpp | 4 +- 41 files changed, 179 insertions(+), 133 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 303c8c2ce4f..a589cd61be8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -480,7 +480,7 @@ void Client::printChangedSettings() const { fmt::print(stderr, ", "); } - fmt::print(stderr, "{} = '{}'", changes[i].name, toString(changes[i].value)); + fmt::print(stderr, "{} = '{}'", changes[i].getName(), changes[i].getValueString()); } fmt::print(stderr, "\n"); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index d97a78c78ab..78bb5c3842e 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -110,7 +110,7 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - const String & setting_name = change.name; + const String & setting_name = change.getName(); if (setting_name == "profile") return true; @@ -145,7 +145,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh { if (e.code() == ErrorCodes::UNKNOWN_SETTING) { - if (const auto hints = current_settings.getHints(change.name); !hints.empty()) + if (const auto hints = current_settings.getHints(change.getName()); !hints.empty()) { e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); } @@ -160,16 +160,16 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (current_settings.tryGet(setting_name, current_value)) { /// Setting isn't checked if value has not changed. - if (change.value == current_value) + if (change.getFieldValue() == current_value) return false; - new_value = cast_value(change.value); + new_value = cast_value(change.getFieldValue()); if ((new_value == current_value) || cannot_cast) return false; } else { - new_value = cast_value(change.value); + new_value = cast_value(change.getFieldValue()); if (cannot_cast) return false; } @@ -179,7 +179,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const { - const String & setting_name = change.name; + const String & setting_name = change.getName(); auto less_or_cannot_compare = [=](const Field & left, const Field & right) { @@ -234,7 +234,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.value = min_value; + change.setFieldValue(min_value); } if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value)) @@ -246,7 +246,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.value = max_value; + change.setFieldValue(max_value); } return true; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 822bf42861b..080e18aefe6 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,7 +12,7 @@ namespace Poco::Util namespace DB { struct Settings; -struct SettingChange; +class SettingChange; class SettingsChanges; class AccessControl; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index a4b20e0b863..6ad95c54b6e 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -83,12 +83,12 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) for (const auto & setting : settings) { #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ - if (setting.name == #NAME) \ - res.NAME = SettingField##TYPE{setting.value}.value; \ + if (setting.getName() == #NAME) \ + res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ else LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); } } diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 63915670fa4..756c1433c47 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -173,12 +173,11 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) for (const auto & setting : settings) { #define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \ - if (setting.name == #NAME) \ - res.NAME = SettingField##TYPE{setting.value}.value; \ + if (setting.getName() == #NAME) \ + res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ else - LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f407fab68f1..c7f4f83157f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1536,8 +1536,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// Save all changes in settings to avoid losing them if the connection is lost. for (const auto & change : set_query->changes) { - if (change.name == "profile") - current_profile = change.value.safeGet(); + if (change.getName() == "profile") + current_profile = change.getFieldValue().safeGet(); else global_context->applySettingChange(change); } diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 9fb4f361e09..6d047684d41 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -1,4 +1,6 @@ #include +#include +#include namespace DB { @@ -6,7 +8,7 @@ namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); if (it == changes.end()) return nullptr; return &*it; @@ -14,19 +16,33 @@ namespace const SettingChange * find(const SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); if (it == changes.end()) return nullptr; return &*it; } } +String SettingChange::getValueString() const +{ + if (ast_value) + return serializeAST(*ast_value); + return convertFieldToString(field_value); +} + +void SettingChange::throwIfASTValue() const +{ + if (getASTValue() != nullptr) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "AST value of the setting must be converted to Field value"); +} bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); if (!change) return false; - out_value = change->value; + out_value = change->getFieldValue(); return true; } @@ -35,7 +51,7 @@ const Field * SettingsChanges::tryGet(std::string_view name) const const auto * change = find(*this, name); if (!change) return nullptr; - return &change->value; + return &change->getFieldValue(); } Field * SettingsChanges::tryGet(std::string_view name) @@ -43,7 +59,7 @@ Field * SettingsChanges::tryGet(std::string_view name) auto * change = find(*this, name); if (!change) return nullptr; - return &change->value; + return &change->getFieldValue(); } } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 10516cfabd4..e83a7b7a695 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -9,22 +9,42 @@ namespace DB class IColumn; -struct SettingChange +class SettingChange { +private: String name; - Field value; - - /// A setting value which cannot be put in Field. - ASTPtr value_ast; + Field field_value; + ASTPtr ast_value; /// A setting value which cannot be put in Field. +public: SettingChange() = default; - SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} - SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} - SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} + SettingChange(std::string_view name_, const Field & value_) : name(name_), field_value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), field_value(std::move(value_)) {} + SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), ast_value(value_->clone()) {} + + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) + { + return (lhs.name == rhs.name) && (lhs.field_value == rhs.field_value) && (lhs.ast_value == rhs.ast_value); + } - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } + + void throwIfASTValue() const; + + const String & getName() const { return name; } + String & getName() { return name; } + + const Field & getFieldValue() const { throwIfASTValue(); return field_value; } + Field & getFieldValue() { throwIfASTValue(); return field_value; } + + const ASTPtr & getASTValue() const { return ast_value; } + ASTPtr & getASTValue() { return ast_value; } + + void setFieldValue(const Field & field) { field_value = field; } + void setASTValue(const ASTPtr & ast) { ast_value = ast->clone(); } + + String getValueString() const; }; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 7b56367769e..41a3d6d2ee1 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -293,7 +293,7 @@ SettingsChanges BaseSettings::changes() const template void BaseSettings::applyChange(const SettingChange & change) { - set(change.name, change.value); + set(change.getName(), change.getFieldValue()); } template diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 796142884a3..0b1d499f9ff 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -736,9 +736,9 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha for (const auto & change : settings_changes) { auto it = std::find_if(storage_settings.begin(), storage_settings.end(), - [&](const auto & prev){ return prev.name == change.name; }); + [&](const auto & prev){ return prev.getName() == change.getName(); }); if (it != storage_settings.end()) - it->value = change.value; + it->setFieldValue(change.getFieldValue()); else storage_settings.push_back(change); } diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 87da701e481..3223452f867 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) SettingsChanges & changes = storage_def.settings->changes; #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.getName() == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, static_cast(NAME)}); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 523cc7041be..3263f7e7479 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -139,17 +139,18 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges for (const auto & change : settings_changes) { - if (!settings->has(change.name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name); + if (!settings->has(change.getName())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.getName()); - if ((change.name == "materialized_postgresql_tables_list")) + if ((change.getName() == "materialized_postgresql_tables_list")) { if (!query_context->isInternalQuery()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.name); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.getName()); need_update_on_disk = true; } - else if ((change.name == "materialized_postgresql_allow_automatic_update") || (change.name == "materialized_postgresql_max_block_size")) + else if ((change.getName() == "materialized_postgresql_allow_automatic_update") + || (change.getName() == "materialized_postgresql_max_block_size")) { replication_handler->setSetting(change); need_update_on_disk = true; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index ef63ca34675..373eb6c181b 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -506,11 +506,11 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & [name, value, _] : settings->changes) + for (const auto & change : settings->changes) { - AutoPtr setting_change_element(doc->createElement(name)); + AutoPtr setting_change_element(doc->createElement(change.getName())); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue()))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index edeb1991825..1cdf3124704 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1335,12 +1335,11 @@ void Context::applySettingChange(const SettingChange & change) { try { - setSetting(change.name, change.value); + setSetting(change.getName(), change.getFieldValue()); } catch (Exception & e) { - e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", - change.name, applyVisitor(FieldVisitorToString(), change.value))); + e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.getName(), change.getValueString())); throw; } } @@ -2716,7 +2715,7 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } -StoragePolicyPtr Context::getOrCreateStoragePolicyForSingleDisk(const String & name) const +StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & name) const { std::lock_guard lock(shared->storage_policies_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5e56825efba..617877c95d3 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -926,7 +926,7 @@ public: /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; - StoragePolicyPtr getOrCreateStoragePolicyForSingleDisk(const String & name) const; + StoragePolicyPtr getStoragePolicyFromDisk(const String & name) const; /// Get the server uptime in seconds. double getUptimeSeconds() const; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4799970b6a1..7a96ee9da9d 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -240,21 +240,25 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) for (const auto & change : set_query.changes) { - if (!settings.has(change.name)) - throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. " - "Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); + if (!settings.has(change.getName())) + throw Exception( + ErrorCodes::UNKNOWN_SETTING, + "Unknown setting {} for EXPLAIN {} query. Supported settings: {}", + doubleQuoteString(change.getName()), Settings::name, settings.getSettingsList()); - if (change.value.getType() != Field::Types::UInt64) + if (change.getFieldValue().getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid type {} for setting \"{}\" only boolean settings are supported", - change.value.getTypeName(), change.name); + change.getFieldValue().getTypeName(), change.getName()); - auto value = change.value.get(); + auto value = change.getFieldValue().get(); if (value > 1) - throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + - "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid value {} for setting {}. Only boolean settings are supported", + value, doubleQuoteString(change.getName())); - settings.setBooleanSetting(change.name, value); + settings.setBooleanSetting(change.getName(), value); } return settings; diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 4af95b96ee3..70ec5f94df0 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -183,7 +183,7 @@ namespace changes, [](const SettingChange & change) { - const String & name = change.name; + const String & name = change.getName(); return (name == "internal") || (name == "async") || (name == "host_id"); }); diff --git a/src/Parsers/ASTDictionary.cpp b/src/Parsers/ASTDictionary.cpp index 66c1c3791b8..d99ecc3fbad 100644 --- a/src/Parsers/ASTDictionary.cpp +++ b/src/Parsers/ASTDictionary.cpp @@ -118,7 +118,7 @@ void ASTDictionarySettings::formatImpl(const FormatSettings & settings, if (it != changes.begin()) settings.ostr << ", "; - settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value); + settings.ostr << it->getName() << " = " << applyVisitor(FieldVisitorToString(), it->getFieldValue()); } settings.ostr << (settings.hilite ? hilite_none : "") << ")"; } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index bc56690a26d..0ed93b9ba5a 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -13,9 +13,9 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const { for (const auto & change : changes) { - hash_state.update(change.name.size()); - hash_state.update(change.name); - applyVisitor(FieldVisitorHash(hash_state), change.value); + hash_state.update(change.getName().size()); + hash_state.update(change.getName()); + applyVisitor(FieldVisitorHash(hash_state), change.getFieldValue()); } } @@ -33,14 +33,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, else first = false; - formatSettingName(change.name, format.ostr); - if (change.value_ast) + formatSettingName(change.getName(), format.ostr); + if (change.getASTValue()) { format.ostr << " = "; - change.value_ast->formatImpl(format, state, stacked); + change.getASTValue()->formatImpl(format, state, stacked); } else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); } for (const auto & [name, value] : query_parameters) diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index 1cebdfde957..0080b6d5540 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -49,7 +49,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS { auto it = std::find_if(insert_settings.begin(), insert_settings.end(), [&](auto & select_setting) { - return select_setting.name == setting.name; + return select_setting.getName() == setting.getName(); }); if (it == insert_settings.end()) insert_settings.push_back(setting); diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.h b/src/Parsers/InsertQuerySettingsPushDownVisitor.h index d1f161fc89b..a32ecc8ee53 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.h +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -struct SettingChange; +class SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause to the INSERT from the SELECT query: diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 649304b1dab..115c7a4b303 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -299,7 +299,7 @@ namespace changes = assert_cast(settings.get())->changes; } - boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); + boost::remove_erase_if(changes, [](const SettingChange & change) { return change.getName() == "async"; }); changes.emplace_back("async", async); auto new_settings = std::make_shared(); diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 92f5b55eab1..0d1e7a3cfee 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -24,11 +24,11 @@ namespace ErrorCodes static NameToNameMap::value_type convertToQueryParameter(SettingChange change) { - auto name = change.name.substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); + auto name = change.getName().substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); if (name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty"); - auto value = applyVisitor(FieldVisitorToString(), change.value); + auto value = applyVisitor(FieldVisitorToString(), change.getFieldValue()); /// writeQuoted is not always quoted in line with SQL standard https://github.com/ClickHouse/ClickHouse/blob/master/src/IO/WriteHelpers.h if (value.starts_with('\'')) { @@ -117,16 +117,16 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p /// for SETTINGS disk=disk(type='s3', path='', ...) else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - tryGetIdentifierNameInto(name, change.name); - change.value_ast = function_ast; + tryGetIdentifierNameInto(name, change.getName()); + change.setASTValue(function_ast); return true; } else if (!literal_or_map_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.name); - change.value = value->as().value; + tryGetIdentifierNameInto(name, change.getName()); + change.setFieldValue(value->as().value); return true; } @@ -162,7 +162,7 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parseNameValuePair(current, pos, expected)) return false; - if (current.name.starts_with(QUERY_PARAMETER_NAME_PREFIX)) + if (current.getName().starts_with(QUERY_PARAMETER_NAME_PREFIX)) query_parameters.emplace(convertToQueryParameter(std::move(current))); else changes.push_back(std::move(current)); diff --git a/src/Parsers/ParserSetQuery.h b/src/Parsers/ParserSetQuery.h index 0bc1cec3093..1cb06d4e86c 100644 --- a/src/Parsers/ParserSetQuery.h +++ b/src/Parsers/ParserSetQuery.h @@ -7,7 +7,7 @@ namespace DB { -struct SettingChange; +class SettingChange; /** Query like this: * SET name1 = value1, name2 = value2, ... diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp index 8cf0d0063ae..e623803410f 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -44,12 +44,12 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query { auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) { - return select_setting.name == setting.name; + return select_setting.getName() == setting.getName(); }); if (it == select_settings.end()) select_settings.push_back(setting); else - it->value = setting.value; + it->setFieldValue(setting.getFieldValue()); } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index fde8a07b555..1aefa16d20c 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -struct SettingChange; +class SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index d68252679a7..f958ac0fe6b 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -625,11 +625,11 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & change : settings_changes) { - auto finder = [&change](const SettingChange & c) { return c.name == change.name; }; + auto finder = [&change](const SettingChange & c) { return c.getName() == change.getName(); }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) - it->value = change.value; + it->setFieldValue(change.getFieldValue()); else settings_from_storage.push_back(change); } @@ -639,7 +639,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & setting_name : settings_resets) { - auto finder = [&setting_name](const SettingChange & c) { return c.name == setting_name; }; + auto finder = [&setting_name](const SettingChange & c) { return c.getName() == setting_name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9a1b06d061f..cef79478b71 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -376,7 +376,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const StoragePolicyPtr storage_policy; if (settings->disk.changed) - storage_policy = context->getOrCreateStoragePolicyForSingleDisk(settings->disk); + storage_policy = context->getStoragePolicyFromDisk(settings->disk); else storage_policy = context->getStoragePolicy(settings->storage_policy); @@ -2607,8 +2607,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { - const auto & setting_name = changed_setting.name; - const auto & new_value = changed_setting.value; + const auto & setting_name = changed_setting.getName(); + const auto & new_value = changed_setting.getFieldValue(); MergeTreeSettings::checkCanSet(setting_name, new_value); const Field * current_value = current_changes.tryGet(setting_name); @@ -2635,7 +2635,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Check if it is safe to reset the settings for (const auto & current_setting : current_changes) { - const auto & setting_name = current_setting.name; + const auto & setting_name = current_setting.getName(); const Field * new_value = new_changes.tryGet(setting_name); /// Prevent unsetting readonly setting if (MergeTreeSettings::isReadonlySetting(setting_name) && !new_value) @@ -2765,9 +2765,9 @@ void MergeTreeData::changeSettings( for (const auto & change : new_changes) { - if (change.name == "storage_policy") + if (change.getName() == "storage_policy") { - StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); + StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.getFieldValue().safeGet()); StoragePolicyPtr old_storage_policy = getStoragePolicy(); /// StoragePolicy of different version or name is guaranteed to have different pointer diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 06f3f5760fb..b15111cd099 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -246,6 +246,8 @@ public: ReadResult read(size_t max_rows, MarkRanges & ranges); + void prefetch(const MarkRanges & ranges); + const Block & getSampleBlock() const { return sample_block; } private: diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 3f51673a6b1..3d90833a3c1 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -90,10 +90,10 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks - need_marks = marks_in_part; - else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_to_read); + // if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + // need_marks = marks_in_part; + // else /// Get whole part to read if it is small enough. + need_marks = std::min(marks_in_part, min_marks_to_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && @@ -203,7 +203,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & const auto & part = parts[i]; bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - do_not_steal_tasks |= part_on_remote_disk; + // do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 5b78a59687b..fd23916aed7 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.getName() == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, (NAME).value}); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 0f71ecf0923..62d9e2034b7 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -609,11 +609,14 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->settings) { - for (auto & [_, value, value_ast] : args.storage_def->settings->changes) + for (auto & change : args.storage_def->settings->changes) { - if (isDiskFunction(value_ast)) + if (isDiskFunction(change.getASTValue())) { - value = createDiskFromDiskAST(*value_ast->as(), context); + const auto & ast_value = assert_cast(*change.getASTValue()); + auto value = createDiskFromDiskAST(ast_value, context); + change.setFieldValue(value); + change.setASTValue(nullptr); break; } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f0c5807f89c..1a4d29e2ddf 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -729,10 +729,10 @@ void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) { - if (setting.name == "materialized_postgresql_max_block_size") - max_block_size = setting.value.safeGet(); - else if (setting.name == "materialized_postgresql_allow_automatic_update") - allow_automatic_update = setting.value.safeGet(); + if (setting.getName() == "materialized_postgresql_max_block_size") + max_block_size = setting.getFieldValue().safeGet(); + else if (setting.getName() == "materialized_postgresql_allow_automatic_update") + allow_automatic_update = setting.getFieldValue().safeGet(); } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 37caa66aae5..9aacf96fe05 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -13,7 +13,7 @@ namespace DB { -struct SettingChange; +class SettingChange; struct StorageInfo { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 89f16457bfe..4b5581fc4b5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -11,7 +11,7 @@ namespace DB { class StorageMaterializedPostgreSQL; -struct SettingChange; +class SettingChange; class PostgreSQLReplicationHandler : WithContext { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 29f2d0667d9..f0449b0d8a9 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1160,9 +1160,9 @@ void registerStorageFile(StorageFactory & factory) const auto & changes = factory_args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) + if (user_format_settings.has(change.getName())) { - user_format_settings.set(change.name, change.value); + user_format_settings.set(change.getName(), change.getFieldValue()); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 2e3e1d443ae..61f48c674b5 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -249,26 +249,28 @@ void registerStorageJoin(StorageFactory & factory) { for (const auto & setting : args.storage_def->settings->changes) { - if (setting.name == "join_use_nulls") - join_use_nulls = setting.value; - else if (setting.name == "max_rows_in_join") - max_rows_in_join = setting.value; - else if (setting.name == "max_bytes_in_join") - max_bytes_in_join = setting.value; - else if (setting.name == "join_overflow_mode") - join_overflow_mode = setting.value; - else if (setting.name == "join_any_take_last_row") - join_any_take_last_row = setting.value; - else if (setting.name == "any_join_distinct_right_table_keys") - old_any_join = setting.value; - else if (setting.name == "disk") - disk_name = setting.value.get(); - else if (setting.name == "persistent") + const auto & setting_name = setting.getName(); + const auto & setting_value = setting.getFieldValue(); + if (setting_name == "join_use_nulls") + join_use_nulls = setting_value; + else if (setting_name == "max_rows_in_join") + max_rows_in_join = setting_value; + else if (setting_name == "max_bytes_in_join") + max_bytes_in_join = setting_value; + else if (setting_name == "join_overflow_mode") + join_overflow_mode = setting_value; + else if (setting_name == "join_any_take_last_row") + join_any_take_last_row = setting_value; + else if (setting_name == "any_join_distinct_right_table_keys") + old_any_join = setting_value; + else if (setting_name == "disk") + disk_name = setting_value.get(); + else if (setting_name == "persistent") { - persistent = setting.value.get(); + persistent = setting_value.get(); } else - throw Exception("Unknown setting " + setting.name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); + throw Exception("Unknown setting " + setting_name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); } } diff --git a/src/Storages/StorageLogSettings.cpp b/src/Storages/StorageLogSettings.cpp index 900e1070eac..e623297d4c0 100644 --- a/src/Storages/StorageLogSettings.cpp +++ b/src/Storages/StorageLogSettings.cpp @@ -10,8 +10,8 @@ String getDiskName(ASTStorage & storage_def) { SettingsChanges changes = storage_def.settings->changes; for (const auto & change : changes) - if (change.name == "disk") - return change.value.safeGet(); + if (change.getName() == "disk") + return change.getFieldValue().safeGet(); } return "default"; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e15956f78be..4f0a3bfbf75 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1300,8 +1300,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); + if (user_format_settings.has(change.getName())) + user_format_settings.set(change.getName(), change.getFieldValue()); } // Apply changes from SETTINGS clause, with validation. diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f1c924a3448..4cb3d8c2a3c 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -955,9 +955,9 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) + if (user_format_settings.has(change.getName())) { - user_format_settings.set(change.name, change.value); + user_format_settings.set(change.getName(), change.getFieldValue()); } } diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 67867b6c577..6714eda3e70 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -122,8 +122,8 @@ static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const for (const auto & change : *task.entry.settings) { Tuple pair; - pair.push_back(change.name); - pair.push_back(toString(change.value)); + pair.push_back(change.getName()); + pair.push_back(toString(change.getFieldValue())); settings_map.push_back(std::move(pair)); } } From bb37f5f2451e64d3f8b04f0aa82a4fc0d941acbf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 7 Nov 2022 17:56:42 +0100 Subject: [PATCH 010/116] Fix style check --- src/Common/SettingsChanges.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 6d047684d41..52ab23dde9e 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -4,6 +4,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) @@ -22,6 +28,7 @@ namespace return &*it; } } + String SettingChange::getValueString() const { if (ast_value) @@ -33,7 +40,7 @@ void SettingChange::throwIfASTValue() const { if (getASTValue() != nullptr) throw Exception( - ErrorCodes::NOT_IMPLEMENTED, + ErrorCodes::LOGICAL_ERROR, "AST value of the setting must be converted to Field value"); } From d876a39e8dc51b73e661366aa699e05e23458efb Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 7 Nov 2022 21:34:28 +0100 Subject: [PATCH 011/116] Fix after merge --- src/Backups/BackupSettings.cpp | 4 ++-- src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Parsers/ParserSetQuery.cpp | 6 +++--- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index c0c8cafdcf5..ab768c83186 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -82,8 +82,8 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "compression_level") - res.compression_level = static_cast(SettingFieldInt64{setting.value}.value); + if (setting.getName() == "compression_level") + res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ if (setting.getName() == #NAME) \ diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index e09aef22122..62644ea9a2d 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -145,7 +145,7 @@ void registerDiskS3(DiskFactory & factory) getSettings(config, config_prefix, context), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); - auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); metadata_storage = std::make_shared(metadata_disk, uri.key); } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 79d1d275b31..582ea4e92b3 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -308,7 +308,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) if (change.getFieldValue().getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid type {} for setting \"{}\" only integer settings are supported", - change.value.getTypeName(), change.name); + change.getFieldValue().getTypeName(), change.getName()); if (settings.hasBooleanSetting(change.getName())) { diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 60df854814f..1ba66c86e24 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -156,11 +156,11 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin else if (!value_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.name); + tryGetIdentifierNameInto(name, change.getName()); if (is_default) - default_settings = change.name; + default_settings = change.getName(); else - change.value = value->as().value; + change.setFieldValue(value->as().value); return true; } From 85e448f04812c4b40bf384005c7db9aa90f96061 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 8 Nov 2022 13:32:49 +0100 Subject: [PATCH 012/116] Fix --- src/Interpreters/Context.cpp | 5 ++++- src/Parsers/ParserSetQuery.cpp | 10 ++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 09d57908f53..1a012ef12b5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2742,7 +2742,10 @@ DiskPtr Context::getOrCreateDisk(const String & name, DiskCreator creator) const auto disk = disk_selector->tryGet(name); if (!disk) - const_cast(disk_selector.get())->addToDiskMap(name, creator(getDisksMap(lock))); + { + disk = creator(getDisksMap(lock)); + const_cast(disk_selector.get())->addToDiskMap(name, disk); + } return disk; } diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 1ba66c86e24..b6ebd76a30f 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -136,10 +136,12 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin ParserCompoundIdentifier name_p; ParserLiteralOrMap value_p; ParserToken s_eq(TokenType::Equals); + ParserFunction function_p; ASTPtr name; ASTPtr value; bool is_default = false; + ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -153,6 +155,14 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin value = std::make_shared(Field(static_cast(0))); else if (ParserKeyword("DEFAULT").ignore(pos, expected)) is_default = true; + /// for SETTINGS disk=disk(type='s3', path='', ...) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + { + tryGetIdentifierNameInto(name, change.getName()); + change.setASTValue(function_ast); + + return true; + } else if (!value_p.parse(pos, value, expected)) return false; From 41852d667fe8622683d73d9802d71214790bb9ed Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 8 Nov 2022 14:44:45 +0100 Subject: [PATCH 013/116] Fix previous commit changes --- src/Common/SettingsChanges.cpp | 26 +++++++++++++++++-- src/Common/SettingsChanges.h | 10 +++---- .../MergeTree/registerStorageMergeTree.cpp | 1 - 3 files changed, 29 insertions(+), 8 deletions(-) diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 52ab23dde9e..0cfa6804e02 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -36,9 +36,31 @@ String SettingChange::getValueString() const return convertFieldToString(field_value); } -void SettingChange::throwIfASTValue() const +const Field & SettingChange::getFieldValue() const { - if (getASTValue() != nullptr) + throwIfASTValueNotConvertedToField(); + return field_value; +} + +Field & SettingChange::getFieldValue() +{ + throwIfASTValueNotConvertedToField(); + return field_value; +} + +void SettingChange::setFieldValue(const Field & field) +{ + field_value = field; +} + +void SettingChange::setASTValue(const ASTPtr & ast) +{ + ast_value = ast ? ast->clone() : ast; +} + +void SettingChange::throwIfASTValueNotConvertedToField() const +{ + if (getASTValue() != nullptr && field_value == Field{}) throw Exception( ErrorCodes::LOGICAL_ERROR, "AST value of the setting must be converted to Field value"); diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index e83a7b7a695..d0d72fd725c 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -30,19 +30,19 @@ public: friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } - void throwIfASTValue() const; + void throwIfASTValueNotConvertedToField() const; const String & getName() const { return name; } String & getName() { return name; } - const Field & getFieldValue() const { throwIfASTValue(); return field_value; } - Field & getFieldValue() { throwIfASTValue(); return field_value; } + const Field & getFieldValue() const; + Field & getFieldValue(); const ASTPtr & getASTValue() const { return ast_value; } ASTPtr & getASTValue() { return ast_value; } - void setFieldValue(const Field & field) { field_value = field; } - void setASTValue(const ASTPtr & ast) { ast_value = ast->clone(); } + void setFieldValue(const Field & field); + void setASTValue(const ASTPtr & ast); String getValueString() const; }; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 94cd3ddf872..83076851b43 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -616,7 +616,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) const auto & ast_value = assert_cast(*change.getASTValue()); auto value = createDiskFromDiskAST(ast_value, context); change.setFieldValue(value); - change.setASTValue(nullptr); break; } } From ba78c2cb55f3cc69bdaa6d43de10865c309f6f4f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 9 Nov 2022 21:27:52 +0100 Subject: [PATCH 014/116] Fix test --- .../0_stateless/02454_create_table_with_custom_disk.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 6fdaf8814d9..20dde68336e 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -9,11 +9,11 @@ SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); +SETTINGS disk = disk(type=local, path='/local/'); -- { serverError BAD_ARGUMENTS } CREATE TABLE test (a Int32) ENGINE = MergeTree() order by tuple() -SETTINGS disk = disk(type=local, path='/local/'); -- { serverError BAD_ARGUMENTS } +SETTINGS disk = disk(type=local, path='/var/lib/clickhouse/disks/local/'); INSERT INTO test SELECT number FROM numbers(100); SELECT count() FROM test; From 5e61022bac0dc90eb4b42ca542b548b1064de1fc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 18 Nov 2022 12:37:34 +0100 Subject: [PATCH 015/116] Revert "Better code around SettingsChanges" This reverts commit bed9f7594af57f335e805dafd63195e13c9115d9. --- programs/client/Client.cpp | 2 +- src/Access/SettingsConstraints.cpp | 16 +++--- src/Access/SettingsConstraints.h | 2 +- src/Backups/BackupSettings.cpp | 6 +-- src/Backups/RestoreSettings.cpp | 7 +-- src/Client/ClientBase.cpp | 4 +- src/Common/SettingsChanges.cpp | 49 ++----------------- src/Common/SettingsChanges.h | 38 ++++---------- src/Core/BaseSettings.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 4 +- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 11 ++--- .../getDictionaryConfigurationFromAST.cpp | 6 +-- src/Interpreters/Context.cpp | 7 +-- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 32 +++++------- src/Parsers/ASTBackupQuery.cpp | 2 +- src/Parsers/ASTDictionary.cpp | 2 +- src/Parsers/ASTSetQuery.cpp | 14 +++--- .../InsertQuerySettingsPushDownVisitor.cpp | 2 +- .../InsertQuerySettingsPushDownVisitor.h | 2 +- src/Parsers/ParserBackupQuery.cpp | 2 +- src/Parsers/ParserSetQuery.cpp | 14 +++--- src/Parsers/ParserSetQuery.h | 2 +- ...QueryWithOutputSettingsPushDownVisitor.cpp | 4 +- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Storages/AlterCommands.cpp | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 12 ++--- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 - src/Storages/MergeTree/MergeTreeReadPool.cpp | 10 ++-- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 8 ++- .../MaterializedPostgreSQLConsumer.cpp | 8 +-- .../MaterializedPostgreSQLConsumer.h | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageJoin.cpp | 36 +++++++------- src/Storages/StorageLogSettings.cpp | 4 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- .../System/StorageSystemDDLWorkerQueue.cpp | 4 +- 41 files changed, 135 insertions(+), 209 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index dc7a54b4611..6e289b57845 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -482,7 +482,7 @@ void Client::printChangedSettings() const { fmt::print(stderr, ", "); } - fmt::print(stderr, "{} = '{}'", changes[i].getName(), changes[i].getValueString()); + fmt::print(stderr, "{} = '{}'", changes[i].name, toString(changes[i].value)); } fmt::print(stderr, "\n"); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 21a35804dcb..0317e43f8d1 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -110,7 +110,7 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - const String & setting_name = change.getName(); + const String & setting_name = change.name; if (setting_name == "profile") return true; @@ -145,7 +145,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh { if (e.code() == ErrorCodes::UNKNOWN_SETTING) { - if (const auto hints = current_settings.getHints(change.getName()); !hints.empty()) + if (const auto hints = current_settings.getHints(change.name); !hints.empty()) { e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); } @@ -160,16 +160,16 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (current_settings.tryGet(setting_name, current_value)) { /// Setting isn't checked if value has not changed. - if (change.getFieldValue() == current_value) + if (change.value == current_value) return false; - new_value = cast_value(change.getFieldValue()); + new_value = cast_value(change.value); if ((new_value == current_value) || cannot_cast) return false; } else { - new_value = cast_value(change.getFieldValue()); + new_value = cast_value(change.value); if (cannot_cast) return false; } @@ -179,7 +179,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const { - const String & setting_name = change.getName(); + const String & setting_name = change.name; auto less_or_cannot_compare = [=](const Field & left, const Field & right) { @@ -234,7 +234,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.setFieldValue(min_value); + change.value = min_value; } if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value)) @@ -246,7 +246,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.setFieldValue(max_value); + change.value = max_value; } return true; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 080e18aefe6..822bf42861b 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,7 +12,7 @@ namespace Poco::Util namespace DB { struct Settings; -class SettingChange; +struct SettingChange; class SettingsChanges; class AccessControl; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index ab768c83186..d297859f1f4 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -86,12 +86,12 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ - if (setting.getName() == #NAME) \ - res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ + if (setting.name == #NAME) \ + res.NAME = SettingField##TYPE{setting.value}.value; \ else LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); } } diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 756c1433c47..63915670fa4 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -173,11 +173,12 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) for (const auto & setting : settings) { #define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \ - if (setting.getName() == #NAME) \ - res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ + if (setting.name == #NAME) \ + res.NAME = SettingField##TYPE{setting.value}.value; \ else + LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index d85ddf47810..1ed9ff58fdc 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1622,8 +1622,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// Save all changes in settings to avoid losing them if the connection is lost. for (const auto & change : set_query->changes) { - if (change.getName() == "profile") - current_profile = change.getFieldValue().safeGet(); + if (change.name == "profile") + current_profile = change.value.safeGet(); else global_context->applySettingChange(change); } diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 0cfa6804e02..005f908c88b 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -1,6 +1,4 @@ #include -#include -#include namespace DB { @@ -14,7 +12,7 @@ namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) return nullptr; return &*it; @@ -22,56 +20,19 @@ namespace const SettingChange * find(const SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) return nullptr; return &*it; } } -String SettingChange::getValueString() const -{ - if (ast_value) - return serializeAST(*ast_value); - return convertFieldToString(field_value); -} - -const Field & SettingChange::getFieldValue() const -{ - throwIfASTValueNotConvertedToField(); - return field_value; -} - -Field & SettingChange::getFieldValue() -{ - throwIfASTValueNotConvertedToField(); - return field_value; -} - -void SettingChange::setFieldValue(const Field & field) -{ - field_value = field; -} - -void SettingChange::setASTValue(const ASTPtr & ast) -{ - ast_value = ast ? ast->clone() : ast; -} - -void SettingChange::throwIfASTValueNotConvertedToField() const -{ - if (getASTValue() != nullptr && field_value == Field{}) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "AST value of the setting must be converted to Field value"); -} - bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); if (!change) return false; - out_value = change->getFieldValue(); + out_value = change->value; return true; } @@ -80,7 +41,7 @@ const Field * SettingsChanges::tryGet(std::string_view name) const const auto * change = find(*this, name); if (!change) return nullptr; - return &change->getFieldValue(); + return &change->value; } Field * SettingsChanges::tryGet(std::string_view name) @@ -88,7 +49,7 @@ Field * SettingsChanges::tryGet(std::string_view name) auto * change = find(*this, name); if (!change) return nullptr; - return &change->getFieldValue(); + return &change->value; } } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index d0d72fd725c..10516cfabd4 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -9,42 +9,22 @@ namespace DB class IColumn; -class SettingChange +struct SettingChange { -private: String name; - Field field_value; - ASTPtr ast_value; /// A setting value which cannot be put in Field. + Field value; + + /// A setting value which cannot be put in Field. + ASTPtr value_ast; -public: SettingChange() = default; - SettingChange(std::string_view name_, const Field & value_) : name(name_), field_value(value_) {} - SettingChange(std::string_view name_, Field && value_) : name(name_), field_value(std::move(value_)) {} - SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), ast_value(value_->clone()) {} - - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) - { - return (lhs.name == rhs.name) && (lhs.field_value == rhs.field_value) && (lhs.ast_value == rhs.ast_value); - } + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} + SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } - - void throwIfASTValueNotConvertedToField() const; - - const String & getName() const { return name; } - String & getName() { return name; } - - const Field & getFieldValue() const; - Field & getFieldValue(); - - const ASTPtr & getASTValue() const { return ast_value; } - ASTPtr & getASTValue() { return ast_value; } - - void setFieldValue(const Field & field); - void setASTValue(const ASTPtr & ast); - - String getValueString() const; }; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 41a3d6d2ee1..7b56367769e 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -293,7 +293,7 @@ SettingsChanges BaseSettings::changes() const template void BaseSettings::applyChange(const SettingChange & change) { - set(change.getName(), change.getFieldValue()); + set(change.name, change.value); } template diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 64b7bee7a63..197f1a0543b 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -741,9 +741,9 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha for (const auto & change : settings_changes) { auto it = std::find_if(storage_settings.begin(), storage_settings.end(), - [&](const auto & prev){ return prev.getName() == change.getName(); }); + [&](const auto & prev){ return prev.name == change.name; }); if (it != storage_settings.end()) - it->setFieldValue(change.getFieldValue()); + it->value = change.value; else storage_settings.push_back(change); } diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 3223452f867..87da701e481 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) SettingsChanges & changes = storage_def.settings->changes; #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.getName() == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, static_cast(NAME)}); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 3263f7e7479..523cc7041be 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -139,18 +139,17 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges for (const auto & change : settings_changes) { - if (!settings->has(change.getName())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.getName()); + if (!settings->has(change.name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name); - if ((change.getName() == "materialized_postgresql_tables_list")) + if ((change.name == "materialized_postgresql_tables_list")) { if (!query_context->isInternalQuery()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.getName()); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.name); need_update_on_disk = true; } - else if ((change.getName() == "materialized_postgresql_allow_automatic_update") - || (change.getName() == "materialized_postgresql_max_block_size")) + else if ((change.name == "materialized_postgresql_allow_automatic_update") || (change.name == "materialized_postgresql_max_block_size")) { replication_handler->setSetting(change); need_update_on_disk = true; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index fa9ff600632..f8d5f9d61db 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -511,11 +511,11 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & change : settings->changes) + for (const auto & [name, value, _] : settings->changes) { - AutoPtr setting_change_element(doc->createElement(change.getName())); + AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue()))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7b5605dadd2..c1798209135 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1388,11 +1388,12 @@ void Context::applySettingChange(const SettingChange & change) { try { - setSetting(change.getName(), change.getFieldValue()); + setSetting(change.name, change.value); } catch (Exception & e) { - e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.getName(), change.getValueString())); + e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", + change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } @@ -2818,7 +2819,7 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } -StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & name) const +StoragePolicyPtr Context::getOrCreateStoragePolicyForSingleDisk(const String & name) const { std::lock_guard lock(shared->storage_policies_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 42f8a9c28b9..537f690408d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -944,7 +944,7 @@ public: /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; - StoragePolicyPtr getStoragePolicyFromDisk(const String & name) const; + StoragePolicyPtr getOrCreateStoragePolicyForSingleDisk(const String & name) const; /// Get the server uptime in seconds. double getUptimeSeconds() const; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index d6e008f8ce9..78900e43bea 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -299,31 +299,21 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) for (const auto & change : set_query.changes) { - if (!settings.has(change.getName())) - throw Exception( - ErrorCodes::UNKNOWN_SETTING, - "Unknown setting {} for EXPLAIN {} query. Supported settings: {}", - doubleQuoteString(change.getName()), Settings::name, settings.getSettingsList()); + if (!settings.has(change.name)) + throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. " + "Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); - if (change.getFieldValue().getType() != Field::Types::UInt64) + if (change.value.getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, - "Invalid type {} for setting \"{}\" only integer settings are supported", - change.getFieldValue().getTypeName(), change.getName()); + "Invalid type {} for setting \"{}\" only boolean settings are supported", + change.value.getTypeName(), change.name); - if (settings.hasBooleanSetting(change.getName())) - { - auto value = change.getFieldValue().get(); - if (value > 1) - throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.getName() + - "\". Expected boolean type", ErrorCodes::INVALID_SETTING_VALUE); + auto value = change.value.get(); + if (value > 1) + throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + + "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); - settings.setBooleanSetting(change.getName(), value); - } - else - { - auto value = change.getFieldValue().get(); - settings.setIntegerSetting(change.getName(), value); - } + settings.setBooleanSetting(change.name, value); } return settings; diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 98dcd0fd2a0..567b52b5669 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -183,7 +183,7 @@ namespace changes, [](const SettingChange & change) { - const String & name = change.getName(); + const String & name = change.name; return (name == "internal") || (name == "async") || (name == "host_id"); }); diff --git a/src/Parsers/ASTDictionary.cpp b/src/Parsers/ASTDictionary.cpp index d99ecc3fbad..66c1c3791b8 100644 --- a/src/Parsers/ASTDictionary.cpp +++ b/src/Parsers/ASTDictionary.cpp @@ -118,7 +118,7 @@ void ASTDictionarySettings::formatImpl(const FormatSettings & settings, if (it != changes.begin()) settings.ostr << ", "; - settings.ostr << it->getName() << " = " << applyVisitor(FieldVisitorToString(), it->getFieldValue()); + settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value); } settings.ostr << (settings.hilite ? hilite_none : "") << ")"; } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index df57b39f282..cf4853ba434 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -13,9 +13,9 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const { for (const auto & change : changes) { - hash_state.update(change.getName().size()); - hash_state.update(change.getName()); - applyVisitor(FieldVisitorHash(hash_state), change.getFieldValue()); + hash_state.update(change.name.size()); + hash_state.update(change.name); + applyVisitor(FieldVisitorHash(hash_state), change.value); } } @@ -33,14 +33,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, else first = false; - formatSettingName(change.getName(), format.ostr); - if (change.getASTValue()) + formatSettingName(change.name, format.ostr); + if (change.value_ast) { format.ostr << " = "; - change.getASTValue()->formatImpl(format, state, stacked); + change.value_ast->formatImpl(format, state, stacked); } else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index 0080b6d5540..1cebdfde957 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -49,7 +49,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS { auto it = std::find_if(insert_settings.begin(), insert_settings.end(), [&](auto & select_setting) { - return select_setting.getName() == setting.getName(); + return select_setting.name == setting.name; }); if (it == insert_settings.end()) insert_settings.push_back(setting); diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.h b/src/Parsers/InsertQuerySettingsPushDownVisitor.h index a32ecc8ee53..d1f161fc89b 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.h +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -class SettingChange; +struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause to the INSERT from the SELECT query: diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 115c7a4b303..649304b1dab 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -299,7 +299,7 @@ namespace changes = assert_cast(settings.get())->changes; } - boost::remove_erase_if(changes, [](const SettingChange & change) { return change.getName() == "async"; }); + boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); changes.emplace_back("async", async); auto new_settings = std::make_shared(); diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index b6ebd76a30f..19351f37385 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -24,11 +24,11 @@ namespace ErrorCodes static NameToNameMap::value_type convertToQueryParameter(SettingChange change) { - auto name = change.getName().substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); + auto name = change.name.substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); if (name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty"); - auto value = applyVisitor(FieldVisitorToString(), change.getFieldValue()); + auto value = applyVisitor(FieldVisitorToString(), change.value); /// writeQuoted is not always quoted in line with SQL standard https://github.com/ClickHouse/ClickHouse/blob/master/src/IO/WriteHelpers.h if (value.starts_with('\'')) { @@ -117,16 +117,16 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p /// for SETTINGS disk=disk(type='s3', path='', ...) else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - tryGetIdentifierNameInto(name, change.getName()); - change.setASTValue(function_ast); + tryGetIdentifierNameInto(name, change.name); + change.value_ast = function_ast; return true; } else if (!literal_or_map_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.getName()); - change.setFieldValue(value->as().value); + tryGetIdentifierNameInto(name, change.name); + change.value = value->as().value; return true; } @@ -208,7 +208,7 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parseNameValuePairWithDefault(current, name_of_default_setting, pos, expected)) return false; - if (current.getName().starts_with(QUERY_PARAMETER_NAME_PREFIX)) + if (current.name.starts_with(QUERY_PARAMETER_NAME_PREFIX)) query_parameters.emplace(convertToQueryParameter(std::move(current))); else if (!name_of_default_setting.empty()) default_settings.emplace_back(std::move(name_of_default_setting)); diff --git a/src/Parsers/ParserSetQuery.h b/src/Parsers/ParserSetQuery.h index 5b5daa6a329..0213667ad7a 100644 --- a/src/Parsers/ParserSetQuery.h +++ b/src/Parsers/ParserSetQuery.h @@ -7,7 +7,7 @@ namespace DB { -class SettingChange; +struct SettingChange; /** Query like this: * SET name1 = value1, name2 = value2, ... diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp index e623803410f..8cf0d0063ae 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -44,12 +44,12 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query { auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) { - return select_setting.getName() == setting.getName(); + return select_setting.name == setting.name; }); if (it == select_settings.end()) select_settings.push_back(setting); else - it->setFieldValue(setting.getFieldValue()); + it->value = setting.value; } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index 1aefa16d20c..fde8a07b555 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -class SettingChange; +struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e390308ad15..c1e7cefd19e 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -625,11 +625,11 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & change : settings_changes) { - auto finder = [&change](const SettingChange & c) { return c.getName() == change.getName(); }; + auto finder = [&change](const SettingChange & c) { return c.name == change.name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) - it->setFieldValue(change.getFieldValue()); + it->value = change.value; else settings_from_storage.push_back(change); } @@ -639,7 +639,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & setting_name : settings_resets) { - auto finder = [&setting_name](const SettingChange & c) { return c.getName() == setting_name; }; + auto finder = [&setting_name](const SettingChange & c) { return c.name == setting_name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ee3499a833..06124fcf2f8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -378,7 +378,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const StoragePolicyPtr storage_policy; if (settings->disk.changed) - storage_policy = context->getStoragePolicyFromDisk(settings->disk); + storage_policy = context->getOrCreateStoragePolicyForSingleDisk(settings->disk); else storage_policy = context->getStoragePolicy(settings->storage_policy); @@ -2647,8 +2647,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { - const auto & setting_name = changed_setting.getName(); - const auto & new_value = changed_setting.getFieldValue(); + const auto & setting_name = changed_setting.name; + const auto & new_value = changed_setting.value; MergeTreeSettings::checkCanSet(setting_name, new_value); const Field * current_value = current_changes.tryGet(setting_name); @@ -2675,7 +2675,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Check if it is safe to reset the settings for (const auto & current_setting : current_changes) { - const auto & setting_name = current_setting.getName(); + const auto & setting_name = current_setting.name; const Field * new_value = new_changes.tryGet(setting_name); /// Prevent unsetting readonly setting if (MergeTreeSettings::isReadonlySetting(setting_name) && !new_value) @@ -2805,9 +2805,9 @@ void MergeTreeData::changeSettings( for (const auto & change : new_changes) { - if (change.getName() == "storage_policy") + if (change.name == "storage_policy") { - StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.getFieldValue().safeGet()); + StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); StoragePolicyPtr old_storage_policy = getStoragePolicy(); /// StoragePolicy of different version or name is guaranteed to have different pointer diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index b15111cd099..06f3f5760fb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -246,8 +246,6 @@ public: ReadResult read(size_t max_rows, MarkRanges & ranges); - void prefetch(const MarkRanges & ranges); - const Block & getSampleBlock() const { return sample_block; } private: diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 4681f8229ab..ca9cde0ae61 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -90,10 +90,10 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - // if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks - // need_marks = marks_in_part; - // else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_to_read); + if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + need_marks = marks_in_part; + else /// Get whole part to read if it is small enough. + need_marks = std::min(marks_in_part, min_marks_to_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && @@ -203,7 +203,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & const auto & part = parts[i]; bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - // do_not_steal_tasks |= part_on_remote_disk; + do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index fd23916aed7..5b78a59687b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.getName() == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, (NAME).value}); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index c130234a8f2..4fc47c1cc11 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -609,13 +609,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->settings) { - for (auto & change : args.storage_def->settings->changes) + for (auto & [_, value, value_ast] : args.storage_def->settings->changes) { - if (isDiskFunction(change.getASTValue())) + if (isDiskFunction(value_ast)) { - const auto & ast_value = assert_cast(*change.getASTValue()); - auto value = createDiskFromDiskAST(ast_value, context); - change.setFieldValue(value); + value = createDiskFromDiskAST(*value_ast->as(), context); break; } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 1a4d29e2ddf..f0c5807f89c 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -729,10 +729,10 @@ void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) { - if (setting.getName() == "materialized_postgresql_max_block_size") - max_block_size = setting.getFieldValue().safeGet(); - else if (setting.getName() == "materialized_postgresql_allow_automatic_update") - allow_automatic_update = setting.getFieldValue().safeGet(); + if (setting.name == "materialized_postgresql_max_block_size") + max_block_size = setting.value.safeGet(); + else if (setting.name == "materialized_postgresql_allow_automatic_update") + allow_automatic_update = setting.value.safeGet(); } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 9aacf96fe05..37caa66aae5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -13,7 +13,7 @@ namespace DB { -class SettingChange; +struct SettingChange; struct StorageInfo { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 4b5581fc4b5..89f16457bfe 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -11,7 +11,7 @@ namespace DB { class StorageMaterializedPostgreSQL; -class SettingChange; +struct SettingChange; class PostgreSQLReplicationHandler : WithContext { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 105824e3100..6e032a47943 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1174,9 +1174,9 @@ void registerStorageFile(StorageFactory & factory) const auto & changes = factory_args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) + if (user_format_settings.has(change.name)) { - user_format_settings.set(change.getName(), change.getFieldValue()); + user_format_settings.set(change.name, change.value); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index b119a445c9a..e4f786cd23b 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -249,28 +249,26 @@ void registerStorageJoin(StorageFactory & factory) { for (const auto & setting : args.storage_def->settings->changes) { - const auto & setting_name = setting.getName(); - const auto & setting_value = setting.getFieldValue(); - if (setting_name == "join_use_nulls") - join_use_nulls = setting_value; - else if (setting_name == "max_rows_in_join") - max_rows_in_join = setting_value; - else if (setting_name == "max_bytes_in_join") - max_bytes_in_join = setting_value; - else if (setting_name == "join_overflow_mode") - join_overflow_mode = setting_value; - else if (setting_name == "join_any_take_last_row") - join_any_take_last_row = setting_value; - else if (setting_name == "any_join_distinct_right_table_keys") - old_any_join = setting_value; - else if (setting_name == "disk") - disk_name = setting_value.get(); - else if (setting_name == "persistent") + if (setting.name == "join_use_nulls") + join_use_nulls = setting.value; + else if (setting.name == "max_rows_in_join") + max_rows_in_join = setting.value; + else if (setting.name == "max_bytes_in_join") + max_bytes_in_join = setting.value; + else if (setting.name == "join_overflow_mode") + join_overflow_mode = setting.value; + else if (setting.name == "join_any_take_last_row") + join_any_take_last_row = setting.value; + else if (setting.name == "any_join_distinct_right_table_keys") + old_any_join = setting.value; + else if (setting.name == "disk") + disk_name = setting.value.get(); + else if (setting.name == "persistent") { - persistent = setting_value.get(); + persistent = setting.value.get(); } else - throw Exception("Unknown setting " + setting_name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); + throw Exception("Unknown setting " + setting.name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); } } diff --git a/src/Storages/StorageLogSettings.cpp b/src/Storages/StorageLogSettings.cpp index e623297d4c0..900e1070eac 100644 --- a/src/Storages/StorageLogSettings.cpp +++ b/src/Storages/StorageLogSettings.cpp @@ -10,8 +10,8 @@ String getDiskName(ASTStorage & storage_def) { SettingsChanges changes = storage_def.settings->changes; for (const auto & change : changes) - if (change.getName() == "disk") - return change.getFieldValue().safeGet(); + if (change.name == "disk") + return change.value.safeGet(); } return "default"; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 08a79f68c5f..ab9b71f5ff3 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1268,8 +1268,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) - user_format_settings.set(change.getName(), change.getFieldValue()); + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); } // Apply changes from SETTINGS clause, with validation. diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 60ceb350364..0f01dc4288c 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -958,9 +958,9 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) + if (user_format_settings.has(change.name)) { - user_format_settings.set(change.getName(), change.getFieldValue()); + user_format_settings.set(change.name, change.value); } } diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 6714eda3e70..67867b6c577 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -122,8 +122,8 @@ static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const for (const auto & change : *task.entry.settings) { Tuple pair; - pair.push_back(change.getName()); - pair.push_back(toString(change.getFieldValue())); + pair.push_back(change.name); + pair.push_back(toString(change.value)); settings_map.push_back(std::move(pair)); } } From 66175cb9c8260ac3d956477c6047f0ab58c8e363 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 18 Nov 2022 13:00:05 +0100 Subject: [PATCH 016/116] Not better --- src/Backups/BackupSettings.cpp | 4 +- src/Common/SettingsChanges.h | 6 +- .../getDictionaryConfigurationFromAST.cpp | 2 +- src/Disks/getDiskConfigurationFromAST.cpp | 93 --------------- src/Disks/getDiskConfigurationFromAST.h | 28 ----- .../maskSensitiveInfoInQueryForLogging.cpp | 31 +++++ src/Parsers/ASTSetQuery.cpp | 10 +- src/Parsers/ParserSetQuery.cpp | 40 ++++--- .../MergeTree/registerStorageMergeTree.cpp | 34 +++--- src/Storages/createDiskFromDiskAST.cpp | 74 ------------ src/Storages/createDiskFromDiskAST.h | 23 ---- src/Storages/getOrCreateDiskFromSettings.cpp | 112 ++++++++++++++++++ src/Storages/getOrCreateDiskFromSettings.h | 21 ++++ 13 files changed, 214 insertions(+), 264 deletions(-) delete mode 100644 src/Disks/getDiskConfigurationFromAST.cpp delete mode 100644 src/Disks/getDiskConfigurationFromAST.h delete mode 100644 src/Storages/createDiskFromDiskAST.cpp delete mode 100644 src/Storages/createDiskFromDiskAST.h create mode 100644 src/Storages/getOrCreateDiskFromSettings.cpp create mode 100644 src/Storages/getOrCreateDiskFromSettings.h diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index d297859f1f4..295ab723326 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -82,8 +82,8 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.getName() == "compression_level") - res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); + if (setting.name == "compression_level") + res.compression_level = static_cast(SettingFieldInt64{setting.value}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ if (setting.name == #NAME) \ diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 10516cfabd4..9e7f874d843 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -14,16 +14,12 @@ struct SettingChange String name; Field value; - /// A setting value which cannot be put in Field. - ASTPtr value_ast; - SettingChange() = default; SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} - SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } }; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index f8d5f9d61db..4868413dabd 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -511,7 +511,7 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & [name, value, _] : settings->changes) + for (const auto & [name, value] : settings->changes) { AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp deleted file mode 100644 index 628defe56ef..00000000000 --- a/src/Disks/getDiskConfigurationFromAST.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -[[noreturn]] static void throwBadConfiguration(const std::string & message = "") -{ - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Incorrect configuration{}. Example of expected configuration: `(type=s3 ...`)`", - message.empty() ? "" : ": " + message); -} - -Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context) -{ - if (disk_args.empty()) - throwBadConfiguration("expected non-empty list of arguments"); - - Poco::AutoPtr xml_document(new Poco::XML::Document()); - Poco::AutoPtr root(xml_document->createElement("disk")); - xml_document->appendChild(root); - Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); - root->appendChild(disk_configuration); - - for (const auto & arg : disk_args) - { - const auto * setting_function = arg->as(); - if (!setting_function || setting_function->name != "equals") - throwBadConfiguration("expected configuration arguments as key=value pairs"); - - const auto * function_args_expr = assert_cast(setting_function->arguments.get()); - if (!function_args_expr) - throwBadConfiguration("expected a list of key=value arguments"); - - auto function_args = function_args_expr->children; - if (function_args.empty()) - throwBadConfiguration("expected a non-empty list of key=value arguments"); - - auto * key_identifier = function_args[0]->as(); - if (!key_identifier) - throwBadConfiguration("expected the key (key=value) to be identifier"); - - const std::string & key = key_identifier->name(); - Poco::AutoPtr key_element(xml_document->createElement(key)); - disk_configuration->appendChild(key_element); - - if (!function_args[1]->as() && !function_args[1]->as()) - throwBadConfiguration("expected values to be literals or identifiers"); - - auto value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); - Poco::AutoPtr value_element(xml_document->createTextNode(convertFieldToString(value->as()->value))); - key_element->appendChild(value_element); - } - - return xml_document; -} - -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) -{ - auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); - - Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); - conf->load(xml_document); - - std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss.exceptions(std::ios::failbit); - conf->save(ss); - LOG_TEST(&Poco::Logger::get("getDiskConfigurationFromAST"), "Received disk configuration: {}", ss.str()); - - return conf; -} - -} diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h deleted file mode 100644 index 1f9d7c1bfe6..00000000000 --- a/src/Disks/getDiskConfigurationFromAST.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -using DiskConfigurationPtr = Poco::AutoPtr; - -/** - * Transform a list of pairs ( key1=value1, key2=value2, ... ), where keys and values are ASTLiteral or ASTIdentifier - * into - * - * value1 - * value2 - * ... - * - * - * Used in case disk configuration is passed via AST when creating - * a disk object on-the-fly without any configuration file. - */ -DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); - -} diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index fe05283eef5..506c3b1e73c 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -169,6 +169,10 @@ namespace /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) wipePasswordFromS3TableEngineArguments(*storage.engine, data); } + else if (storage.engine->arguments && storage.settings) + { + wipeDiskSettingsArguments(*storage.settings, data); + } } static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data) @@ -387,6 +391,33 @@ namespace data.password_was_hidden = true; } + static void wipeDiskSettingsArguments(ASTSetQuery & settings, Data & data) + { + if (settings.changes.empty()) + return; + + for (auto & setting : settings.changes) + { + if (setting.name != "disk") + continue; + + if constexpr (check_only) + { + data.can_contain_password = true; + return; + } + + if (setting.value.getType() != Field::Types::Which::Array) + continue; + + for (auto & disk_setting : setting.value.safeGet()) + disk_setting = Tuple({disk_setting.safeGet()[0], "[HIDDEN]"}); + + data.password_was_hidden = true; + return; + } + } + static void removeArgumentsAfter(ASTFunction & function, Data & data, size_t new_num_arguments) { if (!function.arguments) diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index cf4853ba434..26420f4988c 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -19,7 +19,7 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const } } -void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked stacked) const +void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const { if (is_standalone) format.ostr << (format.hilite ? hilite_keyword : "") << "SET " << (format.hilite ? hilite_none : ""); @@ -34,13 +34,7 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, first = false; formatSettingName(change.name, format.ostr); - if (change.value_ast) - { - format.ostr << " = "; - change.value_ast->formatImpl(format, state, stacked); - } - else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 19351f37385..bc9e5225a5e 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB { @@ -98,11 +99,11 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p ParserLiteralOrMap literal_or_map_p; ParserToken s_eq(TokenType::Equals); ParserSetQuery set_p(true); - ParserFunction function_p; + ParserToken l_br(TokenType::OpeningRoundBracket); + ParserToken r_br(TokenType::ClosingRoundBracket); ASTPtr name; ASTPtr value; - ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -114,12 +115,15 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p value = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); - /// for SETTINGS disk=disk(type='s3', path='', ...) - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + else if (ParserKeyword("SETTINGS").ignore(pos, expected) + && l_br.ignore(pos, expected) + && set_p.parse(pos, value, expected) + && r_br.ignore(pos, expected)) { tryGetIdentifierNameInto(name, change.name); - change.value_ast = function_ast; - + auto changes = value->as()->changes; + auto values = changes | std::views::transform([](const auto & setting) { return Tuple{setting.name, setting.value}; }); + change.value = Array(values.begin(), values.end()); return true; } else if (!literal_or_map_p.parse(pos, value, expected)) @@ -136,12 +140,13 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin ParserCompoundIdentifier name_p; ParserLiteralOrMap value_p; ParserToken s_eq(TokenType::Equals); - ParserFunction function_p; + ParserSetQuery set_p(true); + ParserToken l_br(TokenType::OpeningRoundBracket); + ParserToken r_br(TokenType::ClosingRoundBracket); ASTPtr name; ASTPtr value; bool is_default = false; - ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -155,22 +160,25 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin value = std::make_shared(Field(static_cast(0))); else if (ParserKeyword("DEFAULT").ignore(pos, expected)) is_default = true; - /// for SETTINGS disk=disk(type='s3', path='', ...) - else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") + else if (ParserKeyword("SETTINGS").ignore(pos, expected) + && l_br.ignore(pos, expected) + && set_p.parse(pos, value, expected) + && r_br.ignore(pos, expected)) { - tryGetIdentifierNameInto(name, change.getName()); - change.setASTValue(function_ast); - + tryGetIdentifierNameInto(name, change.name); + auto changes = value->as()->changes; + auto values = changes | std::views::transform([](const auto & setting) { return Tuple{setting.name, setting.value}; }); + change.value = Array(values.begin(), values.end()); return true; } else if (!value_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.getName()); + tryGetIdentifierNameInto(name, change.name); if (is_default) - default_settings = change.getName(); + default_settings = change.name; else - change.setFieldValue(value->as().value); + change.value = value->as().value; return true; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4fc47c1cc11..e8c9e283b9f 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include @@ -24,6 +24,7 @@ #include #include #include +#include namespace DB @@ -607,21 +608,26 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.column_ttls_by_name[name] = new_ttl_entry; } - if (args.storage_def->settings) - { - for (auto & [_, value, value_ast] : args.storage_def->settings->changes) - { - if (isDiskFunction(value_ast)) - { - value = createDiskFromDiskAST(*value_ast->as(), context); - break; - } - } - } - storage_settings->loadFromQuery(*args.storage_def); - // updates the default storage_settings with settings specified via SETTINGS arg in a query + /** + * If settings contain disk setting as disk=setting(), + * create a disk from this configuration and substitute disk setting with result disk name. + */ + Field disk_value; + if (storage_settings->tryGet("disk", disk_value)) + { + auto disk_settings = disk_value.safeGet() | std::views::transform([](const Field & field) + { + auto setting = field.safeGet(); + return SettingChange(setting[0].safeGet(), setting[1]); + }); + auto changes = SettingsChanges(disk_settings.begin(), disk_settings.end()); + auto disk_name = getOrCreateDiskFromSettings(changes, context); + storage_settings->set("disk", disk_name); + } + + /// Updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) metadata.settings_changes = args.storage_def->settings->ptr(); } diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp deleted file mode 100644 index 9abbb5c0a9b..00000000000 --- a/src/Storages/createDiskFromDiskAST.cpp +++ /dev/null @@ -1,74 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -bool isDiskFunction(ASTPtr ast) -{ - if (!ast) - return false; - - const auto * function = ast->as(); - return function && function->name == "disk" && function->arguments->as(); -} - -std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context) -{ - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); - auto disk_name = DiskSelector::TMP_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - - LOG_TRACE( - &Poco::Logger::get("createDiskFromDiskAST"), - "Using disk name `{}` for custom disk {}", - disk_name, disk_setting_string); - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(disk_name, function_args, context); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isRemote()) - { - static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); - - if (disk_path_expected_prefix.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Base path for custom local disks must be defined in config file by `{}`", - custom_disks_base_dir_in_config); - - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path of the custom local disk must be inside `{}` directory", - disk_path_expected_prefix); - } - - return disk_name; -} - -} diff --git a/src/Storages/createDiskFromDiskAST.h b/src/Storages/createDiskFromDiskAST.h deleted file mode 100644 index 6047a494a6f..00000000000 --- a/src/Storages/createDiskFromDiskAST.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** - * Create a DiskPtr from disk AST function like disk(), - * add it to DiskSelector by a unique (but always the same for given configuration) disk name - * and return this name. - */ -std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context); - -/* - * Is given ast has form of a disk() function. - */ -bool isDiskFunction(ASTPtr ast); - -} diff --git a/src/Storages/getOrCreateDiskFromSettings.cpp b/src/Storages/getOrCreateDiskFromSettings.cpp new file mode 100644 index 00000000000..7385a2905c6 --- /dev/null +++ b/src/Storages/getOrCreateDiskFromSettings.cpp @@ -0,0 +1,112 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + using DiskSettings = std::vector>; + + Poco::AutoPtr getDiskConfigurationFromSettingsImpl(const std::string & root_name, const DiskSettings & settings) + { + Poco::AutoPtr xml_document(new Poco::XML::Document()); + Poco::AutoPtr root(xml_document->createElement("disk")); + xml_document->appendChild(root); + Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); + root->appendChild(disk_configuration); + + for (const auto & [name, value] : settings) + { + Poco::AutoPtr key_element(xml_document->createElement(name)); + disk_configuration->appendChild(key_element); + + Poco::AutoPtr value_element(xml_document->createTextNode(value)); + key_element->appendChild(value_element); + } + + return xml_document; + } + + DiskConfigurationPtr getDiskConfigurationFromSettings(const std::string & root_name, const DiskSettings & settings) + { + auto xml_document = getDiskConfigurationFromSettingsImpl(root_name, settings); + Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); + conf->load(xml_document); + return conf; + } +} + +std::string getOrCreateDiskFromSettings(const SettingsChanges & configuration, ContextPtr context) +{ + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + auto settings_range = configuration + | std::views::transform( + [](const auto & setting) { return std::pair(setting.name, convertFieldToString(setting.value)); }); + + DiskSettings sorted_settings(settings_range.begin(), settings_range.end()); + std::sort(sorted_settings.begin(), sorted_settings.end()); + + std::string disk_setting_string; + for (auto it = sorted_settings.begin(); it != sorted_settings.end(); ++it) + { + if (it != sorted_settings.begin()) + disk_setting_string += ","; + disk_setting_string += std::get<0>(*it) + '=' + std::get<1>(*it); + } + + auto disk_name = DiskSelector::TMP_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + + LOG_TRACE( + &Poco::Logger::get("getOrCreateDiskFromSettings"), + "Using disk name `{}` for custom disk {}", + disk_name, disk_setting_string); + + auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + auto config = getDiskConfigurationFromSettings(disk_name, sorted_settings); + auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(); + return disk; + }); + + if (!result_disk->isRemote()) + { + static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + + return disk_name; +} + +} diff --git a/src/Storages/getOrCreateDiskFromSettings.h b/src/Storages/getOrCreateDiskFromSettings.h new file mode 100644 index 00000000000..6c800825ad7 --- /dev/null +++ b/src/Storages/getOrCreateDiskFromSettings.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + + +class SettingsChanges; +using DiskConfigurationPtr = Poco::AutoPtr; + +/** + * Create a DiskPtr from key value list of disk settings. + * add it to DiskSelector by a unique (but always the same for given configuration) disk name + * and return this name. + */ +std::string getOrCreateDiskFromSettings(const SettingsChanges & configuration, ContextPtr context); + +} From babc0c84b08e5d399b52a7f8765dc7e6aca093d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 18 Nov 2022 13:00:46 +0100 Subject: [PATCH 017/116] Revert "Not better" This reverts commit 66175cb9c8260ac3d956477c6047f0ab58c8e363. --- src/Backups/BackupSettings.cpp | 4 +- src/Common/SettingsChanges.h | 6 +- .../getDictionaryConfigurationFromAST.cpp | 2 +- src/Disks/getDiskConfigurationFromAST.cpp | 93 +++++++++++++++ src/Disks/getDiskConfigurationFromAST.h | 28 +++++ .../maskSensitiveInfoInQueryForLogging.cpp | 31 ----- src/Parsers/ASTSetQuery.cpp | 10 +- src/Parsers/ParserSetQuery.cpp | 40 +++---- .../MergeTree/registerStorageMergeTree.cpp | 30 ++--- src/Storages/createDiskFromDiskAST.cpp | 74 ++++++++++++ src/Storages/createDiskFromDiskAST.h | 23 ++++ src/Storages/getOrCreateDiskFromSettings.cpp | 112 ------------------ src/Storages/getOrCreateDiskFromSettings.h | 21 ---- 13 files changed, 262 insertions(+), 212 deletions(-) create mode 100644 src/Disks/getDiskConfigurationFromAST.cpp create mode 100644 src/Disks/getDiskConfigurationFromAST.h create mode 100644 src/Storages/createDiskFromDiskAST.cpp create mode 100644 src/Storages/createDiskFromDiskAST.h delete mode 100644 src/Storages/getOrCreateDiskFromSettings.cpp delete mode 100644 src/Storages/getOrCreateDiskFromSettings.h diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 295ab723326..d297859f1f4 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -82,8 +82,8 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.name == "compression_level") - res.compression_level = static_cast(SettingFieldInt64{setting.value}.value); + if (setting.getName() == "compression_level") + res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ if (setting.name == #NAME) \ diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 9e7f874d843..10516cfabd4 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -14,12 +14,16 @@ struct SettingChange String name; Field value; + /// A setting value which cannot be put in Field. + ASTPtr value_ast; + SettingChange() = default; SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} + SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } }; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 4868413dabd..f8d5f9d61db 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -511,7 +511,7 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & [name, value] : settings->changes) + for (const auto & [name, value, _] : settings->changes) { AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp new file mode 100644 index 00000000000..628defe56ef --- /dev/null +++ b/src/Disks/getDiskConfigurationFromAST.cpp @@ -0,0 +1,93 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +[[noreturn]] static void throwBadConfiguration(const std::string & message = "") +{ + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Incorrect configuration{}. Example of expected configuration: `(type=s3 ...`)`", + message.empty() ? "" : ": " + message); +} + +Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +{ + if (disk_args.empty()) + throwBadConfiguration("expected non-empty list of arguments"); + + Poco::AutoPtr xml_document(new Poco::XML::Document()); + Poco::AutoPtr root(xml_document->createElement("disk")); + xml_document->appendChild(root); + Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); + root->appendChild(disk_configuration); + + for (const auto & arg : disk_args) + { + const auto * setting_function = arg->as(); + if (!setting_function || setting_function->name != "equals") + throwBadConfiguration("expected configuration arguments as key=value pairs"); + + const auto * function_args_expr = assert_cast(setting_function->arguments.get()); + if (!function_args_expr) + throwBadConfiguration("expected a list of key=value arguments"); + + auto function_args = function_args_expr->children; + if (function_args.empty()) + throwBadConfiguration("expected a non-empty list of key=value arguments"); + + auto * key_identifier = function_args[0]->as(); + if (!key_identifier) + throwBadConfiguration("expected the key (key=value) to be identifier"); + + const std::string & key = key_identifier->name(); + Poco::AutoPtr key_element(xml_document->createElement(key)); + disk_configuration->appendChild(key_element); + + if (!function_args[1]->as() && !function_args[1]->as()) + throwBadConfiguration("expected values to be literals or identifiers"); + + auto value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); + Poco::AutoPtr value_element(xml_document->createTextNode(convertFieldToString(value->as()->value))); + key_element->appendChild(value_element); + } + + return xml_document; +} + +DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) +{ + auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); + + Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); + conf->load(xml_document); + + std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + ss.exceptions(std::ios::failbit); + conf->save(ss); + LOG_TEST(&Poco::Logger::get("getDiskConfigurationFromAST"), "Received disk configuration: {}", ss.str()); + + return conf; +} + +} diff --git a/src/Disks/getDiskConfigurationFromAST.h b/src/Disks/getDiskConfigurationFromAST.h new file mode 100644 index 00000000000..1f9d7c1bfe6 --- /dev/null +++ b/src/Disks/getDiskConfigurationFromAST.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +using DiskConfigurationPtr = Poco::AutoPtr; + +/** + * Transform a list of pairs ( key1=value1, key2=value2, ... ), where keys and values are ASTLiteral or ASTIdentifier + * into + * + * value1 + * value2 + * ... + * + * + * Used in case disk configuration is passed via AST when creating + * a disk object on-the-fly without any configuration file. + */ +DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context); + +} diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index 506c3b1e73c..fe05283eef5 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -169,10 +169,6 @@ namespace /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) wipePasswordFromS3TableEngineArguments(*storage.engine, data); } - else if (storage.engine->arguments && storage.settings) - { - wipeDiskSettingsArguments(*storage.settings, data); - } } static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data) @@ -391,33 +387,6 @@ namespace data.password_was_hidden = true; } - static void wipeDiskSettingsArguments(ASTSetQuery & settings, Data & data) - { - if (settings.changes.empty()) - return; - - for (auto & setting : settings.changes) - { - if (setting.name != "disk") - continue; - - if constexpr (check_only) - { - data.can_contain_password = true; - return; - } - - if (setting.value.getType() != Field::Types::Which::Array) - continue; - - for (auto & disk_setting : setting.value.safeGet()) - disk_setting = Tuple({disk_setting.safeGet()[0], "[HIDDEN]"}); - - data.password_was_hidden = true; - return; - } - } - static void removeArgumentsAfter(ASTFunction & function, Data & data, size_t new_num_arguments) { if (!function.arguments) diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 26420f4988c..cf4853ba434 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -19,7 +19,7 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const } } -void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const +void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked stacked) const { if (is_standalone) format.ostr << (format.hilite ? hilite_keyword : "") << "SET " << (format.hilite ? hilite_none : ""); @@ -34,7 +34,13 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma first = false; formatSettingName(change.name, format.ostr); - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + if (change.value_ast) + { + format.ostr << " = "; + change.value_ast->formatImpl(format, state, stacked); + } + else + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index bc9e5225a5e..19351f37385 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -13,7 +13,6 @@ #include #include #include -#include namespace DB { @@ -99,11 +98,11 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p ParserLiteralOrMap literal_or_map_p; ParserToken s_eq(TokenType::Equals); ParserSetQuery set_p(true); - ParserToken l_br(TokenType::OpeningRoundBracket); - ParserToken r_br(TokenType::ClosingRoundBracket); + ParserFunction function_p; ASTPtr name; ASTPtr value; + ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -115,15 +114,12 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p value = std::make_shared(Field(static_cast(1))); else if (ParserKeyword("FALSE").ignore(pos, expected)) value = std::make_shared(Field(static_cast(0))); - else if (ParserKeyword("SETTINGS").ignore(pos, expected) - && l_br.ignore(pos, expected) - && set_p.parse(pos, value, expected) - && r_br.ignore(pos, expected)) + /// for SETTINGS disk=disk(type='s3', path='', ...) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { tryGetIdentifierNameInto(name, change.name); - auto changes = value->as()->changes; - auto values = changes | std::views::transform([](const auto & setting) { return Tuple{setting.name, setting.value}; }); - change.value = Array(values.begin(), values.end()); + change.value_ast = function_ast; + return true; } else if (!literal_or_map_p.parse(pos, value, expected)) @@ -140,13 +136,12 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin ParserCompoundIdentifier name_p; ParserLiteralOrMap value_p; ParserToken s_eq(TokenType::Equals); - ParserSetQuery set_p(true); - ParserToken l_br(TokenType::OpeningRoundBracket); - ParserToken r_br(TokenType::ClosingRoundBracket); + ParserFunction function_p; ASTPtr name; ASTPtr value; bool is_default = false; + ASTPtr function_ast; if (!name_p.parse(pos, name, expected)) return false; @@ -160,25 +155,22 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin value = std::make_shared(Field(static_cast(0))); else if (ParserKeyword("DEFAULT").ignore(pos, expected)) is_default = true; - else if (ParserKeyword("SETTINGS").ignore(pos, expected) - && l_br.ignore(pos, expected) - && set_p.parse(pos, value, expected) - && r_br.ignore(pos, expected)) + /// for SETTINGS disk=disk(type='s3', path='', ...) + else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - tryGetIdentifierNameInto(name, change.name); - auto changes = value->as()->changes; - auto values = changes | std::views::transform([](const auto & setting) { return Tuple{setting.name, setting.value}; }); - change.value = Array(values.begin(), values.end()); + tryGetIdentifierNameInto(name, change.getName()); + change.setASTValue(function_ast); + return true; } else if (!value_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.name); + tryGetIdentifierNameInto(name, change.getName()); if (is_default) - default_settings = change.name; + default_settings = change.getName(); else - change.value = value->as().value; + change.setFieldValue(value->as().value); return true; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e8c9e283b9f..4fc47c1cc11 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include @@ -24,7 +24,6 @@ #include #include #include -#include namespace DB @@ -608,26 +607,21 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.column_ttls_by_name[name] = new_ttl_entry; } - storage_settings->loadFromQuery(*args.storage_def); - - /** - * If settings contain disk setting as disk=setting(), - * create a disk from this configuration and substitute disk setting with result disk name. - */ - Field disk_value; - if (storage_settings->tryGet("disk", disk_value)) + if (args.storage_def->settings) { - auto disk_settings = disk_value.safeGet() | std::views::transform([](const Field & field) + for (auto & [_, value, value_ast] : args.storage_def->settings->changes) { - auto setting = field.safeGet(); - return SettingChange(setting[0].safeGet(), setting[1]); - }); - auto changes = SettingsChanges(disk_settings.begin(), disk_settings.end()); - auto disk_name = getOrCreateDiskFromSettings(changes, context); - storage_settings->set("disk", disk_name); + if (isDiskFunction(value_ast)) + { + value = createDiskFromDiskAST(*value_ast->as(), context); + break; + } + } } - /// Updates the default storage_settings with settings specified via SETTINGS arg in a query + storage_settings->loadFromQuery(*args.storage_def); + + // updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) metadata.settings_changes = args.storage_def->settings->ptr(); } diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp new file mode 100644 index 00000000000..9abbb5c0a9b --- /dev/null +++ b/src/Storages/createDiskFromDiskAST.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +bool isDiskFunction(ASTPtr ast) +{ + if (!ast) + return false; + + const auto * function = ast->as(); + return function && function->name == "disk" && function->arguments->as(); +} + +std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context) +{ + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + auto disk_setting_string = serializeAST(function, true); + auto disk_name = DiskSelector::TMP_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + + LOG_TRACE( + &Poco::Logger::get("createDiskFromDiskAST"), + "Using disk name `{}` for custom disk {}", + disk_name, disk_setting_string); + + auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(disk_name, function_args, context); + auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(); + return disk; + }); + + if (!result_disk->isRemote()) + { + static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + + return disk_name; +} + +} diff --git a/src/Storages/createDiskFromDiskAST.h b/src/Storages/createDiskFromDiskAST.h new file mode 100644 index 00000000000..6047a494a6f --- /dev/null +++ b/src/Storages/createDiskFromDiskAST.h @@ -0,0 +1,23 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class ASTFunction; + +/** + * Create a DiskPtr from disk AST function like disk(), + * add it to DiskSelector by a unique (but always the same for given configuration) disk name + * and return this name. + */ +std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context); + +/* + * Is given ast has form of a disk() function. + */ +bool isDiskFunction(ASTPtr ast); + +} diff --git a/src/Storages/getOrCreateDiskFromSettings.cpp b/src/Storages/getOrCreateDiskFromSettings.cpp deleted file mode 100644 index 7385a2905c6..00000000000 --- a/src/Storages/getOrCreateDiskFromSettings.cpp +++ /dev/null @@ -1,112 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -namespace -{ - using DiskSettings = std::vector>; - - Poco::AutoPtr getDiskConfigurationFromSettingsImpl(const std::string & root_name, const DiskSettings & settings) - { - Poco::AutoPtr xml_document(new Poco::XML::Document()); - Poco::AutoPtr root(xml_document->createElement("disk")); - xml_document->appendChild(root); - Poco::AutoPtr disk_configuration(xml_document->createElement(root_name)); - root->appendChild(disk_configuration); - - for (const auto & [name, value] : settings) - { - Poco::AutoPtr key_element(xml_document->createElement(name)); - disk_configuration->appendChild(key_element); - - Poco::AutoPtr value_element(xml_document->createTextNode(value)); - key_element->appendChild(value_element); - } - - return xml_document; - } - - DiskConfigurationPtr getDiskConfigurationFromSettings(const std::string & root_name, const DiskSettings & settings) - { - auto xml_document = getDiskConfigurationFromSettingsImpl(root_name, settings); - Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); - conf->load(xml_document); - return conf; - } -} - -std::string getOrCreateDiskFromSettings(const SettingsChanges & configuration, ContextPtr context) -{ - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto settings_range = configuration - | std::views::transform( - [](const auto & setting) { return std::pair(setting.name, convertFieldToString(setting.value)); }); - - DiskSettings sorted_settings(settings_range.begin(), settings_range.end()); - std::sort(sorted_settings.begin(), sorted_settings.end()); - - std::string disk_setting_string; - for (auto it = sorted_settings.begin(); it != sorted_settings.end(); ++it) - { - if (it != sorted_settings.begin()) - disk_setting_string += ","; - disk_setting_string += std::get<0>(*it) + '=' + std::get<1>(*it); - } - - auto disk_name = DiskSelector::TMP_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - - LOG_TRACE( - &Poco::Logger::get("getOrCreateDiskFromSettings"), - "Using disk name `{}` for custom disk {}", - disk_name, disk_setting_string); - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - auto config = getDiskConfigurationFromSettings(disk_name, sorted_settings); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isRemote()) - { - static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); - - if (disk_path_expected_prefix.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Base path for custom local disks must be defined in config file by `{}`", - custom_disks_base_dir_in_config); - - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path of the custom local disk must be inside `{}` directory", - disk_path_expected_prefix); - } - - return disk_name; -} - -} diff --git a/src/Storages/getOrCreateDiskFromSettings.h b/src/Storages/getOrCreateDiskFromSettings.h deleted file mode 100644 index 6c800825ad7..00000000000 --- a/src/Storages/getOrCreateDiskFromSettings.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - - -class SettingsChanges; -using DiskConfigurationPtr = Poco::AutoPtr; - -/** - * Create a DiskPtr from key value list of disk settings. - * add it to DiskSelector by a unique (but always the same for given configuration) disk name - * and return this name. - */ -std::string getOrCreateDiskFromSettings(const SettingsChanges & configuration, ContextPtr context); - -} From 41c1a2aad45fa7149a99276c464e7e0bd0baa056 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 18 Nov 2022 13:16:46 +0100 Subject: [PATCH 018/116] Revert "Revert "Better code around SettingsChanges"" This reverts commit 5e61022bac0dc90eb4b42ca542b548b1064de1fc. --- programs/client/Client.cpp | 2 +- src/Access/SettingsConstraints.cpp | 16 +++--- src/Access/SettingsConstraints.h | 2 +- src/Backups/BackupSettings.cpp | 6 +-- src/Backups/RestoreSettings.cpp | 7 ++- src/Client/ClientBase.cpp | 4 +- src/Common/SettingsChanges.cpp | 49 +++++++++++++++++-- src/Common/SettingsChanges.h | 38 ++++++++++---- src/Core/BaseSettings.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 4 +- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 11 +++-- .../getDictionaryConfigurationFromAST.cpp | 6 +-- src/Interpreters/Context.cpp | 7 ++- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 32 +++++++----- src/Parsers/ASTBackupQuery.cpp | 2 +- src/Parsers/ASTDictionary.cpp | 2 +- src/Parsers/ASTSetQuery.cpp | 14 +++--- .../InsertQuerySettingsPushDownVisitor.cpp | 2 +- .../InsertQuerySettingsPushDownVisitor.h | 2 +- src/Parsers/ParserBackupQuery.cpp | 2 +- src/Parsers/ParserSetQuery.cpp | 14 +++--- src/Parsers/ParserSetQuery.h | 2 +- ...QueryWithOutputSettingsPushDownVisitor.cpp | 4 +- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Storages/AlterCommands.cpp | 6 +-- src/Storages/MergeTree/MergeTreeData.cpp | 12 ++--- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 + src/Storages/MergeTree/MergeTreeReadPool.cpp | 10 ++-- src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 8 +-- .../MaterializedPostgreSQLConsumer.cpp | 8 +-- .../MaterializedPostgreSQLConsumer.h | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageJoin.cpp | 36 +++++++------- src/Storages/StorageLogSettings.cpp | 4 +- src/Storages/StorageS3.cpp | 4 +- src/Storages/StorageURL.cpp | 4 +- .../System/StorageSystemDDLWorkerQueue.cpp | 4 +- 41 files changed, 209 insertions(+), 135 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6e289b57845..dc7a54b4611 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -482,7 +482,7 @@ void Client::printChangedSettings() const { fmt::print(stderr, ", "); } - fmt::print(stderr, "{} = '{}'", changes[i].name, toString(changes[i].value)); + fmt::print(stderr, "{} = '{}'", changes[i].getName(), changes[i].getValueString()); } fmt::print(stderr, "\n"); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 0317e43f8d1..21a35804dcb 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -110,7 +110,7 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - const String & setting_name = change.name; + const String & setting_name = change.getName(); if (setting_name == "profile") return true; @@ -145,7 +145,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh { if (e.code() == ErrorCodes::UNKNOWN_SETTING) { - if (const auto hints = current_settings.getHints(change.name); !hints.empty()) + if (const auto hints = current_settings.getHints(change.getName()); !hints.empty()) { e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); } @@ -160,16 +160,16 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (current_settings.tryGet(setting_name, current_value)) { /// Setting isn't checked if value has not changed. - if (change.value == current_value) + if (change.getFieldValue() == current_value) return false; - new_value = cast_value(change.value); + new_value = cast_value(change.getFieldValue()); if ((new_value == current_value) || cannot_cast) return false; } else { - new_value = cast_value(change.value); + new_value = cast_value(change.getFieldValue()); if (cannot_cast) return false; } @@ -179,7 +179,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const { - const String & setting_name = change.name; + const String & setting_name = change.getName(); auto less_or_cannot_compare = [=](const Field & left, const Field & right) { @@ -234,7 +234,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.value = min_value; + change.setFieldValue(min_value); } if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value)) @@ -246,7 +246,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.value = max_value; + change.setFieldValue(max_value); } return true; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 822bf42861b..080e18aefe6 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,7 +12,7 @@ namespace Poco::Util namespace DB { struct Settings; -struct SettingChange; +class SettingChange; class SettingsChanges; class AccessControl; diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index d297859f1f4..ab768c83186 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -86,12 +86,12 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ - if (setting.name == #NAME) \ - res.NAME = SettingField##TYPE{setting.value}.value; \ + if (setting.getName() == #NAME) \ + res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ else LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); } } diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 63915670fa4..756c1433c47 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -173,12 +173,11 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) for (const auto & setting : settings) { #define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \ - if (setting.name == #NAME) \ - res.NAME = SettingField##TYPE{setting.value}.value; \ + if (setting.getName() == #NAME) \ + res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ else - LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 1ed9ff58fdc..d85ddf47810 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1622,8 +1622,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// Save all changes in settings to avoid losing them if the connection is lost. for (const auto & change : set_query->changes) { - if (change.name == "profile") - current_profile = change.value.safeGet(); + if (change.getName() == "profile") + current_profile = change.getFieldValue().safeGet(); else global_context->applySettingChange(change); } diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 005f908c88b..0cfa6804e02 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -1,4 +1,6 @@ #include +#include +#include namespace DB { @@ -12,7 +14,7 @@ namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); if (it == changes.end()) return nullptr; return &*it; @@ -20,19 +22,56 @@ namespace const SettingChange * find(const SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); if (it == changes.end()) return nullptr; return &*it; } } +String SettingChange::getValueString() const +{ + if (ast_value) + return serializeAST(*ast_value); + return convertFieldToString(field_value); +} + +const Field & SettingChange::getFieldValue() const +{ + throwIfASTValueNotConvertedToField(); + return field_value; +} + +Field & SettingChange::getFieldValue() +{ + throwIfASTValueNotConvertedToField(); + return field_value; +} + +void SettingChange::setFieldValue(const Field & field) +{ + field_value = field; +} + +void SettingChange::setASTValue(const ASTPtr & ast) +{ + ast_value = ast ? ast->clone() : ast; +} + +void SettingChange::throwIfASTValueNotConvertedToField() const +{ + if (getASTValue() != nullptr && field_value == Field{}) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "AST value of the setting must be converted to Field value"); +} + bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); if (!change) return false; - out_value = change->value; + out_value = change->getFieldValue(); return true; } @@ -41,7 +80,7 @@ const Field * SettingsChanges::tryGet(std::string_view name) const const auto * change = find(*this, name); if (!change) return nullptr; - return &change->value; + return &change->getFieldValue(); } Field * SettingsChanges::tryGet(std::string_view name) @@ -49,7 +88,7 @@ Field * SettingsChanges::tryGet(std::string_view name) auto * change = find(*this, name); if (!change) return nullptr; - return &change->value; + return &change->getFieldValue(); } } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 10516cfabd4..d0d72fd725c 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -9,22 +9,42 @@ namespace DB class IColumn; -struct SettingChange +class SettingChange { +private: String name; - Field value; - - /// A setting value which cannot be put in Field. - ASTPtr value_ast; + Field field_value; + ASTPtr ast_value; /// A setting value which cannot be put in Field. +public: SettingChange() = default; - SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} - SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} - SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), value_ast(value_->clone()) {} + SettingChange(std::string_view name_, const Field & value_) : name(name_), field_value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), field_value(std::move(value_)) {} + SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), ast_value(value_->clone()) {} + + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) + { + return (lhs.name == rhs.name) && (lhs.field_value == rhs.field_value) && (lhs.ast_value == rhs.ast_value); + } - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value) && (lhs.value_ast == rhs.value_ast); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } + + void throwIfASTValueNotConvertedToField() const; + + const String & getName() const { return name; } + String & getName() { return name; } + + const Field & getFieldValue() const; + Field & getFieldValue(); + + const ASTPtr & getASTValue() const { return ast_value; } + ASTPtr & getASTValue() { return ast_value; } + + void setFieldValue(const Field & field); + void setASTValue(const ASTPtr & ast); + + String getValueString() const; }; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 7b56367769e..41a3d6d2ee1 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -293,7 +293,7 @@ SettingsChanges BaseSettings::changes() const template void BaseSettings::applyChange(const SettingChange & change) { - set(change.name, change.value); + set(change.getName(), change.getFieldValue()); } template diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 197f1a0543b..64b7bee7a63 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -741,9 +741,9 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha for (const auto & change : settings_changes) { auto it = std::find_if(storage_settings.begin(), storage_settings.end(), - [&](const auto & prev){ return prev.name == change.name; }); + [&](const auto & prev){ return prev.getName() == change.getName(); }); if (it != storage_settings.end()) - it->value = change.value; + it->setFieldValue(change.getFieldValue()); else storage_settings.push_back(change); } diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 87da701e481..3223452f867 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) SettingsChanges & changes = storage_def.settings->changes; #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.getName() == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, static_cast(NAME)}); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 523cc7041be..3263f7e7479 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -139,17 +139,18 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges for (const auto & change : settings_changes) { - if (!settings->has(change.name)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name); + if (!settings->has(change.getName())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.getName()); - if ((change.name == "materialized_postgresql_tables_list")) + if ((change.getName() == "materialized_postgresql_tables_list")) { if (!query_context->isInternalQuery()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.name); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.getName()); need_update_on_disk = true; } - else if ((change.name == "materialized_postgresql_allow_automatic_update") || (change.name == "materialized_postgresql_max_block_size")) + else if ((change.getName() == "materialized_postgresql_allow_automatic_update") + || (change.getName() == "materialized_postgresql_max_block_size")) { replication_handler->setSetting(change); need_update_on_disk = true; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index f8d5f9d61db..fa9ff600632 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -511,11 +511,11 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & [name, value, _] : settings->changes) + for (const auto & change : settings->changes) { - AutoPtr setting_change_element(doc->createElement(name)); + AutoPtr setting_change_element(doc->createElement(change.getName())); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue()))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c1798209135..7b5605dadd2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1388,12 +1388,11 @@ void Context::applySettingChange(const SettingChange & change) { try { - setSetting(change.name, change.value); + setSetting(change.getName(), change.getFieldValue()); } catch (Exception & e) { - e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", - change.name, applyVisitor(FieldVisitorToString(), change.value))); + e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.getName(), change.getValueString())); throw; } } @@ -2819,7 +2818,7 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } -StoragePolicyPtr Context::getOrCreateStoragePolicyForSingleDisk(const String & name) const +StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & name) const { std::lock_guard lock(shared->storage_policies_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 537f690408d..42f8a9c28b9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -944,7 +944,7 @@ public: /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; - StoragePolicyPtr getOrCreateStoragePolicyForSingleDisk(const String & name) const; + StoragePolicyPtr getStoragePolicyFromDisk(const String & name) const; /// Get the server uptime in seconds. double getUptimeSeconds() const; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 78900e43bea..d6e008f8ce9 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -299,21 +299,31 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) for (const auto & change : set_query.changes) { - if (!settings.has(change.name)) - throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. " - "Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); + if (!settings.has(change.getName())) + throw Exception( + ErrorCodes::UNKNOWN_SETTING, + "Unknown setting {} for EXPLAIN {} query. Supported settings: {}", + doubleQuoteString(change.getName()), Settings::name, settings.getSettingsList()); - if (change.value.getType() != Field::Types::UInt64) + if (change.getFieldValue().getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, - "Invalid type {} for setting \"{}\" only boolean settings are supported", - change.value.getTypeName(), change.name); + "Invalid type {} for setting \"{}\" only integer settings are supported", + change.getFieldValue().getTypeName(), change.getName()); - auto value = change.value.get(); - if (value > 1) - throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + - "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); + if (settings.hasBooleanSetting(change.getName())) + { + auto value = change.getFieldValue().get(); + if (value > 1) + throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.getName() + + "\". Expected boolean type", ErrorCodes::INVALID_SETTING_VALUE); - settings.setBooleanSetting(change.name, value); + settings.setBooleanSetting(change.getName(), value); + } + else + { + auto value = change.getFieldValue().get(); + settings.setIntegerSetting(change.getName(), value); + } } return settings; diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 567b52b5669..98dcd0fd2a0 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -183,7 +183,7 @@ namespace changes, [](const SettingChange & change) { - const String & name = change.name; + const String & name = change.getName(); return (name == "internal") || (name == "async") || (name == "host_id"); }); diff --git a/src/Parsers/ASTDictionary.cpp b/src/Parsers/ASTDictionary.cpp index 66c1c3791b8..d99ecc3fbad 100644 --- a/src/Parsers/ASTDictionary.cpp +++ b/src/Parsers/ASTDictionary.cpp @@ -118,7 +118,7 @@ void ASTDictionarySettings::formatImpl(const FormatSettings & settings, if (it != changes.begin()) settings.ostr << ", "; - settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value); + settings.ostr << it->getName() << " = " << applyVisitor(FieldVisitorToString(), it->getFieldValue()); } settings.ostr << (settings.hilite ? hilite_none : "") << ")"; } diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index cf4853ba434..df57b39f282 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -13,9 +13,9 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const { for (const auto & change : changes) { - hash_state.update(change.name.size()); - hash_state.update(change.name); - applyVisitor(FieldVisitorHash(hash_state), change.value); + hash_state.update(change.getName().size()); + hash_state.update(change.getName()); + applyVisitor(FieldVisitorHash(hash_state), change.getFieldValue()); } } @@ -33,14 +33,14 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, else first = false; - formatSettingName(change.name, format.ostr); - if (change.value_ast) + formatSettingName(change.getName(), format.ostr); + if (change.getASTValue()) { format.ostr << " = "; - change.value_ast->formatImpl(format, state, stacked); + change.getASTValue()->formatImpl(format, state, stacked); } else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index 1cebdfde957..0080b6d5540 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -49,7 +49,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS { auto it = std::find_if(insert_settings.begin(), insert_settings.end(), [&](auto & select_setting) { - return select_setting.name == setting.name; + return select_setting.getName() == setting.getName(); }); if (it == insert_settings.end()) insert_settings.push_back(setting); diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.h b/src/Parsers/InsertQuerySettingsPushDownVisitor.h index d1f161fc89b..a32ecc8ee53 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.h +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -struct SettingChange; +class SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause to the INSERT from the SELECT query: diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 649304b1dab..115c7a4b303 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -299,7 +299,7 @@ namespace changes = assert_cast(settings.get())->changes; } - boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); + boost::remove_erase_if(changes, [](const SettingChange & change) { return change.getName() == "async"; }); changes.emplace_back("async", async); auto new_settings = std::make_shared(); diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index 19351f37385..b6ebd76a30f 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -24,11 +24,11 @@ namespace ErrorCodes static NameToNameMap::value_type convertToQueryParameter(SettingChange change) { - auto name = change.name.substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); + auto name = change.getName().substr(strlen(QUERY_PARAMETER_NAME_PREFIX)); if (name.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter name cannot be empty"); - auto value = applyVisitor(FieldVisitorToString(), change.value); + auto value = applyVisitor(FieldVisitorToString(), change.getFieldValue()); /// writeQuoted is not always quoted in line with SQL standard https://github.com/ClickHouse/ClickHouse/blob/master/src/IO/WriteHelpers.h if (value.starts_with('\'')) { @@ -117,16 +117,16 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p /// for SETTINGS disk=disk(type='s3', path='', ...) else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - tryGetIdentifierNameInto(name, change.name); - change.value_ast = function_ast; + tryGetIdentifierNameInto(name, change.getName()); + change.setASTValue(function_ast); return true; } else if (!literal_or_map_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.name); - change.value = value->as().value; + tryGetIdentifierNameInto(name, change.getName()); + change.setFieldValue(value->as().value); return true; } @@ -208,7 +208,7 @@ bool ParserSetQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parseNameValuePairWithDefault(current, name_of_default_setting, pos, expected)) return false; - if (current.name.starts_with(QUERY_PARAMETER_NAME_PREFIX)) + if (current.getName().starts_with(QUERY_PARAMETER_NAME_PREFIX)) query_parameters.emplace(convertToQueryParameter(std::move(current))); else if (!name_of_default_setting.empty()) default_settings.emplace_back(std::move(name_of_default_setting)); diff --git a/src/Parsers/ParserSetQuery.h b/src/Parsers/ParserSetQuery.h index 0213667ad7a..5b5daa6a329 100644 --- a/src/Parsers/ParserSetQuery.h +++ b/src/Parsers/ParserSetQuery.h @@ -7,7 +7,7 @@ namespace DB { -struct SettingChange; +class SettingChange; /** Query like this: * SET name1 = value1, name2 = value2, ... diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp index 8cf0d0063ae..e623803410f 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -44,12 +44,12 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query { auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) { - return select_setting.name == setting.name; + return select_setting.getName() == setting.getName(); }); if (it == select_settings.end()) select_settings.push_back(setting); else - it->value = setting.value; + it->setFieldValue(setting.getFieldValue()); } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index fde8a07b555..1aefa16d20c 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -struct SettingChange; +class SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index c1e7cefd19e..e390308ad15 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -625,11 +625,11 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & change : settings_changes) { - auto finder = [&change](const SettingChange & c) { return c.name == change.name; }; + auto finder = [&change](const SettingChange & c) { return c.getName() == change.getName(); }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) - it->value = change.value; + it->setFieldValue(change.getFieldValue()); else settings_from_storage.push_back(change); } @@ -639,7 +639,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & setting_name : settings_resets) { - auto finder = [&setting_name](const SettingChange & c) { return c.name == setting_name; }; + auto finder = [&setting_name](const SettingChange & c) { return c.getName() == setting_name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 06124fcf2f8..0ee3499a833 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -378,7 +378,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const StoragePolicyPtr storage_policy; if (settings->disk.changed) - storage_policy = context->getOrCreateStoragePolicyForSingleDisk(settings->disk); + storage_policy = context->getStoragePolicyFromDisk(settings->disk); else storage_policy = context->getStoragePolicy(settings->storage_policy); @@ -2647,8 +2647,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_changes = new_metadata.settings_changes->as().changes; for (const auto & changed_setting : new_changes) { - const auto & setting_name = changed_setting.name; - const auto & new_value = changed_setting.value; + const auto & setting_name = changed_setting.getName(); + const auto & new_value = changed_setting.getFieldValue(); MergeTreeSettings::checkCanSet(setting_name, new_value); const Field * current_value = current_changes.tryGet(setting_name); @@ -2675,7 +2675,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Check if it is safe to reset the settings for (const auto & current_setting : current_changes) { - const auto & setting_name = current_setting.name; + const auto & setting_name = current_setting.getName(); const Field * new_value = new_changes.tryGet(setting_name); /// Prevent unsetting readonly setting if (MergeTreeSettings::isReadonlySetting(setting_name) && !new_value) @@ -2805,9 +2805,9 @@ void MergeTreeData::changeSettings( for (const auto & change : new_changes) { - if (change.name == "storage_policy") + if (change.getName() == "storage_policy") { - StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); + StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.getFieldValue().safeGet()); StoragePolicyPtr old_storage_policy = getStoragePolicy(); /// StoragePolicy of different version or name is guaranteed to have different pointer diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 06f3f5760fb..b15111cd099 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -246,6 +246,8 @@ public: ReadResult read(size_t max_rows, MarkRanges & ranges); + void prefetch(const MarkRanges & ranges); + const Block & getSampleBlock() const { return sample_block; } private: diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index ca9cde0ae61..4681f8229ab 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -90,10 +90,10 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks - need_marks = marks_in_part; - else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_to_read); + // if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + // need_marks = marks_in_part; + // else /// Get whole part to read if it is small enough. + need_marks = std::min(marks_in_part, min_marks_to_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && @@ -203,7 +203,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & const auto & part = parts[i]; bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - do_not_steal_tasks |= part_on_remote_disk; + // do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 5b78a59687b..fd23916aed7 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.getName() == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, (NAME).value}); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 4fc47c1cc11..c130234a8f2 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -609,11 +609,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->settings) { - for (auto & [_, value, value_ast] : args.storage_def->settings->changes) + for (auto & change : args.storage_def->settings->changes) { - if (isDiskFunction(value_ast)) + if (isDiskFunction(change.getASTValue())) { - value = createDiskFromDiskAST(*value_ast->as(), context); + const auto & ast_value = assert_cast(*change.getASTValue()); + auto value = createDiskFromDiskAST(ast_value, context); + change.setFieldValue(value); break; } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index f0c5807f89c..1a4d29e2ddf 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -729,10 +729,10 @@ void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) { - if (setting.name == "materialized_postgresql_max_block_size") - max_block_size = setting.value.safeGet(); - else if (setting.name == "materialized_postgresql_allow_automatic_update") - allow_automatic_update = setting.value.safeGet(); + if (setting.getName() == "materialized_postgresql_max_block_size") + max_block_size = setting.getFieldValue().safeGet(); + else if (setting.getName() == "materialized_postgresql_allow_automatic_update") + allow_automatic_update = setting.getFieldValue().safeGet(); } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 37caa66aae5..9aacf96fe05 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -13,7 +13,7 @@ namespace DB { -struct SettingChange; +class SettingChange; struct StorageInfo { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 89f16457bfe..4b5581fc4b5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -11,7 +11,7 @@ namespace DB { class StorageMaterializedPostgreSQL; -struct SettingChange; +class SettingChange; class PostgreSQLReplicationHandler : WithContext { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6e032a47943..105824e3100 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1174,9 +1174,9 @@ void registerStorageFile(StorageFactory & factory) const auto & changes = factory_args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) + if (user_format_settings.has(change.getName())) { - user_format_settings.set(change.name, change.value); + user_format_settings.set(change.getName(), change.getFieldValue()); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index e4f786cd23b..b119a445c9a 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -249,26 +249,28 @@ void registerStorageJoin(StorageFactory & factory) { for (const auto & setting : args.storage_def->settings->changes) { - if (setting.name == "join_use_nulls") - join_use_nulls = setting.value; - else if (setting.name == "max_rows_in_join") - max_rows_in_join = setting.value; - else if (setting.name == "max_bytes_in_join") - max_bytes_in_join = setting.value; - else if (setting.name == "join_overflow_mode") - join_overflow_mode = setting.value; - else if (setting.name == "join_any_take_last_row") - join_any_take_last_row = setting.value; - else if (setting.name == "any_join_distinct_right_table_keys") - old_any_join = setting.value; - else if (setting.name == "disk") - disk_name = setting.value.get(); - else if (setting.name == "persistent") + const auto & setting_name = setting.getName(); + const auto & setting_value = setting.getFieldValue(); + if (setting_name == "join_use_nulls") + join_use_nulls = setting_value; + else if (setting_name == "max_rows_in_join") + max_rows_in_join = setting_value; + else if (setting_name == "max_bytes_in_join") + max_bytes_in_join = setting_value; + else if (setting_name == "join_overflow_mode") + join_overflow_mode = setting_value; + else if (setting_name == "join_any_take_last_row") + join_any_take_last_row = setting_value; + else if (setting_name == "any_join_distinct_right_table_keys") + old_any_join = setting_value; + else if (setting_name == "disk") + disk_name = setting_value.get(); + else if (setting_name == "persistent") { - persistent = setting.value.get(); + persistent = setting_value.get(); } else - throw Exception("Unknown setting " + setting.name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); + throw Exception("Unknown setting " + setting_name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); } } diff --git a/src/Storages/StorageLogSettings.cpp b/src/Storages/StorageLogSettings.cpp index 900e1070eac..e623297d4c0 100644 --- a/src/Storages/StorageLogSettings.cpp +++ b/src/Storages/StorageLogSettings.cpp @@ -10,8 +10,8 @@ String getDiskName(ASTStorage & storage_def) { SettingsChanges changes = storage_def.settings->changes; for (const auto & change : changes) - if (change.name == "disk") - return change.value.safeGet(); + if (change.getName() == "disk") + return change.getFieldValue().safeGet(); } return "default"; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ab9b71f5ff3..08a79f68c5f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1268,8 +1268,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); + if (user_format_settings.has(change.getName())) + user_format_settings.set(change.getName(), change.getFieldValue()); } // Apply changes from SETTINGS clause, with validation. diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 0f01dc4288c..60ceb350364 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -958,9 +958,9 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.name)) + if (user_format_settings.has(change.getName())) { - user_format_settings.set(change.name, change.value); + user_format_settings.set(change.getName(), change.getFieldValue()); } } diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 67867b6c577..6714eda3e70 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -122,8 +122,8 @@ static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const for (const auto & change : *task.entry.settings) { Tuple pair; - pair.push_back(change.name); - pair.push_back(toString(change.value)); + pair.push_back(change.getName()); + pair.push_back(toString(change.getFieldValue())); settings_map.push_back(std::move(pair)); } } From 30c5344de4c116f2ae1de4af8be7415d41f16450 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 18 Nov 2022 19:29:45 +0100 Subject: [PATCH 019/116] Get rid of ASTPtr in settings changes --- src/Access/SettingsConstraints.cpp | 4 +- src/Common/ErrorCodes.cpp | 1 + src/Common/SettingsChanges.cpp | 59 ++++++--------- src/Common/SettingsChanges.h | 42 +++++++---- src/Databases/DatabaseOnDisk.cpp | 2 +- src/Disks/getDiskConfigurationFromAST.cpp | 7 -- .../maskSensitiveInfoInQueryForLogging.cpp | 29 ++++++++ src/Parsers/ASTSetQuery.cpp | 10 +-- src/Parsers/ParserSetQuery.cpp | 9 ++- ...QueryWithOutputSettingsPushDownVisitor.cpp | 2 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 3 - .../MergeTree/registerStorageMergeTree.cpp | 18 +++-- src/Storages/createDiskFromDiskAST.cpp | 74 ------------------- src/Storages/createDiskFromDiskAST.h | 23 ------ 15 files changed, 103 insertions(+), 182 deletions(-) delete mode 100644 src/Storages/createDiskFromDiskAST.cpp delete mode 100644 src/Storages/createDiskFromDiskAST.h diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 21a35804dcb..023cae7ea32 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -234,7 +234,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.setFieldValue(min_value); + change.setValue(min_value); } if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value)) @@ -246,7 +246,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n ErrorCodes::SETTING_CONSTRAINT_VIOLATION); } else - change.setFieldValue(max_value); + change.setValue(max_value); } return true; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2bc5d70421a..b5b226e4adb 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -639,6 +639,7 @@ M(668, INVALID_STATE) \ M(669, UNKNOWN_NAMED_COLLECTION) \ M(670, NAMED_COLLECTION_ALREADY_EXISTS) \ + M(671, CANNOT_GET_SETTING_VALUE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 0cfa6804e02..d9c18122119 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -29,43 +29,6 @@ namespace } } -String SettingChange::getValueString() const -{ - if (ast_value) - return serializeAST(*ast_value); - return convertFieldToString(field_value); -} - -const Field & SettingChange::getFieldValue() const -{ - throwIfASTValueNotConvertedToField(); - return field_value; -} - -Field & SettingChange::getFieldValue() -{ - throwIfASTValueNotConvertedToField(); - return field_value; -} - -void SettingChange::setFieldValue(const Field & field) -{ - field_value = field; -} - -void SettingChange::setASTValue(const ASTPtr & ast) -{ - ast_value = ast ? ast->clone() : ast; -} - -void SettingChange::throwIfASTValueNotConvertedToField() const -{ - if (getASTValue() != nullptr && field_value == Field{}) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "AST value of the setting must be converted to Field value"); -} - bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const { const auto * change = find(*this, name); @@ -91,4 +54,26 @@ Field * SettingsChanges::tryGet(std::string_view name) return &change->getFieldValue(); } +struct SettingValueFromField : SettingValue +{ + explicit SettingValueFromField(const Field & value_) : value(value_) {} + explicit SettingValueFromField(Field && value_) : value(std::move(value_)) {} + + const Field & getField() const override { return value; } + Field & getField() override { return value; } + std::string toString() const override { return applyVisitor(FieldVisitorToString(), value); } + + Field value; +}; + +SettingValuePtr getSettingValueFromField(const Field & field) +{ + return std::make_shared(field); +} + +SettingValuePtr getSettingValueFromField(Field && field) +{ + return std::make_shared(std::move(field)); +} + } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index d0d72fd725c..57ccfe46b95 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -9,42 +10,51 @@ namespace DB class IColumn; + +struct SettingValue +{ + virtual ~SettingValue() = default; + virtual const Field & getField() const = 0; + virtual Field & getField() = 0; + virtual std::string toString() const = 0; +}; +using SettingValuePtr = std::shared_ptr; + +SettingValuePtr getSettingValueFromField(const Field & field); +SettingValuePtr getSettingValueFromField(Field && field); + + class SettingChange { private: String name; - Field field_value; - ASTPtr ast_value; /// A setting value which cannot be put in Field. + SettingValuePtr value; public: SettingChange() = default; - SettingChange(std::string_view name_, const Field & value_) : name(name_), field_value(value_) {} - SettingChange(std::string_view name_, Field && value_) : name(name_), field_value(std::move(value_)) {} - SettingChange(std::string_view name_, const ASTPtr & value_) : name(name_), ast_value(value_->clone()) {} + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(getSettingValueFromField(value_)) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), value(getSettingValueFromField(std::move(value_))) {} + SettingChange(std::string_view name_, SettingValuePtr && value_) : name(name_), value(std::move(value_)) {} friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { - return (lhs.name == rhs.name) && (lhs.field_value == rhs.field_value) && (lhs.ast_value == rhs.ast_value); + return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } - void throwIfASTValueNotConvertedToField() const; - const String & getName() const { return name; } String & getName() { return name; } - const Field & getFieldValue() const; - Field & getFieldValue(); + SettingValuePtr getValue() const { return value; } + const Field & getFieldValue() const { return value->getField(); } + Field & getFieldValue() { return value->getField(); } - const ASTPtr & getASTValue() const { return ast_value; } - ASTPtr & getASTValue() { return ast_value; } + void setValue(const Field & field) { value = getSettingValueFromField(field); } + void setValue(SettingValuePtr value_) { value = std::move(value_); } - void setFieldValue(const Field & field); - void setASTValue(const ASTPtr & ast); - - String getValueString() const; + String getValueString() const { return value->toString(); } }; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 64b7bee7a63..383cc44be42 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -743,7 +743,7 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha auto it = std::find_if(storage_settings.begin(), storage_settings.end(), [&](const auto & prev){ return prev.getName() == change.getName(); }); if (it != storage_settings.end()) - it->setFieldValue(change.getFieldValue()); + it->setValue(change.getFieldValue()); else storage_settings.push_back(change); } diff --git a/src/Disks/getDiskConfigurationFromAST.cpp b/src/Disks/getDiskConfigurationFromAST.cpp index 628defe56ef..e6b08046036 100644 --- a/src/Disks/getDiskConfigurationFromAST.cpp +++ b/src/Disks/getDiskConfigurationFromAST.cpp @@ -78,15 +78,8 @@ Poco::AutoPtr getDiskConfigurationFromASTImpl(const std::st DiskConfigurationPtr getDiskConfigurationFromAST(const std::string & root_name, const ASTs & disk_args, ContextPtr context) { auto xml_document = getDiskConfigurationFromASTImpl(root_name, disk_args, context); - Poco::AutoPtr conf(new Poco::Util::XMLConfiguration()); conf->load(xml_document); - - std::ostringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - ss.exceptions(std::ios::failbit); - conf->save(ss); - LOG_TEST(&Poco::Logger::get("getDiskConfigurationFromAST"), "Received disk configuration: {}", ss.str()); - return conf; } diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index fe05283eef5..69cf53366d6 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -9,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -169,6 +171,10 @@ namespace /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) wipePasswordFromS3TableEngineArguments(*storage.engine, data); } + else if (storage.settings) + { + wipeSensitiveInfoFromStorageSettings(storage.settings->changes, data); + } } static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data) @@ -387,6 +393,29 @@ namespace data.password_was_hidden = true; } + static void wipeSensitiveInfoFromStorageSettings(SettingsChanges & changes, Data & data) + { + if (changes.empty()) + return; + + for (auto & change : changes) + { + auto value = change.getValue(); + auto * ast_value = dynamic_cast(value.get()); + if (ast_value && isDiskFunction(ast_value->value)) + { + if constexpr (check_only) + { + data.can_contain_password = true; + return; + } + change.setValue("[HIDDEN]"); + } + } + + data.password_was_hidden = true; + } + static void removeArgumentsAfter(ASTFunction & function, Data & data, size_t new_num_arguments) { if (!function.arguments) diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index df57b39f282..db9209120f2 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -19,7 +19,7 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const } } -void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, FormatStateStacked stacked) const +void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const { if (is_standalone) format.ostr << (format.hilite ? hilite_keyword : "") << "SET " << (format.hilite ? hilite_none : ""); @@ -34,13 +34,7 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState & state, first = false; formatSettingName(change.getName(), format.ostr); - if (change.getASTValue()) - { - format.ostr << " = "; - change.getASTValue()->formatImpl(format, state, stacked); - } - else - format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); + format.ostr << " = " << change.getValueString(); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index b6ebd76a30f..a3226d71f5b 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -118,7 +119,7 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { tryGetIdentifierNameInto(name, change.getName()); - change.setASTValue(function_ast); + change.setValue(std::make_unique(function_ast)); return true; } @@ -126,7 +127,7 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p return false; tryGetIdentifierNameInto(name, change.getName()); - change.setFieldValue(value->as().value); + change.setValue(value->as().value); return true; } @@ -159,7 +160,7 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { tryGetIdentifierNameInto(name, change.getName()); - change.setASTValue(function_ast); + change.setValue(std::make_unique(function_ast)); return true; } @@ -170,7 +171,7 @@ bool ParserSetQuery::parseNameValuePairWithDefault(SettingChange & change, Strin if (is_default) default_settings = change.getName(); else - change.setFieldValue(value->as().value); + change.setValue(value->as().value); return true; } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp index e623803410f..12670cd67b6 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -49,7 +49,7 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query if (it == select_settings.end()) select_settings.push_back(setting); else - it->setFieldValue(setting.getFieldValue()); + it->setValue(setting.getFieldValue()); } } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e390308ad15..a91a34e6a95 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -629,7 +629,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) - it->setFieldValue(change.getFieldValue()); + it->setValue(change.getFieldValue()); else settings_from_storage.push_back(change); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ee3499a833..bca437fff11 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -238,9 +238,6 @@ MergeTreeData::MergeTreeData( const auto settings = getSettings(); - if (settings->disk.changed && settings->storage_policy.changed) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); - allow_nullable_key = attach || settings->allow_nullable_key; if (relative_data_path.empty()) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index c130234a8f2..f3ba081a87e 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -1,11 +1,11 @@ #include +#include #include #include #include #include #include #include -#include #include #include @@ -17,6 +17,7 @@ #include #include #include +#include #include #include @@ -611,11 +612,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) { for (auto & change : args.storage_def->settings->changes) { - if (isDiskFunction(change.getASTValue())) + auto value = change.getValue(); + auto * ast_value = dynamic_cast(value.get()); + if (ast_value && isDiskFunction(ast_value->value)) { - const auto & ast_value = assert_cast(*change.getASTValue()); - auto value = createDiskFromDiskAST(ast_value, context); - change.setFieldValue(value); + const auto & ast_function = assert_cast(*ast_value->value); + auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); + ast_value->setField(disk_name); break; } } @@ -623,6 +626,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) storage_settings->loadFromQuery(*args.storage_def); + if (storage_settings->disk.changed && storage_settings->storage_policy.changed) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); + // updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) metadata.settings_changes = args.storage_def->settings->ptr(); diff --git a/src/Storages/createDiskFromDiskAST.cpp b/src/Storages/createDiskFromDiskAST.cpp deleted file mode 100644 index 9abbb5c0a9b..00000000000 --- a/src/Storages/createDiskFromDiskAST.cpp +++ /dev/null @@ -1,74 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -bool isDiskFunction(ASTPtr ast) -{ - if (!ast) - return false; - - const auto * function = ast->as(); - return function && function->name == "disk" && function->arguments->as(); -} - -std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context) -{ - /// We need a unique name for a created custom disk, but it needs to be the same - /// after table is reattached or server is restarted, so take a hash of the disk - /// configuration serialized ast as a disk name suffix. - auto disk_setting_string = serializeAST(function, true); - auto disk_name = DiskSelector::TMP_DISK_PREFIX - + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); - - LOG_TRACE( - &Poco::Logger::get("createDiskFromDiskAST"), - "Using disk name `{}` for custom disk {}", - disk_name, disk_setting_string); - - auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { - const auto * function_args_expr = assert_cast(function.arguments.get()); - const auto & function_args = function_args_expr->children; - auto config = getDiskConfigurationFromAST(disk_name, function_args, context); - auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); - /// Mark that disk can be used without storage policy. - disk->markDiskAsCustom(); - return disk; - }); - - if (!result_disk->isRemote()) - { - static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; - auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); - - if (disk_path_expected_prefix.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Base path for custom local disks must be defined in config file by `{}`", - custom_disks_base_dir_in_config); - - if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Path of the custom local disk must be inside `{}` directory", - disk_path_expected_prefix); - } - - return disk_name; -} - -} diff --git a/src/Storages/createDiskFromDiskAST.h b/src/Storages/createDiskFromDiskAST.h deleted file mode 100644 index 6047a494a6f..00000000000 --- a/src/Storages/createDiskFromDiskAST.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** - * Create a DiskPtr from disk AST function like disk(), - * add it to DiskSelector by a unique (but always the same for given configuration) disk name - * and return this name. - */ -std::string createDiskFromDiskAST(const ASTFunction & function, ContextPtr context); - -/* - * Is given ast has form of a disk() function. - */ -bool isDiskFunction(ASTPtr ast); - -} From 93365b0b290bd959e17d5f1bfb706f6098873527 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 20 Nov 2022 19:28:11 +0100 Subject: [PATCH 020/116] Commit forgotten files --- src/Common/SettingsChanges.cpp | 5 -- src/Common/SettingsChanges.h | 3 -- src/Disks/getOrCreateDiskFromAST.cpp | 74 ++++++++++++++++++++++++++++ src/Disks/getOrCreateDiskFromAST.h | 23 +++++++++ src/Parsers/SettingValueFromAST.h | 44 +++++++++++++++++ 5 files changed, 141 insertions(+), 8 deletions(-) create mode 100644 src/Disks/getOrCreateDiskFromAST.cpp create mode 100644 src/Disks/getOrCreateDiskFromAST.h create mode 100644 src/Parsers/SettingValueFromAST.h diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index d9c18122119..188947e3f45 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -5,11 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 57ccfe46b95..01856e9edad 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -1,8 +1,5 @@ #pragma once - #include -#include -#include namespace DB diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp new file mode 100644 index 00000000000..48d40864396 --- /dev/null +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +bool isDiskFunction(ASTPtr ast) +{ + if (!ast) + return false; + + const auto * function = ast->as(); + return function && function->name == "disk" && function->arguments->as(); +} + +std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context) +{ + /// We need a unique name for a created custom disk, but it needs to be the same + /// after table is reattached or server is restarted, so take a hash of the disk + /// configuration serialized ast as a disk name suffix. + auto disk_setting_string = serializeAST(function, true); + auto disk_name = DiskSelector::TMP_DISK_PREFIX + + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); + + LOG_TRACE( + &Poco::Logger::get("getOrCreateDiskFromDiskAST"), + "Using disk name `{}` for custom disk {}", + disk_name, disk_setting_string); + + auto result_disk = context->getOrCreateDisk(disk_name, [&](const DisksMap & disks_map) -> DiskPtr { + const auto * function_args_expr = assert_cast(function.arguments.get()); + const auto & function_args = function_args_expr->children; + auto config = getDiskConfigurationFromAST(disk_name, function_args, context); + auto disk = DiskFactory::instance().create(disk_name, *config, disk_name, context, disks_map); + /// Mark that disk can be used without storage policy. + disk->markDiskAsCustom(); + return disk; + }); + + if (!result_disk->isRemote()) + { + static constexpr auto custom_disks_base_dir_in_config = "custom_local_disks_base_directory"; + auto disk_path_expected_prefix = context->getConfigRef().getString(custom_disks_base_dir_in_config, ""); + + if (disk_path_expected_prefix.empty()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Base path for custom local disks must be defined in config file by `{}`", + custom_disks_base_dir_in_config); + + if (!pathStartsWith(result_disk->getPath(), disk_path_expected_prefix)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path of the custom local disk must be inside `{}` directory", + disk_path_expected_prefix); + } + + return disk_name; +} + +} diff --git a/src/Disks/getOrCreateDiskFromAST.h b/src/Disks/getOrCreateDiskFromAST.h new file mode 100644 index 00000000000..c1d4bda1a49 --- /dev/null +++ b/src/Disks/getOrCreateDiskFromAST.h @@ -0,0 +1,23 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class ASTFunction; + +/** + * Create a DiskPtr from disk AST function like disk(), + * add it to DiskSelector by a unique (but always the same for given configuration) disk name + * and return this name. + */ +std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr context); + +/* + * Is given ast has form of a disk() function. + */ +bool isDiskFunction(ASTPtr ast); + +} diff --git a/src/Parsers/SettingValueFromAST.h b/src/Parsers/SettingValueFromAST.h new file mode 100644 index 00000000000..5c0ed4c248f --- /dev/null +++ b/src/Parsers/SettingValueFromAST.h @@ -0,0 +1,44 @@ +#pragma once +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_GET_SETTING_VALUE; +} + +struct SettingValueFromAST : SettingValue +{ + explicit SettingValueFromAST(const ASTPtr & value_) : value(value_) {} + + ASTPtr value; + std::optional field; + + [[noreturn]] void throwNoValue() const + { + throw Exception( + ErrorCodes::CANNOT_GET_SETTING_VALUE, + "Cannot get setting value, it must be converted from AST to Field first"); + } + + const Field & getField() const override + { + if (field) + return *field; + throwNoValue(); + } + + Field & getField() override + { + if (field) + return *field; + throwNoValue(); + } + void setField(const Field & field_) { field = field_; } + std::string toString() const override { return serializeAST(*value); } +}; + +} From bc078dd154547d38c28f004199d0242124d943ef Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Dec 2022 15:50:57 +0100 Subject: [PATCH 021/116] Fix build --- src/Parsers/ASTAlterNamedCollectionQuery.cpp | 4 ++-- src/Parsers/ASTCreateNamedCollectionQuery.cpp | 4 ++-- src/Storages/NamedCollectionConfiguration.cpp | 4 ++-- src/Storages/NamedCollectionUtils.cpp | 18 +++++++++--------- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Parsers/ASTAlterNamedCollectionQuery.cpp b/src/Parsers/ASTAlterNamedCollectionQuery.cpp index 7e95147ad75..7e7b36acb42 100644 --- a/src/Parsers/ASTAlterNamedCollectionQuery.cpp +++ b/src/Parsers/ASTAlterNamedCollectionQuery.cpp @@ -28,9 +28,9 @@ void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & setti else first = false; - formatSettingName(change.name, settings.ostr); + formatSettingName(change.getName(), settings.ostr); if (settings.show_secrets) - settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); else settings.ostr << " = '[HIDDEN]'"; } diff --git a/src/Parsers/ASTCreateNamedCollectionQuery.cpp b/src/Parsers/ASTCreateNamedCollectionQuery.cpp index 97e83541f05..56b6259fc39 100644 --- a/src/Parsers/ASTCreateNamedCollectionQuery.cpp +++ b/src/Parsers/ASTCreateNamedCollectionQuery.cpp @@ -31,10 +31,10 @@ void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & sett else first = false; - formatSettingName(change.name, settings.ostr); + formatSettingName(change.getName(), settings.ostr); if (settings.show_secrets) - settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); + settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); else settings.ostr << " = '[HIDDEN]'"; } diff --git a/src/Storages/NamedCollectionConfiguration.cpp b/src/Storages/NamedCollectionConfiguration.cpp index b0e7bdce32a..1b5f7984266 100644 --- a/src/Storages/NamedCollectionConfiguration.cpp +++ b/src/Storages/NamedCollectionConfiguration.cpp @@ -128,8 +128,8 @@ ConfigurationPtr createConfiguration(const std::string & root_name, const Settin namespace Configuration = NamedCollectionConfiguration; auto config = Configuration::createEmptyConfiguration(root_name); - for (const auto & [name, value] : settings) - Configuration::setConfigValue(*config, name, convertFieldToString(value)); + for (const auto & setting : settings) + Configuration::setConfigValue(*config, setting.getName(), convertFieldToString(setting.getFieldValue())); return config; } diff --git a/src/Storages/NamedCollectionUtils.cpp b/src/Storages/NamedCollectionUtils.cpp index 75d5aace664..2c0b1c2e6c7 100644 --- a/src/Storages/NamedCollectionUtils.cpp +++ b/src/Storages/NamedCollectionUtils.cpp @@ -224,20 +224,20 @@ public: auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings()); std::unordered_map result_changes_map; - for (const auto & [name, value] : query.changes) + for (const auto & change : query.changes) { - auto [it, inserted] = result_changes_map.emplace(name, value); + auto [it, inserted] = result_changes_map.emplace(change.getName(), change.getFieldValue()); if (!inserted) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Value with key `{}` is used twice in the SET query", - name, query.collection_name); + change.getName(), query.collection_name); } } - for (const auto & [name, value] : create_query.changes) - result_changes_map.emplace(name, value); + for (const auto & change : create_query.changes) + result_changes_map.emplace(change.getName(), change.getFieldValue()); for (const auto & delete_key : query.delete_keys) { @@ -297,8 +297,8 @@ private: collection_name, query.changes); std::set keys; - for (const auto & [name, _] : query.changes) - keys.insert(name); + for (const auto & change : query.changes) + keys.insert(change.getName()); return NamedCollection::create( *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); @@ -422,8 +422,8 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name); auto collection_lock = collection->lock(); - for (const auto & [name, value] : query.changes) - collection->setOrUpdate(name, convertFieldToString(value)); + for (const auto & change : query.changes) + collection->setOrUpdate(change.getName(), convertFieldToString(change.getFieldValue())); for (const auto & key : query.delete_keys) collection->remove(key); From 8fde6067684fb2af626feb4b2be03f5c15d6f7f4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 6 Jan 2023 16:10:00 +0100 Subject: [PATCH 022/116] Resolve some review comments --- src/Disks/DiskSelector.cpp | 6 +++--- .../AzureBlobStorage/registerDiskAzureBlobStorage.cpp | 2 +- src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp | 8 ++++---- src/Disks/ObjectStorages/DiskObjectStorageCommon.h | 2 +- src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp | 2 +- src/Disks/ObjectStorages/S3/registerDiskS3.cpp | 2 +- src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/Context.h | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 10 +++++----- 9 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index d7cb1d928b7..7585d3225e0 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -105,20 +105,20 @@ DiskSelectorPtr DiskSelector::updateFromConfig( else writeString("Disks ", warning); - int index = 0; + int num_disks_removed_from_config = 0; for (const auto & [name, disk] : old_disks_minus_new_disks) { /// Custom disks are not present in config. if (disk->isCustomDisk()) continue; - if (index++ > 0) + if (num_disks_removed_from_config++ > 0) writeString(", ", warning); writeBackQuotedString(name, warning); } - if (index > 0) + if (num_disks_removed_from_config > 0) { writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); LOG_WARNING(&Poco::Logger::get("DiskSelector"), fmt::runtime(warning.str())); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 6bf04e2bc1d..562b2b2fec0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -24,7 +24,7 @@ void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access ContextPtr context, const DisksMap & /*map*/) { - auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); ObjectStoragePtr azure_object_storage = std::make_unique( name, diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp index d9c59b296f4..5ac6128c3c0 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp @@ -11,19 +11,19 @@ static String getDiskMetadataPath( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const String & root_disks_path) + ContextPtr context) { - return config.getString(config_prefix + ".metadata_path", fs::path(root_disks_path) / "disks" / name / ""); + return config.getString(config_prefix + ".metadata_path", fs::path(context->getPath()) / "disks" / name / ""); } std::pair prepareForLocalMetadata( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const String & root_disks_path) + ContextPtr context) { /// where the metadata files are stored locally - auto metadata_path = getDiskMetadataPath(name, config, config_prefix, root_disks_path); + auto metadata_path = getDiskMetadataPath(name, config, config_prefix, context); fs::create_directories(metadata_path); auto metadata_disk = std::make_shared(name + "-metadata", metadata_path, 0); return std::make_pair(metadata_path, metadata_disk); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageCommon.h b/src/Disks/ObjectStorages/DiskObjectStorageCommon.h index aa70c9f7a47..0bdbe0dfd36 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageCommon.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageCommon.h @@ -16,7 +16,7 @@ std::pair prepareForLocalMetadata( const String & name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - const String & root_disks_path); + ContextPtr context); bool isFileWithPersistentCache(const String & path); diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index 502787d1371..693b966caf2 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -41,7 +41,7 @@ void registerDiskHDFS(DiskFactory & factory, bool global_skip_access_check) /// FIXME Cache currently unsupported :( ObjectStoragePtr hdfs_storage = std::make_unique(uri, std::move(settings), config); - auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); + auto [_, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); auto metadata_storage = std::make_shared(metadata_disk, uri); uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16); diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 6351013b780..1c192a0d89c 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -131,7 +131,7 @@ void registerDiskS3(DiskFactory & factory, bool global_skip_access_check) else { s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); - auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context->getPath()); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); metadata_storage = std::make_shared(metadata_disk, uri.key); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6f4777f371c..3510634bcd3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2920,19 +2920,19 @@ StoragePolicyPtr Context::getStoragePolicy(const String & name) const return policy_selector->get(name); } -StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & name) const +StoragePolicyPtr Context::getStoragePolicyFromDisk(const String & disk_name) const { std::lock_guard lock(shared->storage_policies_mutex); - const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + name; + const std::string storage_policy_name = StoragePolicySelector::TMP_STORAGE_POLICY_PREFIX + disk_name; auto storage_policy_selector = getStoragePolicySelector(lock); StoragePolicyPtr storage_policy = storage_policy_selector->tryGet(storage_policy_name); if (!storage_policy) { auto disk_selector = getDiskSelector(lock); - auto disk = disk_selector->get(name); - auto volume = std::make_shared("_volume_" + name, disk); + auto disk = disk_selector->get(disk_name); + auto volume = std::make_shared("_volume_" + disk_name, disk); static const auto move_factor_for_single_disk_volume = 0.0; storage_policy = std::make_shared(storage_policy_name, Volumes{volume}, move_factor_for_single_disk_volume); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3077c58aebb..c6d22955563 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -962,7 +962,7 @@ public: /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; - StoragePolicyPtr getStoragePolicyFromDisk(const String & name) const; + StoragePolicyPtr getStoragePolicyFromDisk(const String & disk_name) const; /// Get the server uptime in seconds. double getUptimeSeconds() const; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index f447ef87d00..0f85fd2ad9c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -90,10 +90,10 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(size_t min_marks_to_read, size_t auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - // if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks - // need_marks = marks_in_part; - // else /// Get whole part to read if it is small enough. - need_marks = std::min(marks_in_part, min_marks_to_read); + if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks + need_marks = marks_in_part; + else /// Get whole part to read if it is small enough. + need_marks = std::min(marks_in_part, min_marks_to_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && @@ -203,7 +203,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo(const RangesInDataParts & const auto & part = parts[i]; bool part_on_remote_disk = part.data_part->isStoredOnRemoteDisk(); is_part_on_remote_disk[i] = part_on_remote_disk; - // do_not_steal_tasks |= part_on_remote_disk; + do_not_steal_tasks |= part_on_remote_disk; /// Read marks for every data part. size_t sum_marks = 0; From e4dcaf52666cf37ee0c94f263e40e2b12e331c7e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 6 Jan 2023 17:15:01 +0100 Subject: [PATCH 023/116] Update test --- .../queries/0_stateless/02454_create_table_with_custom_disk.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql index 20dde68336e..6cb1c0774aa 100644 --- a/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql +++ b/tests/queries/0_stateless/02454_create_table_with_custom_disk.sql @@ -1,4 +1,4 @@ --- Tags: no-s3-storage +-- Tags: no-s3-storage, no-replicated-database DROP TABLE IF EXISTS test; From 5bf4704e7a8ac016747af7d50dbe2946d37d0cbb Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 16 Jan 2023 21:01:31 +0000 Subject: [PATCH 024/116] Support FixedSizeBinary type in Parquet/Arrow --- docs/en/interfaces/formats.md | 92 ++++++------ docs/en/operations/settings/settings.md | 12 ++ src/Core/Settings.h | 2 + src/Core/SettingsChangesHistory.h | 4 +- src/Formats/FormatFactory.cpp | 2 + src/Formats/FormatSettings.h | 2 + .../Formats/Impl/ArrowBlockOutputFormat.cpp | 8 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 22 ++- .../Formats/Impl/CHColumnToArrowColumn.cpp | 135 ++++++++++++------ .../Formats/Impl/CHColumnToArrowColumn.h | 5 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 7 +- ...02534_parquet_fixed_binary_array.reference | 41 ++++++ .../02534_parquet_fixed_binary_array.sh | 31 ++++ 13 files changed, 270 insertions(+), 93 deletions(-) create mode 100644 tests/queries/0_stateless/02534_parquet_fixed_binary_array.reference create mode 100755 tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d384ed639eb..7035c5625ee 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1907,28 +1907,28 @@ Setting `format_avro_schema_registry_url` needs to be configured in `users.xml` The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) | -|------------------------------|-----------------------------------------------------------|----------------------------| -| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | -| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | -| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | -| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | -| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | -| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | -| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | -| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | -| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | -| `FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT` | -| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `DOUBLE` | -| `DATE` | [Date32](/docs/en/sql-reference/data-types/date.md) | `DATE` | -| `TIME (ms)` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | -| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `TIMESTAMP` | -| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | -| — | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `BINARY` | -| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | -| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | -| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | -| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | +| Parquet data type (`INSERT`) | ClickHouse data type | Parquet data type (`SELECT`) | +|-----------------------------------------------|-----------------------------------------------------------------|------------------------------| +| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | +| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | +| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | +| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | +| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | +| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | +| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | +| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | +| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | +| `FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT` | +| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `DOUBLE` | +| `DATE` | [Date32](/docs/en/sql-reference/data-types/date.md) | `DATE` | +| `TIME (ms)` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | +| `TIMESTAMP`, `TIME (us, ns)` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `TIMESTAMP` | +| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | +| `STRING`, `BINARY`, `FIXED_LENGTH_BYTE_ARRAY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `FIXED_LENGTH_BYTE_ARRAY` | +| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | +| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | +| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | +| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -1960,6 +1960,7 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`. - [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`. - [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`. +- [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`. ## Arrow {#data-format-arrow} @@ -1971,29 +1972,29 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t The table below shows supported data types and how they match ClickHouse [data types](/docs/en/sql-reference/data-types/index.md) in `INSERT` and `SELECT` queries. -| Arrow data type (`INSERT`) | ClickHouse data type | Arrow data type (`SELECT`) | -|---------------------------------|-----------------------------------------------------------|----------------------------| -| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | -| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | -| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | -| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | -| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | -| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | -| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | -| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | -| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | -| `FLOAT`, `HALF_FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` | -| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` | -| `DATE32` | [Date32](/docs/en/sql-reference/data-types/date32.md) | `UINT16` | -| `DATE64` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | -| `TIMESTAMP`, `TIME32`, `TIME64` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `UINT32` | -| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | -| `STRING`, `BINARY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `BINARY` | -| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | -| `DECIMAL256` | [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL256` | -| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | -| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | -| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | +| Arrow data type (`INSERT`) | ClickHouse data type | Arrow data type (`SELECT`) | +|-----------------------------------------|-----------------------------------------------------------------|----------------------------| +| `BOOL` | [Bool](/docs/en/sql-reference/data-types/boolean.md) | `BOOL` | +| `UINT8`, `BOOL` | [UInt8](/docs/en/sql-reference/data-types/int-uint.md) | `UINT8` | +| `INT8` | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | `INT8` | +| `UINT16` | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | `UINT16` | +| `INT16` | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | `INT16` | +| `UINT32` | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | `UINT32` | +| `INT32` | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | `INT32` | +| `UINT64` | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | `UINT64` | +| `INT64` | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | `INT64` | +| `FLOAT`, `HALF_FLOAT` | [Float32](/docs/en/sql-reference/data-types/float.md) | `FLOAT32` | +| `DOUBLE` | [Float64](/docs/en/sql-reference/data-types/float.md) | `FLOAT64` | +| `DATE32` | [Date32](/docs/en/sql-reference/data-types/date32.md) | `UINT16` | +| `DATE64` | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | `UINT32` | +| `TIMESTAMP`, `TIME32`, `TIME64` | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | `UINT32` | +| `STRING`, `BINARY` | [String](/docs/en/sql-reference/data-types/string.md) | `BINARY` | +| `STRING`, `BINARY`, `FIXED_SIZE_BINARY` | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | `FIXED_SIZE_BINARY` | +| `DECIMAL` | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL` | +| `DECIMAL256` | [Decimal256](/docs/en/sql-reference/data-types/decimal.md) | `DECIMAL256` | +| `LIST` | [Array](/docs/en/sql-reference/data-types/array.md) | `LIST` | +| `STRUCT` | [Tuple](/docs/en/sql-reference/data-types/tuple.md) | `STRUCT` | +| `MAP` | [Map](/docs/en/sql-reference/data-types/map.md) | `MAP` | Arrays can be nested and can have a value of the `Nullable` type as an argument. `Tuple` and `Map` types also can be nested. @@ -2027,6 +2028,7 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam - [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`. - [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`. - [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`. +- [output_format_arrow_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings.md/#output_format_arrow_fixed_string_as_fixed_byte_array) - use Arrow FIXED_SIZE_BINARY type instead of Binary/String for FixedString columns. Default value - `true`. ## ArrowStream {#data-format-arrow-stream} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 53cc9f1e349..7ee18863746 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4625,6 +4625,12 @@ Use Arrow String type instead of Binary for String columns. Disabled by default. +### output_format_arrow_fixed_string_as_fixed_byte_array (#output_format_arrow_fixed_string_as_fixed_byte_array) + +Use Arrow FIXED_SIZE_BINARY type instead of Binary/String for FixedString columns. + +Enabled by default. + ## ORC format settings {#orc-format-settings} ### input_format_orc_import_nested {#input_format_orc_import_nested} @@ -4711,6 +4717,12 @@ Use Parquet String type instead of Binary for String columns. Disabled by default. +### output_format_parquet_fixed_string_as_fixed_byte_array (#output_format_parquet_fixed_string_as_fixed_byte_array) + +Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. + +Enabled by default. + ## Hive format settings {#hive-format-settings} ### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f58bd7ebafb..5f034460315 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -818,6 +818,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(String, output_format_pretty_grid_charset, "UTF-8", "Charset for printing grid borders. Available charsets: ASCII, UTF-8 (default one).", 0) \ M(UInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ M(Bool, output_format_parquet_string_as_string, false, "Use Parquet String type instead of Binary for String columns.", 0) \ + M(Bool, output_format_parquet_fixed_string_as_fixed_byte_array, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary for FixedString columns.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ @@ -859,6 +860,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ M(Bool, output_format_arrow_string_as_string, false, "Use Arrow String type instead of Binary for String columns", 0) \ + M(Bool, output_format_arrow_fixed_string_as_fixed_byte_array, true, "Use Arrow FIXED_SIZE_BINARY type instead of Binary for FixedString columns.", 0) \ \ M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 534fcd42037..16e1428cea6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -81,7 +81,9 @@ namespace SettingsChangesHistory static std::map settings_changes_history = { {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}}}, + {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, + {"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, + {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}}}, {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ed2464f98e8..c73193260d8 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -113,6 +113,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; + format_settings.parquet.output_fixed_string_as_fixed_byte_array = settings.output_format_parquet_fixed_string_as_fixed_byte_array; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; @@ -151,6 +152,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference; format_settings.arrow.case_insensitive_column_matching = settings.input_format_arrow_case_insensitive_column_matching; format_settings.arrow.output_string_as_string = settings.output_format_arrow_string_as_string; + format_settings.arrow.output_fixed_string_as_fixed_byte_array = settings.output_format_arrow_fixed_string_as_fixed_byte_array; format_settings.orc.import_nested = settings.input_format_orc_import_nested; format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 9d8680a009d..fb7fc98ac25 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -94,6 +94,7 @@ struct FormatSettings bool skip_columns_with_unsupported_types_in_schema_inference = false; bool case_insensitive_column_matching = false; bool output_string_as_string = false; + bool output_fixed_string_as_fixed_byte_array = true; } arrow; struct @@ -180,6 +181,7 @@ struct FormatSettings bool case_insensitive_column_matching = false; std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; + bool output_fixed_string_as_fixed_byte_array = true; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 874709cc8d7..bf0e2448082 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -32,8 +32,12 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) if (!ch_column_to_arrow_column) { const Block & header = getPort(PortKind::Main).getHeader(); - ch_column_to_arrow_column - = std::make_unique(header, "Arrow", format_settings.arrow.low_cardinality_as_dictionary, format_settings.arrow.output_string_as_string); + ch_column_to_arrow_column = std::make_unique( + header, + "Arrow", + format_settings.arrow.low_cardinality_as_dictionary, + format_settings.arrow.output_string_as_string, + format_settings.arrow.output_fixed_string_as_fixed_byte_array); } ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 52c868d4e0c..6509290c65d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -143,6 +144,24 @@ static ColumnWithTypeAndName readColumnWithStringData(std::shared_ptr & arrow_column, const String & column_name) +{ + const auto * fixed_type = assert_cast(arrow_column->type().get()); + size_t fixed_len = fixed_type->byte_width(); + auto internal_type = std::make_shared(fixed_len); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + column_chars_t.reserve(arrow_column->length() * fixed_len); + + for (int chunk_i = 0, num_chunks = arrow_column->num_chunks(); chunk_i < num_chunks; ++chunk_i) + { + arrow::FixedSizeBinaryArray & chunk = dynamic_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr buffer = chunk.values(); + column_chars_t.insert_assume_reserved(buffer->data(), buffer->data() + buffer->size()); + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + static ColumnWithTypeAndName readColumnWithBooleanData(std::shared_ptr & arrow_column, const String & column_name) { auto internal_type = DataTypeFactory::instance().get("Bool"); @@ -535,8 +554,9 @@ static ColumnWithTypeAndName readColumnFromArrowColumn( { case arrow::Type::STRING: case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: return readColumnWithStringData(arrow_column, column_name); + case arrow::Type::FIXED_SIZE_BINARY: + return readColumnWithFixedStringData(arrow_column, column_name); case arrow::Type::LARGE_BINARY: case arrow::Type::LARGE_STRING: return readColumnWithStringData(arrow_column, column_name); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 8319ef65e17..cbe2a458221 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -88,6 +89,19 @@ namespace DB throw Exception{fmt::format("Error with a {} column \"{}\": {}.", format_name, column_name, status.ToString()), ErrorCodes::UNKNOWN_EXCEPTION}; } + /// Invert values since Arrow interprets 1 as a non-null value, while CH as a null + static PaddedPODArray revertNullByteMap(const PaddedPODArray * null_bytemap, size_t start, size_t end) + { + PaddedPODArray res; + if (!null_bytemap) + return res; + + res.reserve(end - start); + for (size_t i = start; i < end; ++i) + res.emplace_back(!(*null_bytemap)[i]); + return res; + } + template static void fillArrowArrayWithNumericColumnData( ColumnPtr write_column, @@ -101,17 +115,8 @@ namespace DB ArrowBuilderType & builder = assert_cast(*array_builder); arrow::Status status; - const UInt8 * arrow_null_bytemap_raw_ptr = nullptr; - PaddedPODArray arrow_null_bytemap; - if (null_bytemap) - { - /// Invert values since Arrow interprets 1 as a non-null value, while CH as a null - arrow_null_bytemap.reserve(end - start); - for (size_t i = start; i < end; ++i) - arrow_null_bytemap.template emplace_back(!(*null_bytemap)[i]); - - arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data(); - } + PaddedPODArray arrow_null_bytemap = revertNullByteMap(null_bytemap, start, end); + const UInt8 * arrow_null_bytemap_raw_ptr = arrow_null_bytemap.empty() ? nullptr : arrow_null_bytemap.data(); if constexpr (std::is_same_v) status = builder.AppendValues( @@ -135,17 +140,8 @@ namespace DB arrow::BooleanBuilder & builder = assert_cast(*array_builder); arrow::Status status; - const UInt8 * arrow_null_bytemap_raw_ptr = nullptr; - PaddedPODArray arrow_null_bytemap; - if (null_bytemap) - { - /// Invert values since Arrow interprets 1 as a non-null value, while CH as a null - arrow_null_bytemap.reserve(end - start); - for (size_t i = start; i < end; ++i) - arrow_null_bytemap.template emplace_back(!(*null_bytemap)[i]); - - arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data(); - } + PaddedPODArray arrow_null_bytemap = revertNullByteMap(null_bytemap, start, end); + const UInt8 * arrow_null_bytemap_raw_ptr = arrow_null_bytemap.empty() ? nullptr : arrow_null_bytemap.data(); status = builder.AppendValues(reinterpret_cast(internal_data.data() + start), end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); checkStatus(status, write_column->getName(), format_name); @@ -197,6 +193,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values); template @@ -210,6 +207,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values) { const auto * column_array = assert_cast(column.get()); @@ -226,7 +224,7 @@ namespace DB /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, dictionary_values); + fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } } @@ -240,6 +238,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values) { const auto * column_tuple = assert_cast(column.get()); @@ -259,6 +258,7 @@ namespace DB format_name, start, end, output_string_as_string, + output_fixed_string_as_fixed_byte_array, dictionary_values); } @@ -311,6 +311,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values) { const auto * column_lc = assert_cast(column.get()); @@ -329,7 +330,7 @@ namespace DB auto dict_column = column_lc->getDictionary().getNestedNotNullableColumn(); const auto & dict_type = removeNullable(assert_cast(column_type.get())->getDictionaryType()); - fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, is_nullable, dict_column->size(), output_string_as_string, dictionary_values); + fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, is_nullable, dict_column->size(), output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); status = values_builder->Finish(&dict_values); checkStatus(status, column->getName(), format_name); } @@ -366,6 +367,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values) { auto value_type = assert_cast(array_builder->type().get())->value_type(); @@ -373,7 +375,7 @@ namespace DB #define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \ if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \ { \ - fillArrowArrayWithLowCardinalityColumnDataImpl(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); \ + fillArrowArrayWithLowCardinalityColumnDataImpl(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); \ return; \ } @@ -411,6 +413,28 @@ namespace DB } } + static void fillArrowArrayWithFixedStringColumnData( + ColumnPtr write_column, + const PaddedPODArray * null_bytemap, + const String & format_name, + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) + { + const auto & internal_column = assert_cast(*write_column); + const auto & internal_data = internal_column.getChars(); + size_t fixed_length = internal_column.getN(); + arrow::FixedSizeBinaryBuilder & builder = assert_cast(*array_builder); + arrow::Status status; + + PaddedPODArray arrow_null_bytemap = revertNullByteMap(null_bytemap, start, end); + const UInt8 * arrow_null_bytemap_raw_ptr = arrow_null_bytemap.empty() ? nullptr : arrow_null_bytemap.data(); + + const uint8_t * data_start = reinterpret_cast(internal_data.data() + start * fixed_length); + status = builder.AppendValues(data_start, end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); + checkStatus(status, write_column->getName(), format_name); + } + static void fillArrowArrayWithDateColumnData( ColumnPtr write_column, const PaddedPODArray * null_bytemap, @@ -488,6 +512,7 @@ namespace DB size_t start, size_t end, bool output_string_as_string, + bool output_fixed_string_as_fixed_byte_array, std::unordered_map> & dictionary_values) { const String column_type_name = column_type->getFamilyName(); @@ -499,7 +524,7 @@ namespace DB DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr(); const PaddedPODArray & bytemap = assert_cast &>(*null_column).getData(); - fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } else if (isString(column_type)) { @@ -510,7 +535,9 @@ namespace DB } else if (isFixedString(column_type)) { - if (output_string_as_string) + if (output_fixed_string_as_fixed_byte_array) + fillArrowArrayWithFixedStringColumnData(column, null_bytemap, format_name, array_builder, start, end); + else if (output_string_as_string) fillArrowArrayWithStringColumnData(column, null_bytemap, format_name, array_builder, start, end); else fillArrowArrayWithStringColumnData(column, null_bytemap, format_name, array_builder, start, end); @@ -529,21 +556,21 @@ namespace DB } else if (isArray(column_type)) { - fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } else if (isTuple(column_type)) { - fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } else if (column_type->getTypeId() == TypeIndex::LowCardinality) { - fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } else if (isMap(column_type)) { ColumnPtr column_array = assert_cast(column.get())->getNestedColumnPtr(); DataTypePtr array_type = assert_cast(column_type.get())->getNestedType(); - fillArrowArrayWithArrayColumnData(column_name, column_array, array_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, dictionary_values); + fillArrowArrayWithArrayColumnData(column_name, column_array, array_type, null_bytemap, array_builder, format_name, start, end, output_string_as_string, output_fixed_string_as_fixed_byte_array, dictionary_values); } else if (isDecimal(column_type)) { @@ -659,13 +686,13 @@ namespace DB } static std::shared_ptr getArrowType( - DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool output_string_as_string, bool * out_is_column_nullable) + DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool output_string_as_string, bool output_fixed_string_as_fixed_byte_array, bool * out_is_column_nullable) { if (column_type->isNullable()) { DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); ColumnPtr nested_column = assert_cast(column.get())->getNestedColumnPtr(); - auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, out_is_column_nullable); + auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable); *out_is_column_nullable = true; return arrow_type; } @@ -699,7 +726,7 @@ namespace DB { auto nested_type = assert_cast(column_type.get())->getNestedType(); auto nested_column = assert_cast(column.get())->getDataPtr(); - auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, out_is_column_nullable); + auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable); return arrow::list(nested_arrow_type); } @@ -712,7 +739,7 @@ namespace DB std::vector> nested_fields; for (size_t i = 0; i != nested_types.size(); ++i) { - auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), nested_names[i], format_name, output_string_as_string, out_is_column_nullable); + auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), nested_names[i], format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable); nested_fields.push_back(std::make_shared(nested_names[i], nested_arrow_type, *out_is_column_nullable)); } return arrow::struct_(nested_fields); @@ -726,7 +753,7 @@ namespace DB const auto & indexes_column = lc_column->getIndexesPtr(); return arrow::dictionary( getArrowTypeForLowCardinalityIndexes(indexes_column), - getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, out_is_column_nullable)); + getArrowType(nested_type, nested_column, column_name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable)); } if (isMap(column_type)) @@ -737,8 +764,8 @@ namespace DB const auto & columns = assert_cast(column.get())->getNestedData().getColumns(); return arrow::map( - getArrowType(key_type, columns[0], column_name, format_name, output_string_as_string, out_is_column_nullable), - getArrowType(val_type, columns[1], column_name, format_name, output_string_as_string, out_is_column_nullable)); + getArrowType(key_type, columns[0], column_name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable), + getArrowType(val_type, columns[1], column_name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, out_is_column_nullable)); } if (isDateTime64(column_type)) @@ -747,6 +774,12 @@ namespace DB return arrow::timestamp(getArrowTimeUnit(datetime64_type), datetime64_type->getTimeZone().getTimeZone()); } + if (isFixedString(column_type) && output_fixed_string_as_fixed_byte_array) + { + size_t fixed_length = assert_cast(column_type.get())->getN(); + return arrow::fixed_size_binary(static_cast(fixed_length)); + } + if (isStringOrFixedString(column_type) && output_string_as_string) return arrow::utf8(); @@ -768,8 +801,16 @@ namespace DB column_type->getName(), column_name, format_name); } - CHColumnToArrowColumn::CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_, bool output_string_as_string_) - : format_name(format_name_), low_cardinality_as_dictionary(low_cardinality_as_dictionary_), output_string_as_string(output_string_as_string_) + CHColumnToArrowColumn::CHColumnToArrowColumn( + const Block & header, + const std::string & format_name_, + bool low_cardinality_as_dictionary_, + bool output_string_as_string_, + bool output_fixed_string_as_fixed_byte_array_) + : format_name(format_name_) + , low_cardinality_as_dictionary(low_cardinality_as_dictionary_) + , output_string_as_string(output_string_as_string_) + , output_fixed_string_as_fixed_byte_array(output_fixed_string_as_fixed_byte_array_) { arrow_fields.reserve(header.columns()); header_columns.reserve(header.columns()); @@ -804,7 +845,7 @@ namespace DB if (!is_arrow_fields_initialized) { bool is_column_nullable = false; - auto arrow_type = getArrowType(header_column.type, column, header_column.name, format_name, output_string_as_string, &is_column_nullable); + auto arrow_type = getArrowType(header_column.type, column, header_column.name, format_name, output_string_as_string, output_fixed_string_as_fixed_byte_array, &is_column_nullable); arrow_fields.emplace_back(std::make_shared(header_column.name, arrow_type, is_column_nullable)); } @@ -814,7 +855,17 @@ namespace DB checkStatus(status, column->getName(), format_name); fillArrowArray( - header_column.name, column, header_column.type, nullptr, array_builder.get(), format_name, 0, column->size(), output_string_as_string, dictionary_values); + header_column.name, + column, + header_column.type, + nullptr, + array_builder.get(), + format_name, + 0, + column->size(), + output_string_as_string, + output_fixed_string_as_fixed_byte_array, + dictionary_values); std::shared_ptr arrow_array; status = array_builder->Finish(&arrow_array); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index 1db035e55a6..62fdcaa8086 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -14,7 +14,7 @@ namespace DB class CHColumnToArrowColumn { public: - CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_, bool output_string_as_string_); + CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_, bool output_string_as_string_, bool output_fixed_string_as_fixed_byte_array_); void chChunkToArrowTable(std::shared_ptr & res, const Chunk & chunk, size_t columns_num); @@ -36,6 +36,9 @@ private: /// Output columns with String data type as Arrow::String type. /// By default Arrow::Binary is used. bool output_string_as_string = false; + + /// Output columns with String data type as Arrow::FixedByteArray type. + bool output_fixed_string_as_fixed_byte_array = true; }; } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index e99b308b87b..7ef3eaa6b57 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -29,7 +29,12 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) if (!ch_column_to_arrow_column) { const Block & header = getPort(PortKind::Main).getHeader(); - ch_column_to_arrow_column = std::make_unique(header, "Parquet", false, format_settings.parquet.output_string_as_string); + ch_column_to_arrow_column = std::make_unique( + header, + "Parquet", + false, + format_settings.parquet.output_string_as_string, + format_settings.parquet.output_fixed_string_as_fixed_byte_array); } ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); diff --git a/tests/queries/0_stateless/02534_parquet_fixed_binary_array.reference b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.reference new file mode 100644 index 00000000000..608d630c9ca --- /dev/null +++ b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.reference @@ -0,0 +1,41 @@ +Parquet +fixed Nullable(String) +0 +1 +2 +3 +4 +fixed Nullable(FixedString(1)) +0 +1 +2 +3 +4 +\N +1 +\N +3 +\N +5 +\N +7 +\N +9 +Arrow +fixed Nullable(String) +0 +1 +2 +3 +4 +fixed Nullable(FixedString(1)) +0 +1 +2 +3 +4 +\N +1 +\N +3 +\N diff --git a/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh new file mode 100755 index 00000000000..e3d7efdf8fe --- /dev/null +++ b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "Parquet" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(10) format Parquet" --output_format_parquet_fixed_string_as_fixed_byte_array=0 | $CLICKHOUSE_LOCAL --input-format=Parquet --table test -q "desc test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Parquet" --output_format_parquet_fixed_string_as_fixed_byte_array=0 | $CLICKHOUSE_LOCAL --input-format=Parquet --table test -q "select * from test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(10) format Parquet" --output_format_parquet_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Parquet --table test -q "desc test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Parquet" --output_format_parquet_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Parquet --table test -q "select * from test" + +$CLICKHOUSE_LOCAL -q "select number % 2 ? toFixedString(toString(number % 10), 1) : NULL as fixed from numbers(10) format Parquet" --output_format_parquet_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Parquet --table test -q "select * from test" + +echo "Arrow" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Arrow" --output_format_arrow_fixed_string_as_fixed_byte_array=0 | $CLICKHOUSE_LOCAL --input-format=Arrow --table test -q "desc test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Arrow" --output_format_arrow_fixed_string_as_fixed_byte_array=0 | $CLICKHOUSE_LOCAL --input-format=Arrow --table test -q "select * from test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Arrow" --output_format_arrow_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Arrow --table test -q "desc test" + +$CLICKHOUSE_LOCAL -q "select toFixedString(toString(number % 10), 1) as fixed from numbers(5) format Arrow" --output_format_arrow_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Arrow --table test -q "select * from test" + +$CLICKHOUSE_LOCAL -q "select number % 2 ? toFixedString(toString(number % 10), 1) : NULL as fixed from numbers(5) format Arrow" --output_format_arrow_fixed_string_as_fixed_byte_array=1 | $CLICKHOUSE_LOCAL --input-format=Arrow --table test -q "select * from test" + From 174d1be960cd512d94ee2809a0b41cf397c26f3d Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 16 Jan 2023 21:04:47 +0000 Subject: [PATCH 025/116] Add tags to test --- tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh index e3d7efdf8fe..70b3fc83c77 100755 --- a/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh +++ b/tests/queries/0_stateless/02534_parquet_fixed_binary_array.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From b29510b07f7e04e0ee88b9b7fca4babe86984392 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 19 Jan 2023 17:28:34 +0000 Subject: [PATCH 026/116] Update tests --- tests/queries/0_stateless/00900_long_parquet.reference | 8 ++++---- tests/queries/0_stateless/01273_arrow.reference | 8 ++++---- tests/queries/0_stateless/01273_arrow_stream.reference | 8 ++++---- .../02149_schema_inference_formats_with_schema.reference | 6 +++--- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/00900_long_parquet.reference b/tests/queries/0_stateless/00900_long_parquet.reference index bbdad7243bd..fd1fa800386 100644 --- a/tests/queries/0_stateless/00900_long_parquet.reference +++ b/tests/queries/0_stateless/00900_long_parquet.reference @@ -43,7 +43,7 @@ converted: 127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000 diff: dest: -79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2004-05-06 07:08:09.012000000 +79 81 82 83 84 85 86 87 88 89 str01 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2004-05-06 07:08:09.012000000 80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 2006-08-09 10:11:12.345000000 min: -128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 2003-02-03 04:05:06.789000000 @@ -51,10 +51,10 @@ min: 79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 2004-05-06 07:08:09.012000000 127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 2004-02-03 04:05:06.789000000 max: --128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000 --108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000 +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000 80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 2006-08-09 10:11:12.345000000 -127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789000000 dest from null: -128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789000000 -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789000000 diff --git a/tests/queries/0_stateless/01273_arrow.reference b/tests/queries/0_stateless/01273_arrow.reference index b4d011754dc..297c7b31f63 100644 --- a/tests/queries/0_stateless/01273_arrow.reference +++ b/tests/queries/0_stateless/01273_arrow.reference @@ -41,7 +41,7 @@ converted: 127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 2004-02-03 04:05:06.789012 diff: dest: -79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2005-02-03 04:05:06.789012 +79 81 82 83 84 85 86 87 88 89 str01 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 2005-02-03 04:05:06.789012 80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 2007-02-03 04:05:06.789012 min: -128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 2003-02-03 04:05:06.789012 @@ -49,10 +49,10 @@ min: 79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 2005-02-03 04:05:06.789012 127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 2004-02-03 04:05:06.789012 max: --128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789012 --108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789012 +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1 2003-04-05 00:00:00 2003-02-03 04:05:06 2003-02-03 04:05:06.789012 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring 2001-02-03 00:00:00 2002-02-03 04:05:06 2002-02-03 04:05:06.789012 80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 2007-02-03 04:05:06.789012 -127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789012 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2 2004-06-07 00:00:00 2004-02-03 04:05:06 2004-02-03 04:05:06.789012 dest from null: -128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 2003-02-03 04:05:06.789 -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 2002-02-03 04:05:06.789 diff --git a/tests/queries/0_stateless/01273_arrow_stream.reference b/tests/queries/0_stateless/01273_arrow_stream.reference index 89eca82f8ef..7da58c33feb 100644 --- a/tests/queries/0_stateless/01273_arrow_stream.reference +++ b/tests/queries/0_stateless/01273_arrow_stream.reference @@ -41,7 +41,7 @@ converted: 127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 diff: dest: -79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 +79 81 82 83 84 85 86 87 88 89 str01 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 00:00:00 80 81 82 83 84 85 86 87 88 89 str02 fstr2\0\0\0\0\0\0\0\0\0\0 2005-03-04 2006-08-09 10:11:12 min: -128 0 0 0 0 0 0 0 -1 -1 string-1\0\0\0\0\0\0\0 fixedstring-1\0\0 2003-04-05 2003-02-03 @@ -49,10 +49,10 @@ min: 79 81 82 83 84 85 86 87 88 89 str01\0\0\0\0\0\0\0\0\0\0 fstr1\0\0\0\0\0\0\0\0\0\0 2003-03-04 2004-05-06 127 -1 -1 -1 -1 -1 -1 -1 -1 -1 string-2\0\0\0\0\0\0\0 fixedstring-2\0\0 2004-06-07 2004-02-03 max: --128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1\0\0 2003-04-05 00:00:00 2003-02-03 04:05:06 --108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring\0\0\0\0 2001-02-03 00:00:00 2002-02-03 04:05:06 +-128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1 -1 string-1 fixedstring-1 2003-04-05 00:00:00 2003-02-03 04:05:06 +-108 108 -1016 1116 -1032 1132 -1064 1164 -1 -1 string-0 fixedstring 2001-02-03 00:00:00 2002-02-03 04:05:06 80 81 82 83 84 85 86 87 88 89 str02 fstr2 2005-03-04 05:06:07 2006-08-09 10:11:12 -127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2\0\0 2004-06-07 00:00:00 2004-02-03 04:05:06 +127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1 -1 string-2 fixedstring-2 2004-06-07 00:00:00 2004-02-03 04:05:06 dest from null: -128 0 -32768 0 -2147483648 0 -9223372036854775808 0 -1.032 -1.064 string-1 fixedstring-1\0\0 2003-04-05 2003-02-03 04:05:06 -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 diff --git a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference index b881fce1539..f3bfd9f1048 100644 --- a/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference +++ b/tests/queries/0_stateless/02149_schema_inference_formats_with_schema.reference @@ -20,7 +20,7 @@ date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 str Nullable(String) -fixed_string Nullable(String) +fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) @@ -54,7 +54,7 @@ date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 str Nullable(String) -fixed_string Nullable(String) +fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) @@ -88,7 +88,7 @@ date32 Nullable(Date32) 0 1970-01-01 1 1970-01-02 str Nullable(String) -fixed_string Nullable(String) +fixed_string Nullable(FixedString(3)) Str: 0 100 Str: 1 200 array Array(Nullable(UInt64)) From c1de756d8aa413ebbe2ab7272584822f90a897f2 Mon Sep 17 00:00:00 2001 From: Denys Golotiuk <85636145+mrcrypster@users.noreply.github.com> Date: Tue, 24 Jan 2023 17:50:29 +0200 Subject: [PATCH 027/116] DELETE + WHERE in TTL ClickHouse actually supports only `DELETE` action with `WHERE` clause of a `TTL` statement --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 80427f301f3..08edb79c2f9 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -611,7 +611,10 @@ Type of TTL rule may follow each TTL expression. It affects an action which is t - `TO VOLUME 'bbb'` - move part to the disk `bbb`; - `GROUP BY` - aggregate expired rows. -With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves or recompression). +`DELETE` action can be used together with `WHERE` clause to delete only some of the expired rows based on a filtering condition: +``` sql +TTL time_column + INTERVAL 1 MONTH DELETE WHERE column = 'value' +``` `GROUP BY` expression must be a prefix of the table primary key. From 21e871a0a4c49fee1d297cbf76ce158805bbdfea Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 27 Jan 2023 18:47:22 +0000 Subject: [PATCH 028/116] integrate IO scheduler with buffers --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 11 ++- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- src/IO/IResourceManager.h | 2 +- src/IO/ISchedulerQueue.h | 36 ++++++- src/IO/ReadBufferFromS3.cpp | 12 ++- src/IO/ReadBufferFromS3.h | 2 + src/IO/ReadSettings.h | 4 + .../tests/gtest_resource_manager_static.cpp | 1 + src/IO/ResourceBudget.h | 55 +++++++++++ src/IO/ResourceGuard.h | 98 ++++++++++++++----- src/IO/ResourceLink.h | 39 ++++++++ src/IO/ResourceRequest.h | 41 ++++---- src/IO/WriteBufferFromS3.cpp | 18 +++- src/IO/WriteSettings.h | 4 + src/Storages/HDFS/ReadBufferFromHDFS.cpp | 8 ++ src/Storages/HDFS/WriteBufferFromHDFS.cpp | 10 +- 16 files changed, 285 insertions(+), 58 deletions(-) create mode 100644 src/IO/ResourceBudget.h create mode 100644 src/IO/ResourceLink.h diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 34a649b5469..a4242d9e458 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents @@ -40,10 +41,13 @@ WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() finalize(); } -void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, size_t num_tries) +void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, size_t num_tries, size_t cost) { - auto handle_exception = [&](const auto & e, size_t i) + auto handle_exception = [&, this](const auto & e, size_t i) { + if (cost) + write_settings.resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it + if (i == num_tries - 1) throw; @@ -54,6 +58,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { try { + ResourceGuard rlock(write_settings.resource_link, cost); func(); break; } @@ -87,7 +92,7 @@ void WriteBufferFromAzureBlobStorage::uploadBlock(const char * data, size_t size const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data), size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, DEFAULT_RETRY_NUM); + execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, DEFAULT_RETRY_NUM, size); tmp_buffer_write_offset = 0; LOG_TRACE(log, "Staged block (id: {}) of size {} (blob path: {}).", block_id, size, blob_path); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index c9f0134077f..27492809384 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -39,7 +39,7 @@ public: private: void finalizeImpl() override; - void execWithRetry(std::function func, size_t num_tries); + void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); void uploadBlock(const char * data, size_t size); Poco::Logger * log; diff --git a/src/IO/IResourceManager.h b/src/IO/IResourceManager.h index 019778595bf..f084a903cb1 100644 --- a/src/IO/IResourceManager.h +++ b/src/IO/IResourceManager.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/IO/ISchedulerQueue.h b/src/IO/ISchedulerQueue.h index 75ea05fbee3..9da6c5d33ac 100644 --- a/src/IO/ISchedulerQueue.h +++ b/src/IO/ISchedulerQueue.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include #include @@ -10,17 +12,49 @@ namespace DB /* * Queue for pending requests for specific resource, leaf of hierarchy. + * Note that every queue has budget associated with it. */ class ISchedulerQueue : public ISchedulerNode { public: - ISchedulerQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + explicit ISchedulerQueue(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) : ISchedulerNode(event_queue_, config, config_prefix) {} + // Wrapper for `enqueueRequest()` that should be used to account for available resource budget + void enqueueRequestUsingBudget(ResourceRequest * request) + { + request->cost = budget.ask(request->cost); + enqueueRequest(request); + } + + // Should be called to account for difference between real and estimated costs + void adjustBudget(ResourceCost estimated_cost, ResourceCost real_cost) + { + budget.adjust(estimated_cost, real_cost); + } + + // Adjust budget to account for extra consumtion of `cost` resource units + void consumeBudget(ResourceCost cost) + { + adjustBudget(0, cost); + } + + // Adjust budget to account for requested, but not consumed `cost` resource units + void accumulateBudget(ResourceCost cost) + { + adjustBudget(cost, 0); + } + /// Enqueue new request to be executed using underlying resource. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void enqueueRequest(ResourceRequest * request) = 0; + +private: + // Allows multiple consumers to synchronize with common "debit/credit" balance. + // 1) (positive) to avoid wasting of allocated but not used resource (e.g in case of a failure); + // 2) (negative) to account for overconsumption (e.g. if cost is not know in advance and esimation from below is applied). + ResourceBudget budget; }; } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 0cf77fbee4f..bc970b24dc9 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -1,10 +1,11 @@ #include "config.h" -#include "IO/S3Common.h" +#include #if USE_AWS_S3 #include #include +#include #include #include @@ -324,16 +325,23 @@ std::unique_ptr ReadBufferFromS3::initialize() if (read_settings.for_object_storage) ProfileEvents::increment(ProfileEvents::DiskS3GetObject); + // We do not know in advance how many bytes we are going to consume, to avoid blocking estimated it from below + constexpr ResourceCost estimated_cost = 1; + ResourceGuard rlock(read_settings.resource_link, estimated_cost); Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); + rlock.unlock(); if (outcome.IsSuccess()) { - read_result = outcome.GetResultWithOwnership(); + ResourceCost bytes_read = outcome.GetResult().GetContentLength(); + read_settings.resource_link.adjust(estimated_cost, bytes_read); size_t buffer_size = use_external_buffer ? 0 : read_settings.remote_fs_buffer_size; + read_result = outcome.GetResultWithOwnership(); return std::make_unique(read_result.GetBody(), buffer_size); } else { + read_settings.resource_link.accumulate(estimated_cost); const auto & error = outcome.GetError(); throw S3Exception(error.GetMessage(), error.GetErrorType()); } diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 17b13bf7d62..8ad1674297b 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -88,6 +88,8 @@ private: /// There is different seek policy for disk seek and for non-disk seek /// (non-disk seek is applied for seekable input formats: orc, arrow, parquet). bool restricted_seek; + + ResourceBudget budget; }; /// Creates separate ReadBufferFromS3 for sequence of ranges of particular object diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 1fa4aa637f5..c155c9a71a4 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -100,6 +101,9 @@ struct ReadSettings /// Bandwidth throttler to use during reading ThrottlerPtr remote_throttler; + // Resource to be used during reading + ResourceLink resource_link; + size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; size_t http_retry_max_backoff_ms = 1600; diff --git a/src/IO/Resource/tests/gtest_resource_manager_static.cpp b/src/IO/Resource/tests/gtest_resource_manager_static.cpp index 1a0af9198bc..cd426b1a6bc 100644 --- a/src/IO/Resource/tests/gtest_resource_manager_static.cpp +++ b/src/IO/Resource/tests/gtest_resource_manager_static.cpp @@ -37,6 +37,7 @@ TEST(IOResourceStaticResourceManager, Smoke) for (int i = 0; i < 10; i++) { ResourceGuard ga(ca->get("res1")); + ga.unlock(); ResourceGuard gb(cb->get("res1")); } } diff --git a/src/IO/ResourceBudget.h b/src/IO/ResourceBudget.h new file mode 100644 index 00000000000..4430ddbd78c --- /dev/null +++ b/src/IO/ResourceBudget.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/* + * Helper class to keep track of requested and consumed amount of resource. + * Useful if real amount of consumed resource can differ from requested amount of resource (e.g. in case of failures). + * Can be safely used from multiple threads. + * Usage example: + * ResourceBudget budget; + * while (!stop) { + * ResourceCost est_cost = myEstimateOfCostOrJustUseOne(); + * myAllocateResource(budget.ask(est_cost)); // Ask external system to allocate resource for you + * ResourceCost real_cost = mySynchronousConsumptionOfResource(); // Real consumption can differ from est_cost + * budget.adjust(est_cost, real_cost); // Adjust balance according to the actual cost, may affect the next iteration + * } + */ +class ResourceBudget +{ +public: + // Returns amount of resource to be requested according to current balance and estimated cost of new consumption + ResourceCost ask(ResourceCost estimated_cost) + { + ResourceCost budget = available.load(); + while (true) + { + // Valid resource request must have positive `cost`. Also takes consumption history into account. + ResourceCost cost = std::max(1ll, estimated_cost - budget); + + // Assume every request is satisfied (no resource request cancelation is possible now) + // So we requested additional `cost` units and are going to consume `estimated_cost` + ResourceCost new_budget = budget + cost - estimated_cost; + + // Try to commit this transaction + if (new_budget == budget || available.compare_exchange_strong(budget, new_budget)) + return cost; + } + } + + // Should be called to account for difference between real and estimated costs + // Optional. May be skipped if `real_cost` is known in advance (equals `estimated_cost`). + void adjust(ResourceCost estimated_cost, ResourceCost real_cost) + { + available.fetch_add(estimated_cost - real_cost); + } + +private: + std::atomic available = 0; // requested - consumed +}; + +} diff --git a/src/IO/ResourceGuard.h b/src/IO/ResourceGuard.h index c8b826d0932..dbd9c83fa07 100644 --- a/src/IO/ResourceGuard.h +++ b/src/IO/ResourceGuard.h @@ -3,10 +3,12 @@ #include #include -#include +#include #include -#include +#include +#include + namespace DB { @@ -14,44 +16,91 @@ namespace DB /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor + * IMPORTANT: multiple resources should not be locked concurently by a single thread */ class ResourceGuard { public: enum ResourceGuardCtor { - LockStraightAway, /// Lock inside constructor (default) - PostponeLocking /// Don't lock in constructor, but during later `lock()` call + LockStraightAway, /// Locks inside constructor (default) + + // WARNING: Only for tests. It is not exception-safe because `lock()` must be called after construction. + PostponeLocking /// Don't lock in constructor, but send request }; - struct Request : public ResourceRequest + enum RequestState { - /// Promise to be set on request execution - std::promise dequeued; + Finished, // Last request has already finished; no concurrent access is possible + Enqueued, // Enqueued into the scheduler; thread-safe access is required + Dequeued // Dequeued from the scheduler and is in consumption state; no concurrent access is possible + }; - explicit Request(ResourceCost cost_ = 1) - : ResourceRequest(cost_) - {} + class Request : public ResourceRequest + { + public: + void enqueue(ResourceCost cost_, ResourceLink link_) + { + // lock(mutex) is not required because `Finished` request cannot be used by the scheduler thread + chassert(state == Finished); + state = Enqueued; + ResourceRequest::reset(cost_); + link_.queue->enqueueRequestUsingBudget(this); + } + // This function is executed inside scheduler thread and wakes thread issued this `request`. + // That thread will continue execution and do real consumption of requested resource synchronously. void execute() override { - // This function is executed inside scheduler thread and wakes thread issued this `request` (using ResourceGuard) - // That thread will continue execution and do real consumption of requested resource synchronously. - dequeued.set_value(); + { + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + } + dequeued_cv.notify_one(); } + + void wait() + { + std::unique_lock lock(mutex); + dequeued_cv.wait(lock, [this] { return state == Dequeued; }); + } + + void finish() + { + // lock(mutex) is not required because `Dequeued` request cannot be used by the scheduler thread + chassert(state == Dequeued); + state = Finished; + if (constraint) + constraint->finishRequest(this); + } + + static Request & local() + { + // Since single thread cannot use more than one resource request simultaneously, + // we can reuse thread-local request to avoid allocations + static thread_local Request instance; + return instance; + } + + private: + std::mutex mutex; + std::condition_variable dequeued_cv; + RequestState state = Finished; }; /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) explicit ResourceGuard(ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) : link(link_) - , request(cost) + , request(Request::local()) { - if (link.queue) + if (cost == 0) + link.queue = nullptr; // Ignore zero-cost requests + else if (link.queue) { - dequeued_future = request.dequeued.get_future(); - link.queue->enqueueRequest(&request); + request.enqueue(cost, link); if (ctor == LockStraightAway) - lock(); + request.wait(); } } @@ -64,17 +113,16 @@ public: void lock() { if (link.queue) - dequeued_future.get(); + request.wait(); } - /// Report request execution has finished + /// Report resource consumption has finished void unlock() { if (link.queue) { - assert(!dequeued_future.valid()); // unlock must be called only after lock() - if (request.constraint) - request.constraint->finishRequest(&request); + request.finish(); + link.queue = nullptr; } } @@ -84,10 +132,8 @@ public: request.successful = false; } -public: ResourceLink link; - Request request; - std::future dequeued_future; + Request & request; }; } diff --git a/src/IO/ResourceLink.h b/src/IO/ResourceLink.h new file mode 100644 index 00000000000..2da5e75fcba --- /dev/null +++ b/src/IO/ResourceLink.h @@ -0,0 +1,39 @@ +#pragma once + +#include + +#include +#include + + +namespace DB +{ + +/* + * Everything required for resource consumption. Connection to a specific resource queue. + */ +struct ResourceLink +{ + ISchedulerQueue * queue = nullptr; + bool operator==(const ResourceLink &) const = default; + + void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const + { + if (queue) + queue->adjustBudget(estimated_cost, real_cost); + } + + void consumed(ResourceCost cost) const + { + if (queue) + queue->consumeBudget(cost); + } + + void accumulate(ResourceCost cost) const + { + if (queue) + queue->accumulateBudget(cost); + } +}; + +} diff --git a/src/IO/ResourceRequest.h b/src/IO/ResourceRequest.h index 93c33bcef22..989349148cf 100644 --- a/src/IO/ResourceRequest.h +++ b/src/IO/ResourceRequest.h @@ -8,29 +8,15 @@ namespace DB // Forward declarations class ISchedulerQueue; -class ISchedulerNode; class ISchedulerConstraint; /// Cost in terms of used resource (e.g. bytes for network IO) using ResourceCost = Int64; constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); -/// Internal identifier of a resource (for arrays; unique per scheduler) -using ResourceIdx = size_t; -constexpr ResourceIdx ResourceIdxNotSet = ResourceIdx(-1); - /// Timestamps (nanoseconds since epoch) using ResourceNs = UInt64; -/* - * Info required for resource consumption. - */ -struct ResourceLink -{ - ISchedulerQueue * queue = nullptr; - bool operator==(const ResourceLink &) const = default; -}; - /* * Request for a resource consumption. The main moving part of the scheduling subsystem. * Resource requests processing workflow: @@ -65,25 +51,36 @@ public: /// Request outcome /// Should be filled during resource consumption - bool successful = true; + bool successful; /// Scheduler node to be notified on consumption finish /// Auto-filled during request enqueue/dequeue - ISchedulerConstraint * constraint = nullptr; + ISchedulerConstraint * constraint; /// Timestamps for introspection - ResourceNs enqueue_ns = 0; - ResourceNs execute_ns = 0; - ResourceNs finish_ns = 0; + ResourceNs enqueue_ns; + ResourceNs execute_ns; + ResourceNs finish_ns; explicit ResourceRequest(ResourceCost cost_ = 1) - : cost(cost_) - {} + { + reset(cost_); + } + + void reset(ResourceCost cost_) + { + cost = cost_; + successful = true; + constraint = nullptr; + enqueue_ns = 0; + execute_ns = 0; + finish_ns = 0; + } virtual ~ResourceRequest() = default; /// Callback to trigger resource consumption. - /// IMPORTANT: is called from scheduler thread and must be fast, + /// IMPORTANT: it is called from scheduler thread and must be fast, /// just triggering start of a consumption, not doing the consumption itself /// (e.g. setting an std::promise or creating a job in a thread pool) virtual void execute() = 0; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index bc887b8ca78..cb6ba1299ad 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -1,12 +1,13 @@ #include "config.h" -#include #if USE_AWS_S3 #include +#include #include #include +#include #include #include #include @@ -342,7 +343,10 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) if (write_settings.for_object_storage) ProfileEvents::increment(ProfileEvents::DiskS3UploadPart); + ResourceCost cost = task.req.GetContentLength(); + ResourceGuard rlock(write_settings.resource_link, cost); auto outcome = client_ptr->UploadPart(task.req); + rlock.unlock(); // Avoid acquiring other locks under resource lock if (outcome.IsSuccess()) { @@ -351,7 +355,10 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) LOG_TRACE(log, "Writing part finished. Bucket: {}, Key: {}, Upload_id: {}, Etag: {}, Parts: {}", bucket, key, multipart_upload_id, task.tag, part_tags.size()); } else + { + write_settings.resource_link.adjust(cost, 0); // We assume no resource was used in case of failure throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); + } } void WriteBufferFromS3::completeMultipartUpload() @@ -491,7 +498,12 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) ProfileEvents::increment(ProfileEvents::S3PutObject); if (write_settings.for_object_storage) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); + + ResourceCost cost = task.req.GetContentLength(); + ResourceGuard rlock(write_settings.resource_link, cost); auto outcome = client_ptr->PutObject(task.req); + rlock.unlock(); + bool with_pool = static_cast(schedule); if (outcome.IsSuccess()) { @@ -500,14 +512,18 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) } else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) { + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure /// For unknown reason, at least MinIO can respond with NO_SUCH_KEY for put requests LOG_INFO(log, "Single part upload failed with NO_SUCH_KEY error for Bucket: {}, Key: {}, Object size: {}, WithPool: {}, will retry", bucket, key, task.req.GetContentLength(), with_pool); } else + { + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure throw S3Exception( outcome.GetError().GetErrorType(), "Message: {}, Key: {}, Bucket: {}, Object size: {}, WithPool: {}", outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool); + } } } diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 764d6c8992b..151bc50c01b 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -11,6 +12,9 @@ struct WriteSettings /// Bandwidth throttler to use during writing ThrottlerPtr remote_throttler; + // Resource to be used during reading + ResourceLink resource_link; + /// Filesystem cache settings bool enable_filesystem_cache_on_write_operations = false; bool enable_filesystem_cache_log = false; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 7f80dcce2d2..58bcef38413 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -2,6 +2,7 @@ #if USE_HDFS #include +#include #include #include #include @@ -97,11 +98,18 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); + rlock.unlock(); + if (bytes_read < 0) + { + read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to read from HDFS: {}, file path: {}. Error: {}", hdfs_uri, hdfs_file_path, std::string(hdfsGetLastError())); + } + read_settings.resource_link.adjust(num_bytes_to_read, bytes_read); if (bytes_read) { diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 2198bb65761..466dd8f04dc 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -62,11 +63,18 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl } - int write(const char * start, size_t size) const + int write(const char * start, size_t size) { + ResourceGuard rlock(write_settings.resource_link, size); int bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + rlock.unlock(); + if (bytes_written < 0) + { + write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); + } + write_settings.resource_link.adjust(size, bytes_written); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); From f1a940d7d3a91533f7c514bd97a60ea9e8a51a15 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 27 Jan 2023 18:55:46 +0000 Subject: [PATCH 029/116] fix --- src/IO/ReadBufferFromS3.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 8ad1674297b..17b13bf7d62 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -88,8 +88,6 @@ private: /// There is different seek policy for disk seek and for non-disk seek /// (non-disk seek is applied for seekable input formats: orc, arrow, parquet). bool restricted_seek; - - ResourceBudget budget; }; /// Creates separate ReadBufferFromS3 for sequence of ranges of particular object From 6d6dc40128fb3b3ccd94bedbc41ebba352d816c1 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 27 Jan 2023 22:48:02 +0000 Subject: [PATCH 030/116] fix typos --- src/IO/ISchedulerQueue.h | 4 ++-- src/IO/ResourceBudget.h | 2 +- src/IO/ResourceGuard.h | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/IO/ISchedulerQueue.h b/src/IO/ISchedulerQueue.h index 9da6c5d33ac..fc2f3943d26 100644 --- a/src/IO/ISchedulerQueue.h +++ b/src/IO/ISchedulerQueue.h @@ -34,7 +34,7 @@ public: budget.adjust(estimated_cost, real_cost); } - // Adjust budget to account for extra consumtion of `cost` resource units + // Adjust budget to account for extra consumption of `cost` resource units void consumeBudget(ResourceCost cost) { adjustBudget(0, cost); @@ -53,7 +53,7 @@ public: private: // Allows multiple consumers to synchronize with common "debit/credit" balance. // 1) (positive) to avoid wasting of allocated but not used resource (e.g in case of a failure); - // 2) (negative) to account for overconsumption (e.g. if cost is not know in advance and esimation from below is applied). + // 2) (negative) to account for overconsumption (e.g. if cost is not know in advance and estimation from below is applied). ResourceBudget budget; }; diff --git a/src/IO/ResourceBudget.h b/src/IO/ResourceBudget.h index 4430ddbd78c..7f67f9cfc10 100644 --- a/src/IO/ResourceBudget.h +++ b/src/IO/ResourceBudget.h @@ -31,7 +31,7 @@ public: // Valid resource request must have positive `cost`. Also takes consumption history into account. ResourceCost cost = std::max(1ll, estimated_cost - budget); - // Assume every request is satisfied (no resource request cancelation is possible now) + // Assume every request is satisfied (no resource request cancellation is possible now) // So we requested additional `cost` units and are going to consume `estimated_cost` ResourceCost new_budget = budget + cost - estimated_cost; diff --git a/src/IO/ResourceGuard.h b/src/IO/ResourceGuard.h index dbd9c83fa07..92f25b40f6b 100644 --- a/src/IO/ResourceGuard.h +++ b/src/IO/ResourceGuard.h @@ -16,7 +16,7 @@ namespace DB /* * Scoped resource guard. * Waits for resource to be available in constructor and releases resource in destructor - * IMPORTANT: multiple resources should not be locked concurently by a single thread + * IMPORTANT: multiple resources should not be locked concurrently by a single thread */ class ResourceGuard { From fe3a755064127ecca214db5cc921c30b8b9c8396 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Feb 2023 14:05:59 +0100 Subject: [PATCH 031/116] Fix test after incorrect merge with master --- src/Disks/DiskSelector.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index f984f4ff85c..9894e4251a2 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -118,8 +118,13 @@ DiskSelectorPtr DiskSelector::updateFromConfig( writeBackQuotedString(name, warning); } - LOG_WARNING(&Poco::Logger::get("DiskSelector"), "{} disappeared from configuration, " - "this change will be applied after restart of ClickHouse", warning.str()); + if (num_disks_removed_from_config > 0) + { + LOG_WARNING( + &Poco::Logger::get("DiskSelector"), + "{} disappeared from configuration, this change will be applied after restart of ClickHouse", + warning.str()); + } } return result; From 12b62de3744ff895a43a4b58b50e2acbfe37c9a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Feb 2023 17:07:02 +0100 Subject: [PATCH 032/116] Add field CustomType --- src/Common/FieldVisitorConvertToNumber.h | 5 +++ src/Common/FieldVisitorDump.h | 2 +- src/Common/FieldVisitorHash.h | 1 + src/Common/FieldVisitorSum.h | 1 + src/Common/FieldVisitorToString.cpp | 1 + src/Common/FieldVisitorToString.h | 1 + src/Common/FieldVisitorWriteBinary.h | 2 +- src/Core/Field.cpp | 1 + src/Core/Field.h | 54 +++++++++++++++++++++++- src/DataTypes/FieldToDataType.cpp | 6 +++ src/DataTypes/FieldToDataType.h | 1 + 11 files changed, 72 insertions(+), 3 deletions(-) diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index ed73cd38cda..2e11b3173d7 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -106,6 +106,11 @@ public: throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert AggregateFunctionStateData to {}", demangle(typeid(T).name())); } + T operator() (const CustomType &) const + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert CustomType to {}", demangle(typeid(T).name())); + } + template requires is_big_int_v T operator() (const U & x) const diff --git a/src/Common/FieldVisitorDump.h b/src/Common/FieldVisitorDump.h index 6ffd91bb400..5f5ecc35a8e 100644 --- a/src/Common/FieldVisitorDump.h +++ b/src/Common/FieldVisitorDump.h @@ -30,8 +30,8 @@ public: String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; String operator() (const AggregateFunctionStateData & x) const; + String operator() (const CustomType & x) const; String operator() (const bool & x) const; }; } - diff --git a/src/Common/FieldVisitorHash.h b/src/Common/FieldVisitorHash.h index 1350956146b..47699e96998 100644 --- a/src/Common/FieldVisitorHash.h +++ b/src/Common/FieldVisitorHash.h @@ -36,6 +36,7 @@ public: void operator() (const DecimalField & x) const; void operator() (const DecimalField & x) const; void operator() (const AggregateFunctionStateData & x) const; + void operator() (const CustomType & x) const; void operator() (const bool & x) const; }; diff --git a/src/Common/FieldVisitorSum.h b/src/Common/FieldVisitorSum.h index 5b9c83d1dd1..cbb4c4a1de3 100644 --- a/src/Common/FieldVisitorSum.h +++ b/src/Common/FieldVisitorSum.h @@ -31,6 +31,7 @@ public: bool operator() (IPv4 &) const; bool operator() (IPv6 &) const; bool operator() (AggregateFunctionStateData &) const; + bool operator() (CustomType &) const; bool operator() (bool &) const; template diff --git a/src/Common/FieldVisitorToString.cpp b/src/Common/FieldVisitorToString.cpp index ef21a13b700..e0ec6b7a335 100644 --- a/src/Common/FieldVisitorToString.cpp +++ b/src/Common/FieldVisitorToString.cpp @@ -69,6 +69,7 @@ String FieldVisitorToString::operator() (const IPv4 & x) const { return formatQu String FieldVisitorToString::operator() (const IPv6 & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); } String FieldVisitorToString::operator() (const bool & x) const { return x ? "true" : "false"; } +String FieldVisitorToString::operator() (const CustomType & x) const { return x.toString(); } String FieldVisitorToString::operator() (const Array & x) const { diff --git a/src/Common/FieldVisitorToString.h b/src/Common/FieldVisitorToString.h index 6a10de12ff9..ea4b43ead08 100644 --- a/src/Common/FieldVisitorToString.h +++ b/src/Common/FieldVisitorToString.h @@ -30,6 +30,7 @@ public: String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; String operator() (const AggregateFunctionStateData & x) const; + String operator() (const CustomType & x) const; String operator() (const bool & x) const; }; diff --git a/src/Common/FieldVisitorWriteBinary.h b/src/Common/FieldVisitorWriteBinary.h index bc75150bed2..915e75fbbdd 100644 --- a/src/Common/FieldVisitorWriteBinary.h +++ b/src/Common/FieldVisitorWriteBinary.h @@ -29,8 +29,8 @@ public: void operator() (const DecimalField & x, WriteBuffer & buf) const; void operator() (const DecimalField & x, WriteBuffer & buf) const; void operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const; + void operator() (const CustomType & x, WriteBuffer & buf) const; void operator() (const bool & x, WriteBuffer & buf) const; }; } - diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 9ada9c719fa..976f47cdbe3 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -597,6 +597,7 @@ String fieldTypeToString(Field::Types::Which type) case Field::Types::Which::UUID: return "UUID"; case Field::Types::Which::IPv4: return "IPv4"; case Field::Types::Which::IPv6: return "IPv6"; + case Field::Types::Which::CustomType: return "CustomType"; } } diff --git a/src/Core/Field.h b/src/Core/Field.h index d98ff61c9ea..bfb5cc1f324 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -102,6 +102,43 @@ struct AggregateFunctionStateData } }; +struct CustomType +{ + struct CustomTypeImpl + { + virtual ~CustomTypeImpl() = default; + virtual const char * getTypeName() const = 0; + virtual String toString() const = 0; + + // virtual std::unique_ptr clone() const; + // virtual std::optional getBaseTypeIfConvertible() const; + // virtual std::optional tryConvertToBaseType(Types::Which dest_type) const + + bool operator < (const CustomTypeImpl &) const { throwNotImpleneted("<"); } + bool operator <= (const CustomTypeImpl &) const { throwNotImpleneted("<="); } + bool operator > (const CustomTypeImpl &) const { throwNotImpleneted(">"); } + bool operator >= (const CustomTypeImpl &) const { throwNotImpleneted(">="); } + bool operator == (const CustomTypeImpl &) const { throwNotImpleneted("=="); } + + [[noreturn]] void throwNotImpleneted(const std::string & op) const + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Operation {} is not implemented for custom type {}", op, getTypeName()); + } + }; + + String toString() const { return impl->toString(); } + + bool operator < (const CustomType & rhs) const { return *impl < *rhs.impl; } + bool operator <= (const CustomType & rhs) const { return *impl <= *rhs.impl; } + bool operator > (const CustomType & rhs) const { return *impl > *rhs.impl; } + bool operator >= (const CustomType & rhs) const { return *impl >= *rhs.impl; } + bool operator == (const CustomType & rhs) const { return *impl == *rhs.impl; } + + std::shared_ptr impl; + +}; + template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); @@ -233,6 +270,7 @@ template <> struct NearestFieldTypeImpl { using Type = UInt64; }; template <> struct NearestFieldTypeImpl { using Type = Null; }; template <> struct NearestFieldTypeImpl { using Type = AggregateFunctionStateData; }; +template <> struct NearestFieldTypeImpl { using Type = CustomType; }; // For enum types, use the field type that corresponds to their underlying type. template @@ -297,6 +335,7 @@ public: Object = 29, IPv4 = 30, IPv6 = 31, + CustomType = 32, }; }; @@ -486,6 +525,7 @@ public: case Types::Decimal128: return get>() < rhs.get>(); case Types::Decimal256: return get>() < rhs.get>(); case Types::AggregateFunctionState: return get() < rhs.get(); + case Types::CustomType: return get() < rhs.get(); } throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Bad type of Field"); @@ -527,6 +567,7 @@ public: case Types::Decimal128: return get>() <= rhs.get>(); case Types::Decimal256: return get>() <= rhs.get>(); case Types::AggregateFunctionState: return get() <= rhs.get(); + case Types::CustomType: return get() <= rhs.get(); } throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Bad type of Field"); @@ -572,6 +613,7 @@ public: case Types::Decimal128: return get>() == rhs.get>(); case Types::Decimal256: return get>() == rhs.get>(); case Types::AggregateFunctionState: return get() == rhs.get(); + case Types::CustomType: return get() == rhs.get(); } throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Bad type of Field"); @@ -615,6 +657,7 @@ public: case Types::Decimal128: return f(field.template get>()); case Types::Decimal256: return f(field.template get>()); case Types::AggregateFunctionState: return f(field.template get()); + case Types::CustomType: return f(field.template get()); } UNREACHABLE(); @@ -627,7 +670,7 @@ private: std::aligned_union_t, DecimalField, DecimalField, DecimalField, - AggregateFunctionStateData + AggregateFunctionStateData, CustomType > storage; Types::Which which; @@ -731,6 +774,9 @@ private: case Types::AggregateFunctionState: destroy(); break; + case Types::CustomType: + destroy(); + break; default: break; } @@ -774,6 +820,7 @@ template <> struct Field::TypeToEnum>{ static constexpr template <> struct Field::TypeToEnum>{ static constexpr Types::Which value = Types::Decimal256; }; template <> struct Field::TypeToEnum>{ static constexpr Types::Which value = Types::Decimal64; }; template <> struct Field::TypeToEnum{ static constexpr Types::Which value = Types::AggregateFunctionState; }; +template <> struct Field::TypeToEnum{ static constexpr Types::Which value = Types::CustomType; }; template <> struct Field::TypeToEnum{ static constexpr Types::Which value = Types::Bool; }; template <> struct Field::EnumToType { using Type = Null; }; @@ -797,6 +844,7 @@ template <> struct Field::EnumToType { using Type = Dec template <> struct Field::EnumToType { using Type = DecimalField; }; template <> struct Field::EnumToType { using Type = DecimalField; }; template <> struct Field::EnumToType { using Type = DecimalField; }; +template <> struct Field::EnumToType { using Type = CustomType; }; template <> struct Field::EnumToType { using Type = UInt64; }; inline constexpr bool isInt64OrUInt64FieldType(Field::Types::Which t) @@ -935,6 +983,10 @@ void readBinary(Object & x, ReadBuffer & buf); void writeBinary(const Object & x, WriteBuffer & buf); void writeText(const Object & x, WriteBuffer & buf); + +void writeBinary(const CustomType & x, WriteBuffer & buf); +void writeText(const CustomType & x, WriteBuffer & buf); + [[noreturn]] inline void writeQuoted(const Object &, WriteBuffer &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot write Object quoted."); } __attribute__ ((noreturn)) inline void writeText(const AggregateFunctionStateData &, WriteBuffer &) diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 2841ccee12f..0a70761a64e 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -191,6 +191,12 @@ DataTypePtr FieldToDataType::operator() (const AggregateFunctionStateD return DataTypeFactory::instance().get(name); } +template +DataTypePtr FieldToDataType::operator() (const CustomType &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); +} + template DataTypePtr FieldToDataType::operator()(const bool &) const { diff --git a/src/DataTypes/FieldToDataType.h b/src/DataTypes/FieldToDataType.h index bd7d5b1af85..8febadc1a0d 100644 --- a/src/DataTypes/FieldToDataType.h +++ b/src/DataTypes/FieldToDataType.h @@ -41,6 +41,7 @@ public: DataTypePtr operator() (const DecimalField & x) const; DataTypePtr operator() (const DecimalField & x) const; DataTypePtr operator() (const AggregateFunctionStateData & x) const; + DataTypePtr operator() (const CustomType & x) const; DataTypePtr operator() (const UInt256 & x) const; DataTypePtr operator() (const Int256 & x) const; DataTypePtr operator() (const bool & x) const; From 5f124263a6cc6aa1896de75bc1a4ee6d88afc4bd Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Feb 2023 21:03:13 +0100 Subject: [PATCH 033/116] Revert changes with complex SettingsChanges --- programs/client/Client.cpp | 2 +- src/Access/SettingsConstraints.cpp | 20 +++++++++--------- src/Access/SettingsConstraints.h | 2 +- src/Analyzer/QueryNode.cpp | 2 +- src/Backups/BackupSettings.cpp | 10 ++++----- src/Backups/RestoreSettings.cpp | 6 +++--- src/Client/ClientBase.cpp | 4 ++-- .../NamedCollectionConfiguration.cpp | 2 +- .../NamedCollections/NamedCollectionUtils.cpp | 10 ++++----- src/Core/BaseSettings.h | 2 +- src/Databases/DatabaseOnDisk.cpp | 4 ++-- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 12 +++++------ .../getDictionaryConfigurationFromAST.cpp | 8 +++---- src/Interpreters/Cache/QueryCache.cpp | 2 +- src/Interpreters/Context.cpp | 4 ++-- src/Interpreters/InterpreterExplainQuery.cpp | 20 +++++++++--------- src/Parsers/ASTAlterNamedCollectionQuery.cpp | 4 ++-- src/Parsers/ASTBackupQuery.cpp | 2 +- src/Parsers/ASTCreateNamedCollectionQuery.cpp | 4 ++-- src/Parsers/ASTDictionary.cpp | 2 +- .../InsertQuerySettingsPushDownVisitor.cpp | 2 +- .../InsertQuerySettingsPushDownVisitor.h | 2 +- src/Parsers/ParserBackupQuery.cpp | 2 +- ...QueryWithOutputSettingsPushDownVisitor.cpp | 4 ++-- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Storages/AlterCommands.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeData.cpp | 10 ++++----- src/Storages/MergeTree/MergeTreePartition.cpp | 8 +++++++ src/Storages/MergeTree/MergeTreeSettings.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 21 +++++++++++-------- .../MaterializedPostgreSQLConsumer.cpp | 8 +++---- .../MaterializedPostgreSQLConsumer.h | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- src/Storages/StorageFile.cpp | 4 ++-- src/Storages/StorageJoin.cpp | 4 ++-- src/Storages/StorageLogSettings.cpp | 4 ++-- src/Storages/StorageS3.cpp | 4 ++-- src/Storages/StorageURL.cpp | 4 ++-- .../System/StorageSystemDDLWorkerQueue.cpp | 4 ++-- 40 files changed, 115 insertions(+), 104 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 3515a87e139..d741eb30d4a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -548,7 +548,7 @@ void Client::printChangedSettings() const { fmt::print(stderr, ", "); } - fmt::print(stderr, "{} = '{}'", changes[i].getName(), changes[i].getValueString()); + fmt::print(stderr, "{} = '{}'", changes[i].name, toString(changes[i].value)); } fmt::print(stderr, "\n"); } diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 46905f08c06..ded33005fc0 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -175,19 +175,19 @@ template bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) { Field current_value; - bool has_current_value = current_settings.tryGet(change.getName(), current_value); + bool has_current_value = current_settings.tryGet(change.name, current_value); /// Setting isn't checked if value has not changed. - if (has_current_value && change.getFieldValue() == current_value) + if (has_current_value && change.value == current_value) return false; if (throw_on_failure) - new_value = T::castValueUtil(change.getName(), change.getFieldValue()); + new_value = T::castValueUtil(change.name, change.value); else { try { - new_value = T::castValueUtil(change.getName(), change.getFieldValue()); + new_value = T::castValueUtil(change.name, change.value); } catch (...) { @@ -204,7 +204,7 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - const String & setting_name = change.getName(); + const String & setting_name = change.name; if (setting_name == "profile") return true; @@ -219,7 +219,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh { if (e.code() == ErrorCodes::UNKNOWN_SETTING) { - if (const auto hints = current_settings.getHints(change.getName()); !hints.empty()) + if (const auto hints = current_settings.getHints(change.name); !hints.empty()) { e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); } @@ -242,12 +242,12 @@ bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, Field new_value; if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - return getMergeTreeChecker(change.getName()).check(change, new_value, reaction); + return getMergeTreeChecker(change.name).check(change, new_value, reaction); } bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const { - const String & setting_name = change.getName(); + const String & setting_name = change.name; auto less_or_cannot_compare = [=](const Field & left, const Field & right) { @@ -298,7 +298,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n setting_name, applyVisitor(FieldVisitorToString(), min_value)); } else - change.setValue(min_value); + change.value = min_value; } if (!max_value.isNull() && less_or_cannot_compare(max_value, new_value)) @@ -309,7 +309,7 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n setting_name, applyVisitor(FieldVisitorToString(), max_value)); } else - change.setValue(max_value); + change.value = max_value; } return true; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 7b0ad40185c..9f66f591835 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,7 +12,7 @@ namespace Poco::Util namespace DB { struct Settings; -class SettingChange; +struct SettingChange; struct MergeTreeSettings; class SettingsChanges; class AccessControl; diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index ca226431ca4..91fb3a1851d 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -186,7 +186,7 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s { buffer << '\n' << std::string(indent + 2, ' ') << "SETTINGS"; for (const auto & change : settings_changes) - buffer << fmt::format(" {}={}", change.getName(), toString(change.getFieldValue())); + buffer << fmt::format(" {}={}", change.name, toString(change.value)); } } diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 95c490a28d7..c801430cad7 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -41,16 +41,16 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) const auto & settings = query.settings->as().changes; for (const auto & setting : settings) { - if (setting.getName() == "compression_level") - res.compression_level = static_cast(SettingFieldInt64{setting.getFieldValue()}.value); + if (setting.name == "compression_level") + res.compression_level = static_cast(SettingFieldInt64{setting.value}.value); else #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ - if (setting.getName() == #NAME) \ - res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ + if (setting.name == #NAME) \ + res.NAME = SettingField##TYPE{setting.value}.value; \ else LIST_OF_BACKUP_SETTINGS(GET_SETTINGS_FROM_BACKUP_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); } } diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index db2ffcc426c..43fc0e4dce0 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -177,11 +177,11 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) for (const auto & setting : settings) { #define GET_SETTINGS_FROM_RESTORE_QUERY_HELPER(TYPE, NAME) \ - if (setting.getName() == #NAME) \ - res.NAME = SettingField##TYPE{setting.getFieldValue()}.value; \ + if (setting.name == #NAME) \ + res.NAME = SettingField##TYPE{setting.value}.value; \ else LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) - throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.getName()); + throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c0419714a0f..0caedf67513 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1638,8 +1638,8 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin /// Save all changes in settings to avoid losing them if the connection is lost. for (const auto & change : set_query->changes) { - if (change.getName() == "profile") - current_profile = change.getFieldValue().safeGet(); + if (change.name == "profile") + current_profile = change.value.safeGet(); else global_context->applySettingChange(change); } diff --git a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp index 4f5e0fd837d..d593cad5a21 100644 --- a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp +++ b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp @@ -151,7 +151,7 @@ ConfigurationPtr createConfiguration(const std::string & root_name, const Settin auto config = Configuration::createEmptyConfiguration(root_name); for (const auto & setting : settings) - Configuration::setConfigValue(*config, setting.getName(), convertFieldToString(setting.getFieldValue())); + Configuration::setConfigValue(*config, setting.name, convertFieldToString(setting.value)); return config; } diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 05dcaad22e9..8c84352dbc5 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -182,18 +182,18 @@ public: std::unordered_map result_changes_map; for (const auto & change : query.changes) { - auto [it, inserted] = result_changes_map.emplace(change.getName(), change.getFieldValue()); + auto [it, inserted] = result_changes_map.emplace(change.name, change.value); if (!inserted) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Value with key `{}` is used twice in the SET query (collection name: {})", - change.getName(), query.collection_name); + change.name, query.collection_name); } } for (const auto & change : create_query.changes) - result_changes_map.emplace(change.getName(), change.getFieldValue()); + result_changes_map.emplace(change.name, change.value); for (const auto & delete_key : query.delete_keys) { @@ -254,7 +254,7 @@ private: std::set> keys; for (const auto & change : query.changes) - keys.insert(change.getName()); + keys.insert(change.name); return NamedCollection::create( *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); @@ -379,7 +379,7 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex auto collection_lock = collection->lock(); for (const auto & change : query.changes) - collection->setOrUpdate(change.getName(), convertFieldToString(change.getFieldValue())); + collection->setOrUpdate(change.name, convertFieldToString(change.value)); for (const auto & key : query.delete_keys) collection->remove(key); diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 41a3d6d2ee1..7b56367769e 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -293,7 +293,7 @@ SettingsChanges BaseSettings::changes() const template void BaseSettings::applyChange(const SettingChange & change) { - set(change.getName(), change.getFieldValue()); + set(change.name, change.value); } template diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 24a401edfcf..4d9e22bd15d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -755,9 +755,9 @@ void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_cha for (const auto & change : settings_changes) { auto it = std::find_if(storage_settings.begin(), storage_settings.end(), - [&](const auto & prev){ return prev.getName() == change.getName(); }); + [&](const auto & prev){ return prev.name == change.name; }); if (it != storage_settings.end()) - it->setValue(change.getFieldValue()); + it->value = change.value; else storage_settings.push_back(change); } diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 1fe452de47a..3f4c2d8e466 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) SettingsChanges & changes = storage_def.settings->changes; #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.getName() == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, static_cast(NAME)}); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index c951d648fe7..5cfc3535a0f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -139,18 +139,18 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges for (const auto & change : settings_changes) { - if (!settings->has(change.getName())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.getName()); + if (!settings->has(change.name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine {} does not support setting `{}`", getEngineName(), change.name); - if ((change.getName() == "materialized_postgresql_tables_list")) + if ((change.name == "materialized_postgresql_tables_list")) { if (!query_context->isInternalQuery()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.getName()); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Changing setting `{}` is not allowed", change.name); need_update_on_disk = true; } - else if ((change.getName() == "materialized_postgresql_allow_automatic_update") - || (change.getName() == "materialized_postgresql_max_block_size")) + else if ((change.name == "materialized_postgresql_allow_automatic_update") + || (change.name == "materialized_postgresql_max_block_size")) { replication_handler->setSetting(change); need_update_on_disk = true; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index ea62479f99a..8ef63a55d43 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -127,9 +127,9 @@ void buildLayoutConfiguration( root->appendChild(settings_element); for (const auto & change : settings->changes) { - AutoPtr setting_change_element(doc->createElement(change.getName())); + AutoPtr setting_change_element(doc->createElement(change.name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue()))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.value))); setting_change_element->appendChild(setting_value); } } @@ -531,9 +531,9 @@ void buildSourceConfiguration( outer_element->appendChild(settings_element); for (const auto & change : settings->changes) { - AutoPtr setting_change_element(doc->createElement(change.getName())); + AutoPtr setting_change_element(doc->createElement(change.name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.getFieldValue()))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.value))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 6d3a6d96b33..ea20e323459 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -79,7 +79,7 @@ public: auto is_query_cache_related_setting = [](const auto & change) { - const auto & name = change.getName(); + const auto & name = change.name; return name == "allow_experimental_query_cache" || name.starts_with("query_cache") || name.ends_with("query_cache"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6f4cef72cf3..0ef49fec556 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1513,11 +1513,11 @@ void Context::applySettingChange(const SettingChange & change) { try { - setSetting(change.getName(), change.getFieldValue()); + setSetting(change.name, change.value); } catch (Exception & e) { - e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.getName(), change.getValueString())); + e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.name, toString(change.value))); throw; } } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index cdcc5031dbb..3c225522cc4 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -313,28 +313,28 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) for (const auto & change : set_query.changes) { - if (!settings.has(change.getName())) + if (!settings.has(change.name)) throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting \"{}\" for EXPLAIN {} query. " - "Supported settings: {}", change.getName(), Settings::name, settings.getSettingsList()); + "Supported settings: {}", change.name, Settings::name, settings.getSettingsList()); - if (change.getFieldValue().getType() != Field::Types::UInt64) + if (change.value.getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid type {} for setting \"{}\" only integer settings are supported", - change.getFieldValue().getTypeName(), change.getName()); + change.value.getTypeName(), change.name); - if (settings.hasBooleanSetting(change.getName())) + if (settings.hasBooleanSetting(change.name)) { - auto value = change.getFieldValue().get(); + auto value = change.value.get(); if (value > 1) throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Invalid value {} for setting \"{}\". " - "Expected boolean type", value, change.getName()); + "Expected boolean type", value, change.name); - settings.setBooleanSetting(change.getName(), value); + settings.setBooleanSetting(change.name, value); } else { - auto value = change.getFieldValue().get(); - settings.setIntegerSetting(change.getName(), value); + auto value = change.value.get(); + settings.setIntegerSetting(change.name, value); } } diff --git a/src/Parsers/ASTAlterNamedCollectionQuery.cpp b/src/Parsers/ASTAlterNamedCollectionQuery.cpp index 7e7b36acb42..7e95147ad75 100644 --- a/src/Parsers/ASTAlterNamedCollectionQuery.cpp +++ b/src/Parsers/ASTAlterNamedCollectionQuery.cpp @@ -28,9 +28,9 @@ void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & setti else first = false; - formatSettingName(change.getName(), settings.ostr); + formatSettingName(change.name, settings.ostr); if (settings.show_secrets) - settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); + settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); else settings.ostr << " = '[HIDDEN]'"; } diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 4d8b0ca7960..85a0a0c7799 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -184,7 +184,7 @@ namespace changes, [](const SettingChange & change) { - const String & name = change.getName(); + const String & name = change.name; return (name == "internal") || (name == "async") || (name == "host_id"); }); diff --git a/src/Parsers/ASTCreateNamedCollectionQuery.cpp b/src/Parsers/ASTCreateNamedCollectionQuery.cpp index 56b6259fc39..97e83541f05 100644 --- a/src/Parsers/ASTCreateNamedCollectionQuery.cpp +++ b/src/Parsers/ASTCreateNamedCollectionQuery.cpp @@ -31,10 +31,10 @@ void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & sett else first = false; - formatSettingName(change.getName(), settings.ostr); + formatSettingName(change.name, settings.ostr); if (settings.show_secrets) - settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.getFieldValue()); + settings.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); else settings.ostr << " = '[HIDDEN]'"; } diff --git a/src/Parsers/ASTDictionary.cpp b/src/Parsers/ASTDictionary.cpp index d99ecc3fbad..66c1c3791b8 100644 --- a/src/Parsers/ASTDictionary.cpp +++ b/src/Parsers/ASTDictionary.cpp @@ -118,7 +118,7 @@ void ASTDictionarySettings::formatImpl(const FormatSettings & settings, if (it != changes.begin()) settings.ostr << ", "; - settings.ostr << it->getName() << " = " << applyVisitor(FieldVisitorToString(), it->getFieldValue()); + settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value); } settings.ostr << (settings.hilite ? hilite_none : "") << ")"; } diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index 0080b6d5540..1cebdfde957 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -49,7 +49,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS { auto it = std::find_if(insert_settings.begin(), insert_settings.end(), [&](auto & select_setting) { - return select_setting.getName() == setting.getName(); + return select_setting.name == setting.name; }); if (it == insert_settings.end()) insert_settings.push_back(setting); diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.h b/src/Parsers/InsertQuerySettingsPushDownVisitor.h index a32ecc8ee53..d1f161fc89b 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.h +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -class SettingChange; +struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause to the INSERT from the SELECT query: diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 1235474f315..cead1a98c1a 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -304,7 +304,7 @@ namespace changes = assert_cast(settings.get())->changes; } - boost::remove_erase_if(changes, [](const SettingChange & change) { return change.getName() == "async"; }); + boost::remove_erase_if(changes, [](const SettingChange & change) { return change.name == "async"; }); changes.emplace_back("async", async); auto new_settings = std::make_shared(); diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp index 12670cd67b6..8cf0d0063ae 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.cpp @@ -44,12 +44,12 @@ void QueryWithOutputSettingsPushDownMatcher::visit(ASTSelectQuery & select_query { auto it = std::find_if(select_settings.begin(), select_settings.end(), [&](auto & select_setting) { - return select_setting.getName() == setting.getName(); + return select_setting.name == setting.name; }); if (it == select_settings.end()) select_settings.push_back(setting); else - it->setValue(setting.getFieldValue()); + it->value = setting.value; } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index 1aefa16d20c..fde8a07b555 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -7,7 +7,7 @@ namespace DB { class ASTSelectQuery; -class SettingChange; +struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index fa1faff3bff..da11a87eb4d 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -619,11 +619,11 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & change : settings_changes) { - auto finder = [&change](const SettingChange & c) { return c.getName() == change.getName(); }; + auto finder = [&change](const SettingChange & c) { return c.name == change.name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) - it->setValue(change.getFieldValue()); + it->value = change.value; else settings_from_storage.push_back(change); } @@ -633,7 +633,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto & settings_from_storage = metadata.settings_changes->as().changes; for (const auto & setting_name : settings_resets) { - auto finder = [&setting_name](const SettingChange & c) { return c.getName() == setting_name; }; + auto finder = [&setting_name](const SettingChange & c) { return c.name == setting_name; }; auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder); if (it != settings_from_storage.end()) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 601f8f5876c..a5f42f93f0f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3049,8 +3049,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context for (const auto & changed_setting : new_changes) { - const auto & setting_name = changed_setting.getName(); - const auto & new_value = changed_setting.getFieldValue(); + const auto & setting_name = changed_setting.name; + const auto & new_value = changed_setting.value; MergeTreeSettings::checkCanSet(setting_name, new_value); const Field * current_value = current_changes.tryGet(setting_name); @@ -3076,7 +3076,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context /// Check if it is safe to reset the settings for (const auto & current_setting : current_changes) { - const auto & setting_name = current_setting.getName(); + const auto & setting_name = current_setting.name; const Field * new_value = new_changes.tryGet(setting_name); /// Prevent unsetting readonly setting if (MergeTreeSettings::isReadonlySetting(setting_name) && !new_value) @@ -3174,9 +3174,9 @@ void MergeTreeData::changeSettings( for (const auto & change : new_changes) { - if (change.getName() == "storage_policy") + if (change.name == "storage_policy") { - StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.getFieldValue().safeGet()); + StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); StoragePolicyPtr old_storage_policy = getStoragePolicy(); /// StoragePolicy of different version or name is guaranteed to have different pointer diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 903f467d159..d6c432ad4bf 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -181,6 +181,14 @@ namespace hash.update(x.data.size()); hash.update(x.data.data(), x.data.size()); } + void operator() (const CustomType & x) const + { + UInt8 type = Field::Types::CustomType; + hash.update(type); + auto result = x.toString(); + hash.update(result.size()); + hash.update(result.data(), result.size()); + } void operator() (const bool & x) const { UInt8 type = Field::Types::Bool; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 147faee9c4b..2bf717c883a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -65,7 +65,7 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.getName() == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, (NAME).value}); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index d23cac6cdbd..e5aa2607ee0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include @@ -592,16 +592,19 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->settings) { - for (auto & change : args.storage_def->settings->changes) + for (auto & [name, value] : args.storage_def->settings->changes) { - auto value = change.getValue(); - auto * ast_value = dynamic_cast(value.get()); - if (ast_value && isDiskFunction(ast_value->value)) + CustomType custom; + if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) { - const auto & ast_function = assert_cast(*ast_value->value); - auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); - ast_value->setField(disk_name); - break; + auto ast = dynamic_cast(custom.getImpl()).ast; + if (ast && isDiskFunction(ast)) + { + const auto & ast_function = assert_cast(*ast); + auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); + value = disk_name; + break; + } } } } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 8c0cd8073c8..574b5d76bbe 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -731,10 +731,10 @@ void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_ void MaterializedPostgreSQLConsumer::setSetting(const SettingChange & setting) { - if (setting.getName() == "materialized_postgresql_max_block_size") - max_block_size = setting.getFieldValue().safeGet(); - else if (setting.getName() == "materialized_postgresql_allow_automatic_update") - allow_automatic_update = setting.getFieldValue().safeGet(); + if (setting.name == "materialized_postgresql_max_block_size") + max_block_size = setting.value.safeGet(); + else if (setting.name == "materialized_postgresql_allow_automatic_update") + allow_automatic_update = setting.value.safeGet(); } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 9aacf96fe05..37caa66aae5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -13,7 +13,7 @@ namespace DB { -class SettingChange; +struct SettingChange; struct StorageInfo { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 4b5581fc4b5..89f16457bfe 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -11,7 +11,7 @@ namespace DB { class StorageMaterializedPostgreSQL; -class SettingChange; +struct SettingChange; class PostgreSQLReplicationHandler : WithContext { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 10e3aaaf641..e2a2f84bc72 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1218,9 +1218,9 @@ void registerStorageFile(StorageFactory & factory) const auto & changes = factory_args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) + if (user_format_settings.has(change.name)) { - user_format_settings.set(change.getName(), change.getFieldValue()); + user_format_settings.set(change.name, change.value); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index f54d83b147d..0be7097ad2c 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -305,8 +305,8 @@ void registerStorageJoin(StorageFactory & factory) { for (const auto & setting : args.storage_def->settings->changes) { - const auto & setting_name = setting.getName(); - const auto & setting_value = setting.getFieldValue(); + const auto & setting_name = setting.name; + const auto & setting_value = setting.value; if (setting_name == "join_use_nulls") join_use_nulls = setting_value; else if (setting_name == "max_rows_in_join") diff --git a/src/Storages/StorageLogSettings.cpp b/src/Storages/StorageLogSettings.cpp index e623297d4c0..900e1070eac 100644 --- a/src/Storages/StorageLogSettings.cpp +++ b/src/Storages/StorageLogSettings.cpp @@ -10,8 +10,8 @@ String getDiskName(ASTStorage & storage_def) { SettingsChanges changes = storage_def.settings->changes; for (const auto & change : changes) - if (change.getName() == "disk") - return change.getFieldValue().safeGet(); + if (change.name == "disk") + return change.value.safeGet(); } return "default"; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ec017784f9f..205b0c7d067 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1451,8 +1451,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) - user_format_settings.set(change.getName(), change.getFieldValue()); + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); } // Apply changes from SETTINGS clause, with validation. diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index b057edf4a6a..9bcbc9e6f45 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -983,9 +983,9 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum const auto & changes = args.getContext()->getSettingsRef().changes(); for (const auto & change : changes) { - if (user_format_settings.has(change.getName())) + if (user_format_settings.has(change.name)) { - user_format_settings.set(change.getName(), change.getFieldValue()); + user_format_settings.set(change.name, change.value); } } diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 6714eda3e70..67867b6c577 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -122,8 +122,8 @@ static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const for (const auto & change : *task.entry.settings) { Tuple pair; - pair.push_back(change.getName()); - pair.push_back(toString(change.getFieldValue())); + pair.push_back(change.name); + pair.push_back(toString(change.value)); settings_map.push_back(std::move(pair)); } } From 536c02ca8a660ea192d2d4d5902e555e663bb112 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Feb 2023 21:05:18 +0100 Subject: [PATCH 034/116] Adding custom field continued --- src/Common/SettingsChanges.cpp | 35 ++++-------------------- src/Common/SettingsChanges.h | 44 +++++-------------------------- src/Core/Field.h | 10 +++---- src/Parsers/ASTSetQuery.cpp | 10 +++---- src/Parsers/FieldFromAST.h | 26 ++++++++++++++++++ src/Parsers/ParserSetQuery.cpp | 18 ++++++------- src/Parsers/ParserSetQuery.h | 2 +- src/Parsers/SettingValueFromAST.h | 44 ------------------------------- 8 files changed, 57 insertions(+), 132 deletions(-) create mode 100644 src/Parsers/FieldFromAST.h delete mode 100644 src/Parsers/SettingValueFromAST.h diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 188947e3f45..9fb4f361e09 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -1,15 +1,12 @@ #include -#include -#include namespace DB { - namespace { SettingChange * find(SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) return nullptr; return &*it; @@ -17,7 +14,7 @@ namespace const SettingChange * find(const SettingsChanges & changes, std::string_view name) { - auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.getName() == name; }); + auto it = std::find_if(changes.begin(), changes.end(), [&name](const SettingChange & change) { return change.name == name; }); if (it == changes.end()) return nullptr; return &*it; @@ -29,7 +26,7 @@ bool SettingsChanges::tryGet(std::string_view name, Field & out_value) const const auto * change = find(*this, name); if (!change) return false; - out_value = change->getFieldValue(); + out_value = change->value; return true; } @@ -38,7 +35,7 @@ const Field * SettingsChanges::tryGet(std::string_view name) const const auto * change = find(*this, name); if (!change) return nullptr; - return &change->getFieldValue(); + return &change->value; } Field * SettingsChanges::tryGet(std::string_view name) @@ -46,29 +43,7 @@ Field * SettingsChanges::tryGet(std::string_view name) auto * change = find(*this, name); if (!change) return nullptr; - return &change->getFieldValue(); -} - -struct SettingValueFromField : SettingValue -{ - explicit SettingValueFromField(const Field & value_) : value(value_) {} - explicit SettingValueFromField(Field && value_) : value(std::move(value_)) {} - - const Field & getField() const override { return value; } - Field & getField() override { return value; } - std::string toString() const override { return applyVisitor(FieldVisitorToString(), value); } - - Field value; -}; - -SettingValuePtr getSettingValueFromField(const Field & field) -{ - return std::make_shared(field); -} - -SettingValuePtr getSettingValueFromField(Field && field) -{ - return std::make_shared(std::move(field)); + return &change->value; } } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 183d69b3112..776dacb93e8 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -1,54 +1,22 @@ #pragma once + #include namespace DB { -struct SettingValue +struct SettingChange { - virtual ~SettingValue() = default; - virtual const Field & getField() const = 0; - virtual Field & getField() = 0; - virtual std::string toString() const = 0; -}; -using SettingValuePtr = std::shared_ptr; - -SettingValuePtr getSettingValueFromField(const Field & field); -SettingValuePtr getSettingValueFromField(Field && field); - - -class SettingChange -{ -private: String name; - SettingValuePtr value; + Field value; -public: SettingChange() = default; + SettingChange(std::string_view name_, const Field & value_) : name(name_), value(value_) {} + SettingChange(std::string_view name_, Field && value_) : name(name_), value(std::move(value_)) {} - SettingChange(std::string_view name_, const Field & value_) : name(name_), value(getSettingValueFromField(value_)) {} - SettingChange(std::string_view name_, Field && value_) : name(name_), value(getSettingValueFromField(std::move(value_))) {} - SettingChange(std::string_view name_, SettingValuePtr && value_) : name(name_), value(std::move(value_)) {} - - friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) - { - return (lhs.name == rhs.name) && (lhs.value == rhs.value); - } - + friend bool operator ==(const SettingChange & lhs, const SettingChange & rhs) { return (lhs.name == rhs.name) && (lhs.value == rhs.value); } friend bool operator !=(const SettingChange & lhs, const SettingChange & rhs) { return !(lhs == rhs); } - - const String & getName() const { return name; } - String & getName() { return name; } - - SettingValuePtr getValue() const { return value; } - const Field & getFieldValue() const { return value->getField(); } - Field & getFieldValue() { return value->getField(); } - - void setValue(const Field & field) { value = getSettingValueFromField(field); } - void setValue(SettingValuePtr value_) { value = std::move(value_); } - - String getValueString() const { return value->toString(); } }; diff --git a/src/Core/Field.h b/src/Core/Field.h index bfb5cc1f324..beb460c258c 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -110,10 +110,6 @@ struct CustomType virtual const char * getTypeName() const = 0; virtual String toString() const = 0; - // virtual std::unique_ptr clone() const; - // virtual std::optional getBaseTypeIfConvertible() const; - // virtual std::optional tryConvertToBaseType(Types::Which dest_type) const - bool operator < (const CustomTypeImpl &) const { throwNotImpleneted("<"); } bool operator <= (const CustomTypeImpl &) const { throwNotImpleneted("<="); } bool operator > (const CustomTypeImpl &) const { throwNotImpleneted(">"); } @@ -127,7 +123,12 @@ struct CustomType } }; + CustomType() = default; + explicit CustomType(std::shared_ptr impl_) : impl(impl_) {} + + const char * getTypeName() const { return impl->getTypeName(); } String toString() const { return impl->toString(); } + const CustomTypeImpl & getImpl() { return *impl; } bool operator < (const CustomType & rhs) const { return *impl < *rhs.impl; } bool operator <= (const CustomType & rhs) const { return *impl <= *rhs.impl; } @@ -136,7 +137,6 @@ struct CustomType bool operator == (const CustomType & rhs) const { return *impl == *rhs.impl; } std::shared_ptr impl; - }; template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index db9209120f2..26420f4988c 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -13,9 +13,9 @@ void ASTSetQuery::updateTreeHashImpl(SipHash & hash_state) const { for (const auto & change : changes) { - hash_state.update(change.getName().size()); - hash_state.update(change.getName()); - applyVisitor(FieldVisitorHash(hash_state), change.getFieldValue()); + hash_state.update(change.name.size()); + hash_state.update(change.name); + applyVisitor(FieldVisitorHash(hash_state), change.value); } } @@ -33,8 +33,8 @@ void ASTSetQuery::formatImpl(const FormatSettings & format, FormatState &, Forma else first = false; - formatSettingName(change.getName(), format.ostr); - format.ostr << " = " << change.getValueString(); + formatSettingName(change.name, format.ostr); + format.ostr << " = " << applyVisitor(FieldVisitorToString(), change.value); } for (const auto & setting_name : default_settings) diff --git a/src/Parsers/FieldFromAST.h b/src/Parsers/FieldFromAST.h new file mode 100644 index 00000000000..10f15948601 --- /dev/null +++ b/src/Parsers/FieldFromAST.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +struct FieldFromASTImpl : public CustomType::CustomTypeImpl +{ + static constexpr auto name = "AST"; + + explicit FieldFromASTImpl(ASTPtr ast_) : ast(ast_) {} + + const char * getTypeName() const override { return name; } + String toString() const override { return serializeAST(*ast); } + + ASTPtr ast; +}; + +Field createFieldFromAST(ASTPtr ast) +{ + return CustomType(std::make_shared(ast)); +} + +} diff --git a/src/Parsers/ParserSetQuery.cpp b/src/Parsers/ParserSetQuery.cpp index ed2e21f15cf..4df74c2dd82 100644 --- a/src/Parsers/ParserSetQuery.cpp +++ b/src/Parsers/ParserSetQuery.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include @@ -217,16 +217,16 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p /// for SETTINGS disk=disk(type='s3', path='', ...) else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - tryGetIdentifierNameInto(name, change.getName()); - change.setValue(std::make_unique(function_ast)); + tryGetIdentifierNameInto(name, change.name); + change.value = createFieldFromAST(function_ast); return true; } else if (!literal_or_map_p.parse(pos, value, expected)) return false; - tryGetIdentifierNameInto(name, change.getName()); - change.setValue(value->as().value); + tryGetIdentifierNameInto(name, change.name); + change.value = value->as().value; return true; } @@ -282,16 +282,16 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault( node = std::make_shared(Field(static_cast(0))); else if (function_p.parse(pos, function_ast, expected) && function_ast->as()->name == "disk") { - change.getName() = name; - change.setValue(std::make_unique(function_ast)); + change.name = name; + change.value = createFieldFromAST(function_ast); return true; } else if (!value_p.parse(pos, node, expected)) return false; - change.getName() = name; - change.setValue(node->as().value); + change.name = name; + change.value = node->as().value; return true; } diff --git a/src/Parsers/ParserSetQuery.h b/src/Parsers/ParserSetQuery.h index dbf52832fdd..4d3803e78b4 100644 --- a/src/Parsers/ParserSetQuery.h +++ b/src/Parsers/ParserSetQuery.h @@ -7,7 +7,7 @@ namespace DB { -class SettingChange; +struct SettingChange; /** Query like this: * SET name1 = value1, name2 = value2, ... diff --git a/src/Parsers/SettingValueFromAST.h b/src/Parsers/SettingValueFromAST.h deleted file mode 100644 index 5c0ed4c248f..00000000000 --- a/src/Parsers/SettingValueFromAST.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_GET_SETTING_VALUE; -} - -struct SettingValueFromAST : SettingValue -{ - explicit SettingValueFromAST(const ASTPtr & value_) : value(value_) {} - - ASTPtr value; - std::optional field; - - [[noreturn]] void throwNoValue() const - { - throw Exception( - ErrorCodes::CANNOT_GET_SETTING_VALUE, - "Cannot get setting value, it must be converted from AST to Field first"); - } - - const Field & getField() const override - { - if (field) - return *field; - throwNoValue(); - } - - Field & getField() override - { - if (field) - return *field; - throwNoValue(); - } - void setField(const Field & field_) { field = field_; } - std::string toString() const override { return serializeAST(*value); } -}; - -} From 519bb173e8bae1414f22df6cfdb0f8363979186d Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 4 Feb 2023 12:47:17 +0000 Subject: [PATCH 035/116] Forbid wrong create view syntax --- src/Parsers/ParserCreateQuery.cpp | 5 ++--- .../0_stateless/02554_invalid_create_view_syntax.reference | 0 .../queries/0_stateless/02554_invalid_create_view_syntax.sql | 1 + 3 files changed, 3 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02554_invalid_create_view_syntax.reference create mode 100644 tests/queries/0_stateless/02554_invalid_create_view_syntax.sql diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 208737b5bda..052bb3863b4 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -1299,14 +1299,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - - if (ParserKeyword{"TO INNER UUID"}.ignore(pos, expected)) + if (!is_ordinary_view && ParserKeyword{"TO INNER UUID"}.ignore(pos, expected)) { ParserStringLiteral literal_p; if (!literal_p.parse(pos, to_inner_uuid, expected)) return false; } - else if (ParserKeyword{"TO"}.ignore(pos, expected)) + else if (!is_ordinary_view && ParserKeyword{"TO"}.ignore(pos, expected)) { // TO [db.]table if (!table_name_p.parse(pos, to_table, expected)) diff --git a/tests/queries/0_stateless/02554_invalid_create_view_syntax.reference b/tests/queries/0_stateless/02554_invalid_create_view_syntax.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02554_invalid_create_view_syntax.sql b/tests/queries/0_stateless/02554_invalid_create_view_syntax.sql new file mode 100644 index 00000000000..bf16d635312 --- /dev/null +++ b/tests/queries/0_stateless/02554_invalid_create_view_syntax.sql @@ -0,0 +1 @@ +CREATE VIEW X TO Y AS SELECT 1; -- { clientError 62 } From cce23cee5152281a0e0893afe839d8e19b4d8145 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 4 Feb 2023 13:49:32 +0100 Subject: [PATCH 036/116] Fix build --- src/Common/FieldVisitorDump.cpp | 10 ++++++++++ src/Common/FieldVisitorHash.cpp | 10 ++++++++++ src/Common/FieldVisitorSum.cpp | 1 + src/Common/FieldVisitorWriteBinary.cpp | 2 +- src/Core/Field.cpp | 10 ++++++++++ src/DataTypes/FieldToDataType.cpp | 1 + src/Parsers/FieldFromAST.cpp | 11 +++++++++++ src/Parsers/FieldFromAST.h | 5 +---- 8 files changed, 45 insertions(+), 5 deletions(-) create mode 100644 src/Parsers/FieldFromAST.cpp diff --git a/src/Common/FieldVisitorDump.cpp b/src/Common/FieldVisitorDump.cpp index be4331ca478..8061b9357ac 100644 --- a/src/Common/FieldVisitorDump.cpp +++ b/src/Common/FieldVisitorDump.cpp @@ -125,5 +125,15 @@ String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const return wb.str(); } +String FieldVisitorDump::operator() (const CustomType & x) const +{ + WriteBufferFromOwnString wb; + wb << "CustomType_("; + writeQuoted(std::string_view(x.getTypeName()), wb); + wb << ", "; + writeQuoted(x.toString(), wb); + wb << ')'; + return wb.str(); } +} diff --git a/src/Common/FieldVisitorHash.cpp b/src/Common/FieldVisitorHash.cpp index d759635c65b..f73c9f064e2 100644 --- a/src/Common/FieldVisitorHash.cpp +++ b/src/Common/FieldVisitorHash.cpp @@ -180,4 +180,14 @@ void FieldVisitorHash::operator() (const bool & x) const hash.update(x); } +void FieldVisitorHash::operator() (const CustomType & x) const +{ + UInt8 type = Field::Types::CustomType; + hash.update(type); + hash.update(x.getTypeName()); + auto result = x.toString(); + hash.update(result.size()); + hash.update(result.data(), result.size()); +} + } diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index d6c5374f9db..b825f188586 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -35,6 +35,7 @@ bool FieldVisitorSum::operator() (Object &) const { throw Exception(ErrorCodes:: bool FieldVisitorSum::operator() (UUID &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum UUIDs"); } bool FieldVisitorSum::operator() (IPv4 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum IPv4s"); } bool FieldVisitorSum::operator() (IPv6 &) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum IPv6s"); } +bool FieldVisitorSum::operator() (CustomType & x) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot sum custom type {}", x.getTypeName()); } bool FieldVisitorSum::operator() (AggregateFunctionStateData &) const { diff --git a/src/Common/FieldVisitorWriteBinary.cpp b/src/Common/FieldVisitorWriteBinary.cpp index c9b0cf16414..c3bd3f147c2 100644 --- a/src/Common/FieldVisitorWriteBinary.cpp +++ b/src/Common/FieldVisitorWriteBinary.cpp @@ -18,6 +18,7 @@ void FieldVisitorWriteBinary::operator() (const Int256 & x, WriteBuffer & buf) c void FieldVisitorWriteBinary::operator() (const UUID & x, WriteBuffer & buf) const { writeBinary(x, buf); } void FieldVisitorWriteBinary::operator() (const IPv4 & x, WriteBuffer & buf) const { writeBinary(x, buf); } void FieldVisitorWriteBinary::operator() (const IPv6 & x, WriteBuffer & buf) const { writeBinary(x, buf); } +void FieldVisitorWriteBinary::operator() (const CustomType & x, WriteBuffer & buf) const { writeBinary(x, buf); } void FieldVisitorWriteBinary::operator() (const DecimalField & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); } void FieldVisitorWriteBinary::operator() (const DecimalField & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); } void FieldVisitorWriteBinary::operator() (const DecimalField & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); } @@ -88,4 +89,3 @@ void FieldVisitorWriteBinary::operator()(const bool & x, WriteBuffer & buf) cons } } - diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 976f47cdbe3..c06e9aa244e 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -260,6 +260,16 @@ void writeText(const Object & x, WriteBuffer & buf) writeFieldText(Field(x), buf); } +void writeBinary(const CustomType &, WriteBuffer &) +{ + /// TODO: +} + +void writeText(const CustomType & x, WriteBuffer & buf) +{ + writeFieldText(Field(x), buf); +} + template void readQuoted(DecimalField & x, ReadBuffer & buf) { diff --git a/src/DataTypes/FieldToDataType.cpp b/src/DataTypes/FieldToDataType.cpp index 0a70761a64e..210dab9921e 100644 --- a/src/DataTypes/FieldToDataType.cpp +++ b/src/DataTypes/FieldToDataType.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int EMPTY_DATA_PASSED; + extern const int NOT_IMPLEMENTED; } template diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp new file mode 100644 index 00000000000..a7326bbfa69 --- /dev/null +++ b/src/Parsers/FieldFromAST.cpp @@ -0,0 +1,11 @@ +#include + +namespace DB +{ + +Field createFieldFromAST(ASTPtr ast) +{ + return CustomType(std::make_shared(ast)); +} + +} diff --git a/src/Parsers/FieldFromAST.h b/src/Parsers/FieldFromAST.h index 10f15948601..f8bc37a7f15 100644 --- a/src/Parsers/FieldFromAST.h +++ b/src/Parsers/FieldFromAST.h @@ -18,9 +18,6 @@ struct FieldFromASTImpl : public CustomType::CustomTypeImpl ASTPtr ast; }; -Field createFieldFromAST(ASTPtr ast) -{ - return CustomType(std::make_shared(ast)); -} +Field createFieldFromAST(ASTPtr ast); } From 1cbff88a7c52c8740e2421c79382b98a20903f9b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 4 Feb 2023 19:31:19 +0100 Subject: [PATCH 037/116] Fix integration test --- src/Interpreters/SystemLog.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.cpp | 29 +++++++++++++++++-- src/Storages/MergeTree/MergeTreeSettings.h | 3 +- .../MergeTree/registerStorageMergeTree.cpp | 27 ++--------------- 4 files changed, 33 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index f9343b7889d..d1a576e48f5 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -569,7 +569,7 @@ ASTPtr SystemLog::getCreateTableQuery() if (endsWith(engine.name, "MergeTree")) { auto storage_settings = std::make_unique(getContext()->getMergeTreeSettings()); - storage_settings->loadFromQuery(*create->storage); + storage_settings->loadFromQuery(*create->storage, getContext()); } return create; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 2bf717c883a..03892f8c67c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -3,7 +3,12 @@ #include #include #include +#include +#include #include +#include +#include +#include #include @@ -39,13 +44,33 @@ void MergeTreeSettings::loadFromConfig(const String & config_elem, const Poco::U } } -void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def) +void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr context) { if (storage_def.settings) { try { - applyChanges(storage_def.settings->changes); + auto changes = storage_def.settings->changes; + { + for (auto & [name, value] : changes) + { + CustomType custom; + if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) + { + auto ast = dynamic_cast(custom.getImpl()).ast; + if (ast && isDiskFunction(ast)) + { + const auto & ast_function = assert_cast(*ast); + auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); + LOG_TRACE(&Poco::Logger::get("MergeTreeSettings"), "Created custom disk {}", disk_name); + value = disk_name; + break; + } + } + } + } + + applyChanges(changes); } catch (Exception & e) { diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 35a806a917c..377fce82fe0 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -192,7 +193,7 @@ struct MergeTreeSettings : public BaseSettings void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); /// NOTE: will rewrite the AST to add immutable settings. - void loadFromQuery(ASTStorage & storage_def); + void loadFromQuery(ASTStorage & storage_def, ContextPtr context); /// We check settings after storage creation static bool isReadonlySetting(const String & name) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index e5aa2607ee0..6d597eaee09 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -17,7 +16,6 @@ #include #include #include -#include #include #include @@ -590,37 +588,18 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.column_ttls_by_name[name] = new_ttl_entry; } - if (args.storage_def->settings) - { - for (auto & [name, value] : args.storage_def->settings->changes) - { - CustomType custom; - if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) - { - auto ast = dynamic_cast(custom.getImpl()).ast; - if (ast && isDiskFunction(ast)) - { - const auto & ast_function = assert_cast(*ast); - auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); - value = disk_name; - break; - } - } - } - } - - storage_settings->loadFromQuery(*args.storage_def); - if (storage_settings->disk.changed && storage_settings->storage_policy.changed) throw Exception( ErrorCodes::BAD_ARGUMENTS, "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); + storage_settings->loadFromQuery(*args.storage_def, context); + // updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) { if (!args.attach) - args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, args.storage_def->settings->changes); + args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, storage_settings->changes()); metadata.settings_changes = args.storage_def->settings->ptr(); } } From 5abbc7cad61f18e442e69d48d5c082a74c0f6009 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 4 Feb 2023 19:41:41 +0100 Subject: [PATCH 038/116] Cleanup --- src/Access/SettingsConstraints.h | 2 +- src/Backups/RestoreSettings.cpp | 1 + src/Common/ErrorCodes.cpp | 1 - .../NamedCollectionConfiguration.cpp | 4 +-- .../NamedCollections/NamedCollectionUtils.cpp | 18 +++++----- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 3 +- .../getDictionaryConfigurationFromAST.cpp | 12 +++---- src/Interpreters/Cache/QueryCache.cpp | 7 ++-- src/Interpreters/Context.cpp | 4 ++- src/Storages/MergeTree/MergeTreeSettings.cpp | 3 +- src/Storages/StorageJoin.cpp | 36 +++++++++---------- 12 files changed, 45 insertions(+), 48 deletions(-) diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 9f66f591835..2e9f12d0b9c 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,8 +12,8 @@ namespace Poco::Util namespace DB { struct Settings; -struct SettingChange; struct MergeTreeSettings; +struct SettingChange; class SettingsChanges; class AccessControl; diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 43fc0e4dce0..5e06764a247 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -180,6 +180,7 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query) if (setting.name == #NAME) \ res.NAME = SettingField##TYPE{setting.value}.value; \ else + LIST_OF_RESTORE_SETTINGS(GET_SETTINGS_FROM_RESTORE_QUERY_HELPER) throw Exception(ErrorCodes::CANNOT_PARSE_BACKUP_SETTINGS, "Unknown setting {}", setting.name); } diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 83d09dce432..028663a2176 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -648,7 +648,6 @@ M(677, THREAD_WAS_CANCELED) \ M(678, IO_URING_INIT_FAILED) \ M(679, IO_URING_SUBMIT_ERROR) \ - M(680, CANNOT_GET_SETTING_VALUE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp index d593cad5a21..1c42b001ceb 100644 --- a/src/Common/NamedCollections/NamedCollectionConfiguration.cpp +++ b/src/Common/NamedCollections/NamedCollectionConfiguration.cpp @@ -150,8 +150,8 @@ ConfigurationPtr createConfiguration(const std::string & root_name, const Settin namespace Configuration = NamedCollectionConfiguration; auto config = Configuration::createEmptyConfiguration(root_name); - for (const auto & setting : settings) - Configuration::setConfigValue(*config, setting.name, convertFieldToString(setting.value)); + for (const auto & [name, value] : settings) + Configuration::setConfigValue(*config, name, convertFieldToString(value)); return config; } diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 8c84352dbc5..8beaa38210e 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -180,20 +180,20 @@ public: auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings()); std::unordered_map result_changes_map; - for (const auto & change : query.changes) + for (const auto & [name, value] : query.changes) { - auto [it, inserted] = result_changes_map.emplace(change.name, change.value); + auto [it, inserted] = result_changes_map.emplace(name, value); if (!inserted) { throw Exception( ErrorCodes::BAD_ARGUMENTS, "Value with key `{}` is used twice in the SET query (collection name: {})", - change.name, query.collection_name); + name, query.collection_name); } } - for (const auto & change : create_query.changes) - result_changes_map.emplace(change.name, change.value); + for (const auto & [name, value] : create_query.changes) + result_changes_map.emplace(name, value); for (const auto & delete_key : query.delete_keys) { @@ -253,8 +253,8 @@ private: collection_name, query.changes); std::set> keys; - for (const auto & change : query.changes) - keys.insert(change.name); + for (const auto & [name, _] : query.changes) + keys.insert(name); return NamedCollection::create( *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); @@ -378,8 +378,8 @@ void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr contex auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name); auto collection_lock = collection->lock(); - for (const auto & change : query.changes) - collection->setOrUpdate(change.name, convertFieldToString(change.value)); + for (const auto & [name, value] : query.changes) + collection->setOrUpdate(name, convertFieldToString(value)); for (const auto & key : query.delete_keys) collection->remove(key); diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 3f4c2d8e466..50c4c7e0613 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -42,7 +42,7 @@ void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) SettingsChanges & changes = storage_def.settings->changes; #define ADD_IF_ABSENT(NAME) \ if (std::find_if(changes.begin(), changes.end(), \ - [](const SettingChange & c) { return c.name == #NAME; }) \ + [](const SettingChange & c) { return c.name == #NAME; }) \ == changes.end()) \ changes.push_back(SettingChange{#NAME, static_cast(NAME)}); diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 5cfc3535a0f..04807bb3daf 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -149,8 +149,7 @@ void DatabaseMaterializedPostgreSQL::applySettingsChanges(const SettingsChanges need_update_on_disk = true; } - else if ((change.name == "materialized_postgresql_allow_automatic_update") - || (change.name == "materialized_postgresql_max_block_size")) + else if ((change.name == "materialized_postgresql_allow_automatic_update") || (change.name == "materialized_postgresql_max_block_size")) { replication_handler->setSetting(change); need_update_on_disk = true; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 8ef63a55d43..0de8b843604 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -125,11 +125,11 @@ void buildLayoutConfiguration( { AutoPtr settings_element(doc->createElement("settings")); root->appendChild(settings_element); - for (const auto & change : settings->changes) + for (const auto & [name, value] : settings->changes) { - AutoPtr setting_change_element(doc->createElement(change.name)); + AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.value))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); setting_change_element->appendChild(setting_value); } } @@ -529,11 +529,11 @@ void buildSourceConfiguration( { AutoPtr settings_element(doc->createElement("settings")); outer_element->appendChild(settings_element); - for (const auto & change : settings->changes) + for (const auto & [name, value] : settings->changes) { - AutoPtr setting_change_element(doc->createElement(change.name)); + AutoPtr setting_change_element(doc->createElement(name)); settings_element->appendChild(setting_change_element); - AutoPtr setting_value(doc->createTextNode(convertFieldToString(change.value))); + AutoPtr setting_value(doc->createTextNode(convertFieldToString(value))); setting_change_element->appendChild(setting_value); } } diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index ea20e323459..06fd0689993 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -79,10 +79,9 @@ public: auto is_query_cache_related_setting = [](const auto & change) { - const auto & name = change.name; - return name == "allow_experimental_query_cache" - || name.starts_with("query_cache") - || name.ends_with("query_cache"); + return change.name == "allow_experimental_query_cache" + || change.name.starts_with("query_cache") + || change.name.ends_with("query_cache"); }; std::erase_if(set_clause->changes, is_query_cache_related_setting); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0ef49fec556..ff0b5db8f2f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1517,7 +1517,9 @@ void Context::applySettingChange(const SettingChange & change) } catch (Exception & e) { - e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}", change.name, toString(change.value))); + e.addMessage(fmt::format( + "in attempt to set the value of setting '{}' to {}", + change.name, applyVisitor(FieldVisitorToString(), change.value))); throw; } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 03892f8c67c..00ce3c2efd9 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -1,13 +1,12 @@ #include #include +#include #include #include #include #include #include #include -#include -#include #include #include diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 0be7097ad2c..b57e717c272 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -305,28 +305,26 @@ void registerStorageJoin(StorageFactory & factory) { for (const auto & setting : args.storage_def->settings->changes) { - const auto & setting_name = setting.name; - const auto & setting_value = setting.value; - if (setting_name == "join_use_nulls") - join_use_nulls = setting_value; - else if (setting_name == "max_rows_in_join") - max_rows_in_join = setting_value; - else if (setting_name == "max_bytes_in_join") - max_bytes_in_join = setting_value; - else if (setting_name == "join_overflow_mode") - join_overflow_mode = setting_value; - else if (setting_name == "join_any_take_last_row") - join_any_take_last_row = setting_value; - else if (setting_name == "any_join_distinct_right_table_keys") - old_any_join = setting_value; - else if (setting_name == "disk") - disk_name = setting_value.get(); - else if (setting_name == "persistent") + if (setting.name == "join_use_nulls") + join_use_nulls = setting.value; + else if (setting.name == "max_rows_in_join") + max_rows_in_join = setting.value; + else if (setting.name == "max_bytes_in_join") + max_bytes_in_join = setting.value; + else if (setting.name == "join_overflow_mode") + join_overflow_mode = setting.value; + else if (setting.name == "join_any_take_last_row") + join_any_take_last_row = setting.value; + else if (setting.name == "any_join_distinct_right_table_keys") + old_any_join = setting.value; + else if (setting.name == "disk") + disk_name = setting.value.get(); + else if (setting.name == "persistent") { - persistent = setting_value.get(); + persistent = setting.value.get(); } else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown setting {} for storage {}", setting_name, args.engine_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown setting {} for storage {}", setting.name, args.engine_name); } } From e1f345cc6b91b53553b5695a81557470bc85dd10 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Sat, 4 Feb 2023 11:16:18 +0000 Subject: [PATCH 039/116] Improve behaviour of conversion into Date for boundary value 65535 - with fixed tests --- src/Functions/FunctionsConversion.h | 4 +- ...lumn_consistent_insert_behaviour.reference | 36 +++++++ ...ate_column_consistent_insert_behaviour.sql | 99 +++++++++++++++++++ 3 files changed, 137 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference create mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c34cd3ac875..55003044ff5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -377,7 +377,7 @@ struct ToDateTransform32Or64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // since converting to Date, no need in values outside of default LUT range. - return (from < DATE_LUT_MAX_DAY_NUM) + return (from <= DATE_LUT_MAX_DAY_NUM) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } @@ -394,7 +394,7 @@ struct ToDateTransform32Or64Signed /// The function should be monotonic (better for query optimizations), so we saturate instead of overflow. if (from < 0) return 0; - return (from < DATE_LUT_MAX_DAY_NUM) + return (from <= DATE_LUT_MAX_DAY_NUM) ? static_cast(from) : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference new file mode 100644 index 00000000000..5059130c000 --- /dev/null +++ b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference @@ -0,0 +1,36 @@ +2149-06-06 65535 +2149-06-06 toUInt16(65535) +2149-06-06 toInt32(65535) +2149-06-06 toUInt32(65535) +2149-06-06 toDate(65535) +2149-06-06 CAST(65535 as UInt16) +2149-06-06 CAST(65535 as Int32) +2149-06-06 CAST(65535 as UInt32) +2149-06-06 CAST(65535 as Date) +2149-06-05 65534 +2149-06-05 toUInt16(65534) +2149-06-05 toInt32(65534) +2149-06-05 toUInt32(65534) +2149-06-05 toDate(65534) +2149-06-05 CAST(65534 as UInt16) +2149-06-05 CAST(65534 as Int32) +2149-06-05 CAST(65534 as UInt32) +2149-06-05 CAST(65534 as Date) +1970-01-01 0 +1970-01-01 toUInt16(0) +1970-01-01 toInt32(0) +1970-01-01 toUInt32(0) +1970-01-01 toDate(0) +1970-01-01 CAST(0 as UInt16) +1970-01-01 CAST(0 as Int32) +1970-01-01 CAST(0 as UInt32) +1970-01-01 CAST(0 as Date) +1 65536 +1 toInt32(65536) +1 toUInt32(65536) +1 toDate(65536) +1 CAST(65536 as Int32) +1 CAST(65536 as UInt32) +1 CAST(65536 as Date) +1970-01-01 toUInt16(65536) +1970-01-01 CAST(65536 as UInt16) diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql new file mode 100644 index 00000000000..8a5c88fcae4 --- /dev/null +++ b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql @@ -0,0 +1,99 @@ +DROP TABLE IF EXISTS 02540_date; +CREATE TABLE 02540_date (txt String, x Date) engine=Memory; + +-- Date: Supported range of values: [1970-01-01, 2149-06-06]. +-- ^----closed interval---^ + +INSERT INTO 02540_date VALUES('65535', 65535); +INSERT INTO 02540_date VALUES('toUInt16(65535)', toUInt16(65535)); -- #43370 weird one -> used to be 1970-01-01 +INSERT INTO 02540_date VALUES('toInt32(65535)', toInt32(65535)); +INSERT INTO 02540_date VALUES('toUInt32(65535)', toUInt32(65535)); +INSERT INTO 02540_date VALUES('toDate(65535)', toDate(65535)); + +INSERT INTO 02540_date VALUES('CAST(65535 as UInt16)', CAST(65535 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(65535 as Int32)', CAST(65535 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65535 as UInt32)', CAST(65535 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65535 as Date)', CAST(65535 as Date)); + +INSERT INTO 02540_date VALUES('65534', 65534); +INSERT INTO 02540_date VALUES('toUInt16(65534)', toUInt16(65534)); +INSERT INTO 02540_date VALUES('toInt32(65534)', toInt32(65534)); +INSERT INTO 02540_date VALUES('toUInt32(65534)', toUInt32(65534)); +INSERT INTO 02540_date VALUES('toDate(65534)', toDate(65534)); + +INSERT INTO 02540_date VALUES('CAST(65534 as UInt16)', CAST(65534 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(65534 as Int32)', CAST(65534 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65534 as UInt32)', CAST(65534 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65534 as Date)', CAST(65534 as Date)); + +INSERT INTO 02540_date VALUES('0', 0); +INSERT INTO 02540_date VALUES('toUInt16(0)', toUInt16(0)); +INSERT INTO 02540_date VALUES('toInt32(0)', toInt32(0)); +INSERT INTO 02540_date VALUES('toUInt32(0)', toUInt32(0)); +INSERT INTO 02540_date VALUES('toDate(0)', toDate(0)); + +INSERT INTO 02540_date VALUES('CAST(0 as UInt16)', CAST(0 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(0 as Int32)', CAST(0 as Int32)); +INSERT INTO 02540_date VALUES('CAST(0 as UInt32)', CAST(0 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(0 as Date)', CAST(0 as Date)); + + +-- 65536 will be done using the TZ settings (comments in #45914) +-- We can expect either 1970-01-01 or 1970-01-02 +-- time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))) +INSERT INTO 02540_date VALUES('65536', 65536); +INSERT INTO 02540_date VALUES('toUInt16(65536)', toUInt16(65536)); -- Narrowing conversion 65536 ==> 0 +INSERT INTO 02540_date VALUES('toInt32(65536)', toInt32(65536)); +INSERT INTO 02540_date VALUES('toUInt32(65536)', toUInt32(65536)); +INSERT INTO 02540_date VALUES('toDate(65536)', toDate(65536)); + +INSERT INTO 02540_date VALUES('CAST(65536 as UInt16)', CAST(65536 as UInt16)); -- Narrowing conversion 65536 ==> 0 +INSERT INTO 02540_date VALUES('CAST(65536 as Int32)', CAST(65536 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65536 as UInt32)', CAST(65536 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65536 as Date)', CAST(65536 as Date)); + + +SELECT x, txt FROM 02540_date WHERE txt == '65535'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65535)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == '65534'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65534)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == '0'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(0)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as Date)'; + +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == '65536'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toInt32(65536)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toUInt32(65536)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toDate(65536)'; + +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as Int32)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt32)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65536)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt16)'; From a7eb1b1419cd9e77ca874d5ceaec4e4a306cfb5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 4 Feb 2023 22:57:30 +0100 Subject: [PATCH 040/116] Fix integration test --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6d597eaee09..d79f59bbc03 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -588,13 +588,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.column_ttls_by_name[name] = new_ttl_entry; } + storage_settings->loadFromQuery(*args.storage_def, context); + if (storage_settings->disk.changed && storage_settings->storage_policy.changed) throw Exception( ErrorCodes::BAD_ARGUMENTS, "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); - storage_settings->loadFromQuery(*args.storage_def, context); - // updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) { From 6cea67d58b54a93571731ea41fe2b3a5f6fc56bd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 Feb 2023 17:20:29 +0100 Subject: [PATCH 041/116] Cleanup --- src/Core/Field.cpp | 5 +++-- src/Core/Field.h | 16 +++++--------- src/Parsers/FieldFromAST.cpp | 9 ++++++++ src/Parsers/FieldFromAST.h | 8 +++++++ src/Storages/MergeTree/MergeTreeData.cpp | 1 - src/Storages/MergeTree/MergeTreePartition.cpp | 1 + src/Storages/MergeTree/MergeTreeSettings.cpp | 22 +++++++++---------- 7 files changed, 36 insertions(+), 26 deletions(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index c06e9aa244e..90dcb2e5f8e 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -260,9 +260,10 @@ void writeText(const Object & x, WriteBuffer & buf) writeFieldText(Field(x), buf); } -void writeBinary(const CustomType &, WriteBuffer &) +void writeBinary(const CustomType & x, WriteBuffer & buf) { - /// TODO: + writeBinary(std::string_view(x.getTypeName()), buf); + writeBinary(x.toString(), buf); } void writeText(const CustomType & x, WriteBuffer & buf) diff --git a/src/Core/Field.h b/src/Core/Field.h index beb460c258c..6a4581e8944 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -110,17 +110,11 @@ struct CustomType virtual const char * getTypeName() const = 0; virtual String toString() const = 0; - bool operator < (const CustomTypeImpl &) const { throwNotImpleneted("<"); } - bool operator <= (const CustomTypeImpl &) const { throwNotImpleneted("<="); } - bool operator > (const CustomTypeImpl &) const { throwNotImpleneted(">"); } - bool operator >= (const CustomTypeImpl &) const { throwNotImpleneted(">="); } - bool operator == (const CustomTypeImpl &) const { throwNotImpleneted("=="); } - - [[noreturn]] void throwNotImpleneted(const std::string & op) const - { - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Operation {} is not implemented for custom type {}", op, getTypeName()); - } + virtual bool operator < (const CustomTypeImpl &) const = 0; + virtual bool operator <= (const CustomTypeImpl &) const = 0; + virtual bool operator > (const CustomTypeImpl &) const = 0; + virtual bool operator >= (const CustomTypeImpl &) const = 0; + virtual bool operator == (const CustomTypeImpl &) const = 0; }; CustomType() = default; diff --git a/src/Parsers/FieldFromAST.cpp b/src/Parsers/FieldFromAST.cpp index a7326bbfa69..7b7302696ed 100644 --- a/src/Parsers/FieldFromAST.cpp +++ b/src/Parsers/FieldFromAST.cpp @@ -2,10 +2,19 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} Field createFieldFromAST(ASTPtr ast) { return CustomType(std::make_shared(ast)); } +[[noreturn]] void FieldFromASTImpl::throwNotImplemented(std::string_view method) const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Method {} not implemented for {}", method, getTypeName()); +} + } diff --git a/src/Parsers/FieldFromAST.h b/src/Parsers/FieldFromAST.h index f8bc37a7f15..132f7e3e705 100644 --- a/src/Parsers/FieldFromAST.h +++ b/src/Parsers/FieldFromAST.h @@ -15,6 +15,14 @@ struct FieldFromASTImpl : public CustomType::CustomTypeImpl const char * getTypeName() const override { return name; } String toString() const override { return serializeAST(*ast); } + [[noreturn]] void throwNotImplemented(std::string_view method) const; + + bool operator < (const CustomTypeImpl &) const override { throwNotImplemented("<"); } + bool operator <= (const CustomTypeImpl &) const override { throwNotImplemented("<="); } + bool operator > (const CustomTypeImpl &) const override { throwNotImplemented(">"); } + bool operator >= (const CustomTypeImpl &) const override { throwNotImplemented(">="); } + bool operator == (const CustomTypeImpl &) const override { throwNotImplemented("=="); } + ASTPtr ast; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a5f42f93f0f..1ea640ed2ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -80,7 +80,6 @@ #include #include #include -#include #include diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index d6c432ad4bf..3a6908ef32d 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -185,6 +185,7 @@ namespace { UInt8 type = Field::Types::CustomType; hash.update(type); + hash.update(x.getTypeName()); auto result = x.toString(); hash.update(result.size()); hash.update(result.data(), result.size()); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 00ce3c2efd9..059b64875a8 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -50,21 +50,19 @@ void MergeTreeSettings::loadFromQuery(ASTStorage & storage_def, ContextPtr conte try { auto changes = storage_def.settings->changes; + for (auto & [name, value] : changes) { - for (auto & [name, value] : changes) + CustomType custom; + if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) { - CustomType custom; - if (value.tryGet(custom) && 0 == strcmp(custom.getTypeName(), "AST")) + auto ast = dynamic_cast(custom.getImpl()).ast; + if (ast && isDiskFunction(ast)) { - auto ast = dynamic_cast(custom.getImpl()).ast; - if (ast && isDiskFunction(ast)) - { - const auto & ast_function = assert_cast(*ast); - auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); - LOG_TRACE(&Poco::Logger::get("MergeTreeSettings"), "Created custom disk {}", disk_name); - value = disk_name; - break; - } + const auto & ast_function = assert_cast(*ast); + auto disk_name = getOrCreateDiskFromDiskAST(ast_function, context); + LOG_TRACE(&Poco::Logger::get("MergeTreeSettings"), "Created custom disk {}", disk_name); + value = disk_name; + break; } } } From f1b8d1d9d722d09a279b36f2b91b7887c025e7e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 Feb 2023 00:27:12 +0100 Subject: [PATCH 042/116] Improve performance of Decimal conversion when scale does not change --- src/DataTypes/DataTypesDecimal.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 88b3a33a5b8..7a49238b5be 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -100,7 +100,7 @@ inline UInt32 getDecimalScale(const DataTypeDecimal & data_type) template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result) +inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) { using FromFieldType = typename FromDataType::FieldType; using ToFieldType = typename ToDataType::FieldType; @@ -121,8 +121,14 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v return ReturnType(false); } } + else if (scale_to == scale_from) + { + converted_value = value.value; + } else + { converted_value = value.value / DecimalUtils::scaleMultiplier(scale_from - scale_to); + } if constexpr (sizeof(FromFieldType) > sizeof(ToFieldType)) { @@ -155,7 +161,7 @@ inline typename ToDataType::FieldType convertDecimals(const typename FromDataTyp template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result) +inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) { return convertDecimalsImpl(value, scale_from, scale_to, result); } From 001b714d3eb09757843e42b119aaf40453b3a028 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 11:50:46 +0100 Subject: [PATCH 043/116] Common: SipHash: add original 128-bit output implementation --- src/Common/SipHash.h | 65 +++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 61 insertions(+), 4 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 1d602a3b191..404f4390aec 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -13,13 +13,21 @@ * (~ 700 MB/sec, 15 million strings per second) */ -#include -#include #include #include #include #include +#include +#include +#include +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} +} #define ROTL(x, b) static_cast(((x) << (b)) | ((x) >> (64 - (b)))) @@ -52,6 +60,9 @@ private: /// How many bytes have been processed. UInt64 cnt; + /// Whether it should use the reference algo for 128-bit or CH's version + bool is_reference_128; + /// The current 8 bytes of input data. union { @@ -69,7 +80,10 @@ private: SIPROUND; v0 ^= current_word; - v2 ^= 0xff; + if (is_reference_128) + v2 ^= 0xee; + else + v2 ^= 0xff; SIPROUND; SIPROUND; SIPROUND; @@ -78,13 +92,17 @@ private: public: /// Arguments - seed. - SipHash(UInt64 key0 = 0, UInt64 key1 = 0) /// NOLINT + SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT { /// Initialize the state with some random bytes and seed. v0 = 0x736f6d6570736575ULL ^ key0; v1 = 0x646f72616e646f6dULL ^ key1; v2 = 0x6c7967656e657261ULL ^ key0; v3 = 0x7465646279746573ULL ^ key1; + is_reference_128 = is_reference_128_; + + if (is_reference_128) + v1 ^= 0xee; cnt = 0; current_word = 0; @@ -201,6 +219,33 @@ public: get128(res); return res; } + + UInt128 get128Reference() + { + if (!is_reference_128) + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, "Logical error: can't call get128Reference when is_reference_128 is not set"); + finalize(); + auto lo = v0 ^ v1 ^ v2 ^ v3; + v1 ^= 0xdd; + SIPROUND; + SIPROUND; + SIPROUND; + SIPROUND; + auto hi = v0 ^ v1 ^ v2 ^ v3; + if constexpr (std::endian::native == std::endian::big) + { + lo = __builtin_bswap64(lo); + hi = __builtin_bswap64(hi); + auto tmp = hi; + hi = lo; + lo = tmp; + } + UInt128 res = hi; + res <<= 64; + res |= lo; + return res; + } }; @@ -228,6 +273,18 @@ inline UInt128 sipHash128(const char * data, const size_t size) return sipHash128Keyed(0, 0, data, size); } +inline UInt128 sipHash128ReferenceKeyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) +{ + SipHash hash(key0, key1, true); + hash.update(data, size); + return hash.get128Reference(); +} + +inline UInt128 sipHash128Reference(const char * data, const size_t size) +{ + return sipHash128ReferenceKeyed(0, 0, data, size); +} + inline UInt64 sipHash64Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) { SipHash hash(key0, key1); From 476342602b49dd8e31e33cf4aa119f8a4c87dd29 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 16:08:23 +0100 Subject: [PATCH 044/116] Functions: Hashing: add sipHash128Reference{,Keyed} --- src/Functions/FunctionsHashing.h | 37 ++++++++++++++++++++++++++ src/Functions/FunctionsHashingMisc.cpp | 2 ++ 2 files changed, 39 insertions(+) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 031d6e3b586..69c3a299eea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -363,6 +363,41 @@ struct SipHash128KeyedImpl static constexpr bool use_int_hash_for_pods = false; }; +struct SipHash128ReferenceImpl +{ + static constexpr auto name = "sipHash128Reference"; + + using ReturnType = UInt128; + + static UInt128 combineHashes(UInt128 h1, UInt128 h2) { return combineHashesFunc(h1, h2); } + + static UInt128 apply(const char * data, const size_t size) { return sipHash128Reference(data, size); } + + static constexpr bool use_int_hash_for_pods = false; +}; + +struct SipHash128ReferenceKeyedImpl +{ + static constexpr auto name = "sipHash128ReferenceKeyed"; + using ReturnType = UInt128; + using Key = impl::SipHashKey; + + static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } + + static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) + { + return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); + } + + static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) + { + UInt128 hashes[] = {h1, h2}; + return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); + } + + static constexpr bool use_int_hash_for_pods = false; +}; + /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. * Usually there is no reason to use MurmurHash. @@ -1624,6 +1659,8 @@ using FunctionSHA512 = FunctionStringHashFixedString; #endif using FunctionSipHash128 = FunctionAnyHash; using FunctionSipHash128Keyed = FunctionAnyHash; +using FunctionSipHash128Reference = FunctionAnyHash; +using FunctionSipHash128ReferenceKeyed = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index 2a705e87a1e..f544960b7b4 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -15,6 +15,8 @@ REGISTER_FUNCTION(Hashing) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); From 970a4586a18ff2d2cad36dcbab213c0cff674c67 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 15:13:57 +0100 Subject: [PATCH 045/116] docs: functions: hash: add warning about sipHash128{,Keyed} sipHash128{,Keyed} does not comply with the reference implementation of SipHash and it's weaker then it should --- docs/en/sql-reference/functions/hash-functions.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index ae6cdb7052d..9ceb8e7920d 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -125,6 +125,12 @@ SELECT sipHash64Keyed((506097522914230528, 1084818905618843912), array('e','x',' Like [sipHash64](#hash_functions-siphash64) but produces a 128-bit hash value, i.e. the final xor-folding state is done up to 128 bits. +:::warning +This 128-bit variant differs from the reference implementation and it's weaker. +This version exists because, when it was written, there was no official 128-bit extension for SipHash. +New projects should probably use [sipHash128Reference](#hash_functions-siphash128reference). +::: + **Syntax** ```sql @@ -159,6 +165,12 @@ Result: Same as [sipHash128](#hash_functions-siphash128) but additionally takes an explicit key argument instead of using a fixed key. +:::warning +This 128-bit variant differs from the reference implementation and it's weaker. +This version exists because, when it was written, there was no official 128-bit extension for SipHash. +New projects should probably use [sipHash128ReferenceKeyed](#hash_functions-siphash128referencekeyed). +::: + **Syntax** ```sql From 14acaa04a90f951fb7b9592edc615b93c5d7dbfa Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 15:14:15 +0100 Subject: [PATCH 046/116] docs: functions: hash: fix wrong doc --- docs/en/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 9ceb8e7920d..8a56148f7dc 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -183,7 +183,7 @@ Same as [sipHash128](#hash_functions-siphash128), but the first argument is a tu **Returned value** -A [UInt64](/docs/en/sql-reference/data-types/int-uint.md) data type hash value. +A 128-bit `SipHash` hash value of type [FixedString(16)](/docs/en/sql-reference/data-types/fixedstring.md). **Example** From aa4faae85d06f80e7e54ad33cadd1ca017bf620b Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 15:14:44 +0100 Subject: [PATCH 047/116] docs: functions: hash: add doc for sipHash126Reference{,Keyed} --- .../sql-reference/functions/hash-functions.md | 68 +++++++++++++++++++ ...new_functions_must_be_documented.reference | 2 + 2 files changed, 70 insertions(+) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 8a56148f7dc..69dc73e2fb0 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -201,6 +201,74 @@ Result: └───────────────────────────────────────────────────────────────────────────────┘ ``` +## sipHash128Reference + +Like [sipHash128](#hash_functions-siphash128) but implements the 128-bit algorithm from the original authors of SipHash. + +**Syntax** + +```sql +sipHash128Reference(par1,...) +``` + +**Arguments** + +Same as for [sipHash128](#hash_functions-siphash128). + +**Returned value** + +A 128-bit `SipHash` hash value of type [FixedString(16)](/docs/en/sql-reference/data-types/fixedstring.md). + +**Example** + +Query: + +```sql +SELECT hex(sipHash128Reference('foo', '\x01', 3)); +``` + +Result: + +```response +┌─hex(sipHash128Reference('foo', '', 3))─┐ +│ 4D1BE1A22D7F5933C0873E1698426260 │ +└────────────────────────────────────────┘ +``` + +## sipHash128ReferenceKeyed + +Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key. + +**Syntax** + +```sql +sipHash128ReferenceKeyed((k0, k1), par1,...) +``` + +**Arguments** + +Same as [sipHash128Reference](#hash_functions-siphash128reference), but the first argument is a tuple of two UInt64 values representing the key. + +**Returned value** + +A 128-bit `SipHash` hash value of type [FixedString(16)](/docs/en/sql-reference/data-types/fixedstring.md). + +**Example** + +Query: + +```sql +SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\x01', 3)); +``` + +Result: + +```response +┌─hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912), 'foo', '', 3))─┐ +│ 630133C9722DC08646156B8130C4CDC8 │ +└────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## cityHash64 Produces a 64-bit [CityHash](https://github.com/google/cityhash) hash value. 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 e41249af54c..dc106dc5bd9 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 @@ -647,6 +647,8 @@ sin sinh sipHash128 sipHash128Keyed +sipHash128Reference +sipHash128ReferenceKeyed sipHash64 sipHash64Keyed sleep From b627e6d8271e2cbaa116ffe4ccbf19f2e8ba2d3a Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Fri, 3 Feb 2023 15:31:29 +0100 Subject: [PATCH 048/116] tests: add tests for sipHash128Reference{,Keyed} --- .../02552_siphash128_reference.reference | 130 +++++++++++ .../02552_siphash128_reference.sql | 207 ++++++++++++++++++ 2 files changed, 337 insertions(+) create mode 100644 tests/queries/0_stateless/02552_siphash128_reference.reference create mode 100644 tests/queries/0_stateless/02552_siphash128_reference.sql diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference new file mode 100644 index 00000000000..a831c691ce7 --- /dev/null +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -0,0 +1,130 @@ +A3817F04BA25A8E66DF67214C7550293 +DA87C1D86B99AF44347659119B22FC45 +8177228DA4A45DC7FCA38BDEF60AFFE4 +9C70B60C5267A94E5F33B6B02985ED51 +F88164C12D9C8FAF7D0F6E7C7BCD5579 +1368875980776F8854527A07690E9627 +14EECA338B208613485EA0308FD7A15E +A1F1EBBED8DBC153C0B84AA61FF08239 +3B62A9BA6258F5610F83E264F31497B4 +264499060AD9BAABC47F8B02BB6D71ED +00110DC378146956C95447D3F3D0FBBA +0151C568386B6677A2B4DC6F81E5DC18 +D626B266905EF35882634DF68532C125 +9869E247E9C08B10D029934FC4B952F7 +31FCEFAC66D7DE9C7EC7485FE4494902 +5493E99933B0A8117E08EC0F97CFC3D9 +6EE2A4CA67B054BBFD3315BF85230577 +473D06E8738DB89854C066C47AE47740 +A426E5E423BF4885294DA481FEAEF723 +78017731CF65FAB074D5208952512EB1 +9E25FC833F2290733E9344A5E83839EB +568E495ABE525A218A2214CD3E071D12 +4A29B54552D16B9A469C10528EFF0AAE +C9D184DDD5A9F5E0CF8CE29A9ABF691C +2DB479AE78BD50D8882A8A178A6132AD +8ECE5F042D5E447B5051B9EACB8D8F6F +9C0B53B4B3C307E87EAEE08678141F66 +ABF248AF69A6EAE4BFD3EB2F129EEB94 +0664DA1668574B88B935F3027358AEF4 +AA4B9DC4BF337DE90CD4FD3C467C6AB7 +EA5C7F471FAF6BDE2B1AD7D4686D2287 +2939B0183223FAFC1723DE4F52C43D35 +7C3956CA5EEAFC3E363E9D556546EB68 +77C6077146F01C32B6B69D5F4EA9FFCF +37A6986CB8847EDF0925F0F1309B54DE +A705F0E69DA9A8F907241A2E923C8CC8 +3DC47D1F29C448461E9E76ED904F6711 +0D62BF01E6FC0E1A0D3C4751C5D3692B +8C03468BCA7C669EE4FD5E084BBEE7B5 +528A5BB93BAF2C9C4473CCE5D0D22BD9 +DF6A301E95C95DAD97AE0CC8C6913BD8 +801189902C857F39E73591285E70B6DB +E617346AC9C231BB3650AE34CCCA0C5B +27D93437EFB721AA401821DCEC5ADF89 +89237D9DED9C5E78D8B1C9B166CC7342 +4A6D8091BF5E7D651189FA94A250B14C +0E33F96055E7AE893FFC0E3DCF492902 +E61C432B720B19D18EC8D84BDC63151B +F7E5AEF549F782CF379055A608269B16 +438D030FD0B7A54FA837F2AD201A6403 +A590D3EE4FBF04E3247E0D27F286423F +5FE2C1A172FE93C4B15CD37CAEF9F538 +2C97325CBD06B36EB2133DD08B3A017C +92C814227A6BCA949FF0659F002AD39E +DCE850110BD8328CFBD50841D6911D87 +67F14984C7DA791248E32BB5922583DA +1938F2CF72D54EE97E94166FA91D2A36 +74481E9646ED49FE0F6224301604698E +57FCA5DE98A9D6D8006438D0583D8A1D +9FECDE1CEFDC1CBED4763674D9575359 +E3040C00EB28F15366CA73CBD872E740 +7697009A6A831DFECCA91C5993670F7A +5853542321F567A005D547A4F04759BD +5150D1772F50834A503E069A973FBD7C +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1CE422FEE7BD8DE20000000000000000 +1CE422FEE7BD8DE20000000000000000 diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql new file mode 100644 index 00000000000..323561654b9 --- /dev/null +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -0,0 +1,207 @@ +-- Test Vectors from the SipHash reference C implementation: +-- Written by +-- Jean-Philippe Aumasson +-- Daniel J. Bernstein +-- Released under CC0 +-- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 + +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + '')); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); +select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), + char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); + +-- CH tests +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); +select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); + +select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } +select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } + +select hex(sipHash128Reference()); +select hex(sipHash128ReferenceKeyed()); From 168fbc9d7bf1767f5b03363f8efb9a70d3de6601 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 02:17:23 +0100 Subject: [PATCH 049/116] Add a test --- tests/performance/datetime64_conversion.xml | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 tests/performance/datetime64_conversion.xml diff --git a/tests/performance/datetime64_conversion.xml b/tests/performance/datetime64_conversion.xml new file mode 100644 index 00000000000..ec13fe64242 --- /dev/null +++ b/tests/performance/datetime64_conversion.xml @@ -0,0 +1,4 @@ + + SELECT count() FROM numbers(20000000) WHERE NOT ignore(toDateTime64(rand(), 0)) + SELECT count() FROM numbers(20000000) WHERE NOT ignore(toDateTime64(rand(), 3)) + From d66beb9ac41d0be0a0f3b4a1877a5f23b53eba61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 07:56:00 +0100 Subject: [PATCH 050/116] Allow accurate comparison of Big Int with other integers --- src/Common/FieldVisitorsAccurateComparison.h | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Common/FieldVisitorsAccurateComparison.h b/src/Common/FieldVisitorsAccurateComparison.h index dfed2609040..2809974efe5 100644 --- a/src/Common/FieldVisitorsAccurateComparison.h +++ b/src/Common/FieldVisitorsAccurateComparison.h @@ -45,20 +45,20 @@ public: if constexpr (std::is_same_v) return l == r; - if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + if constexpr (is_arithmetic_v && is_arithmetic_v) return accurate::equalsOp(l, r); /// TODO This is wrong (does not respect scale). if constexpr (is_decimal_field && is_decimal_field) return l == r; - if constexpr (is_decimal_field && std::is_arithmetic_v) + if constexpr (is_decimal_field && is_arithmetic_v) return l == DecimalField(Decimal256(r), 0); - if constexpr (std::is_arithmetic_v && is_decimal_field) + if constexpr (is_arithmetic_v && is_decimal_field) return DecimalField(Decimal256(l), 0) == r; - if constexpr (std::is_same_v && std::is_arithmetic_v) + if constexpr (std::is_same_v && is_arithmetic_v) { ReadBufferFromString in(l); U parsed; @@ -66,7 +66,7 @@ public: return operator()(parsed, r); } - if constexpr (std::is_same_v && std::is_arithmetic_v) + if constexpr (std::is_same_v && is_arithmetic_v) { ReadBufferFromString in(r); T parsed; @@ -112,20 +112,20 @@ public: if constexpr (std::is_same_v) return l < r; - if constexpr (std::is_arithmetic_v && std::is_arithmetic_v) + if constexpr (is_arithmetic_v && is_arithmetic_v) return accurate::lessOp(l, r); /// TODO This is wrong (does not respect scale). if constexpr (is_decimal_field && is_decimal_field) return l < r; - if constexpr (is_decimal_field && std::is_arithmetic_v) + if constexpr (is_decimal_field && is_arithmetic_v) return l < DecimalField(Decimal256(r), 0); - if constexpr (std::is_arithmetic_v && is_decimal_field) + if constexpr (is_arithmetic_v && is_decimal_field) return DecimalField(Decimal256(l), 0) < r; - if constexpr (std::is_same_v && std::is_arithmetic_v) + if constexpr (std::is_same_v && is_arithmetic_v) { ReadBufferFromString in(l); U parsed; @@ -133,7 +133,7 @@ public: return operator()(parsed, r); } - if constexpr (std::is_same_v && std::is_arithmetic_v) + if constexpr (std::is_same_v && is_arithmetic_v) { ReadBufferFromString in(r); T parsed; From 2678defc627911afcf7b826f4122a50bad64454c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 07:58:03 +0100 Subject: [PATCH 051/116] Add a test --- .../02560_with_fill_int256_int.reference | 52 +++++++++++++++++++ .../02560_with_fill_int256_int.sql | 4 ++ 2 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/02560_with_fill_int256_int.reference create mode 100644 tests/queries/0_stateless/02560_with_fill_int256_int.sql diff --git a/tests/queries/0_stateless/02560_with_fill_int256_int.reference b/tests/queries/0_stateless/02560_with_fill_int256_int.reference new file mode 100644 index 00000000000..349e5688e29 --- /dev/null +++ b/tests/queries/0_stateless/02560_with_fill_int256_int.reference @@ -0,0 +1,52 @@ +0 +2 +3 +4 +5 +6 +7 +8 +10 +12 +14 +16 +18 +0 +2 +3 +4 +5 +6 +7 +8 +10 +12 +14 +16 +18 +0 +2 +3 +4 +5 +6 +7 +8 +10 +12 +14 +16 +18 +0 +2 +3 +4 +5 +6 +7 +8 +10 +12 +14 +16 +18 diff --git a/tests/queries/0_stateless/02560_with_fill_int256_int.sql b/tests/queries/0_stateless/02560_with_fill_int256_int.sql new file mode 100644 index 00000000000..77e621c84c7 --- /dev/null +++ b/tests/queries/0_stateless/02560_with_fill_int256_int.sql @@ -0,0 +1,4 @@ +SELECT (number * 2)::Int128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; +SELECT (number * 2)::Int256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; +SELECT (number * 2)::UInt128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; +SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; From 0791b85f57d8a2f7a9f60d9874e81f22c9f6cb05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 08:18:31 +0100 Subject: [PATCH 052/116] Remove bits of trash --- src/Common/FieldVisitorConvertToNumber.h | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index ed73cd38cda..b55f3ff23b7 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -53,7 +53,6 @@ public: T operator() (const UInt64 & x) const { return T(x); } T operator() (const Int64 & x) const { return T(x); } - T operator() (const Int128 & x) const { return T(x); } T operator() (const UUID & x) const { return T(x.toUnderType()); } T operator() (const IPv4 & x) const { return T(x.toUnderType()); } T operator() (const IPv6 & x) const { return T(x.toUnderType()); } @@ -87,11 +86,6 @@ public: } } - T operator() (const UInt128 &) const - { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert UInt128 to {}", demangle(typeid(T).name())); - } - template T operator() (const DecimalField & x) const { @@ -112,8 +106,6 @@ public: { if constexpr (is_decimal) return static_cast(static_cast(x)); - else if constexpr (std::is_same_v) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "No conversion to old UInt128 from {}", demangle(typeid(U).name())); else return static_cast(x); } From a9ec73a20938bc023b42c1923e331d79815e1f43 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 08:26:27 +0100 Subject: [PATCH 053/116] Fix strange trash --- src/Interpreters/FillingRow.cpp | 10 +++++--- .../Transforms/FillingTransform.cpp | 23 ++++++++++++++----- src/Processors/Transforms/FillingTransform.h | 2 ++ 3 files changed, 26 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 561ac255326..b03049a209f 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -49,7 +49,7 @@ bool FillingRow::next(const FillingRow & to_row) size_t pos = 0; /// Find position we need to increment for generating next row. - for (; pos < size(); ++pos) + for (size_t s = size(); pos < s; ++pos) if (!row[pos].isNull() && !to_row.row[pos].isNull() && !equals(row[pos], to_row.row[pos])) break; @@ -110,12 +110,16 @@ void FillingRow::initFromDefaults(size_t from_pos) void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & interpolate_columns, MutableColumns & other_columns, const FillingRow & filling_row, const Block & interpolate_block) { - for (size_t i = 0; i < filling_columns.size(); ++i) + for (size_t i = 0, size = filling_columns.size(); i < size; ++i) { if (filling_row[i].isNull()) + { filling_columns[i]->insertDefault(); + } else + { filling_columns[i]->insert(filling_row[i]); + } } if (size_t size = interpolate_block.columns()) @@ -134,7 +138,7 @@ void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & int void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num) { - for (size_t i = 0; i < source.size(); ++i) + for (size_t i = 0, size = source.size(); i < size; ++i) dest[i]->insertFrom(*source[i], row_num); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 2c5c550ffe2..c569dff15c1 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -73,8 +73,17 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & return false; } - /// TODO Wrong results for big integers. - if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) + if (which.isInt128() || which.isUInt128()) + { + max_type = Field::Types::Int128; + to_type = std::make_shared(); + } + else if (which.isInt256() || which.isUInt256()) + { + max_type = Field::Types::Int256; + to_type = std::make_shared(); + } + else if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) { max_type = Field::Types::Int64; to_type = std::make_shared(); @@ -184,18 +193,20 @@ FillingTransform::FillingTransform( fill_column_positions.push_back(block_position); auto & descr = filling_row.getFillDescription(i); - const auto & type = header_.getByPosition(block_position).type; + + const Block & output_header = getOutputPort().getHeader(); + const DataTypePtr & type = output_header.getByPosition(block_position).type; if (!tryConvertFields(descr, type)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "Incompatible types of WITH FILL expression values with column type {}", type->getName()); + "Incompatible types of WITH FILL expression values with column type {}", type->getName()); if (type->isValueRepresentedByUnsignedInteger() && ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) { throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, - "WITH FILL bound values cannot be negative for unsigned type {}", type->getName()); + "WITH FILL bound values cannot be negative for unsigned type {}", type->getName()); } } @@ -213,7 +224,7 @@ FillingTransform::FillingTransform( input_positions.emplace_back(idx, p->second); if (!is_fill_column[idx] && !(interpolate_description && interpolate_description->result_columns_set.contains(column.name))) - other_column_positions.push_back(idx); + other_column_positions.push_back(idx); ++idx; } diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 5e1068e18e2..7b41ab795d1 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -1,9 +1,11 @@ #pragma once + #include #include #include #include + namespace DB { From 966f5b01704aa2928c2d987fec61379d463117c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 08:28:36 +0100 Subject: [PATCH 054/116] Add a test --- .../02560_with_fill_int256_int.reference | 30 +++++++++++++++++++ .../02560_with_fill_int256_int.sql | 5 ++++ 2 files changed, 35 insertions(+) diff --git a/tests/queries/0_stateless/02560_with_fill_int256_int.reference b/tests/queries/0_stateless/02560_with_fill_int256_int.reference index 349e5688e29..1e7b1161f54 100644 --- a/tests/queries/0_stateless/02560_with_fill_int256_int.reference +++ b/tests/queries/0_stateless/02560_with_fill_int256_int.reference @@ -50,3 +50,33 @@ 14 16 18 +-3 +-2 +-1 +0 +1 +2 +3 +4 +6 +8 +10 +12 +14 +16 +18 +-3 +-2 +-1 +0 +1 +2 +3 +4 +6 +8 +10 +12 +14 +16 +18 diff --git a/tests/queries/0_stateless/02560_with_fill_int256_int.sql b/tests/queries/0_stateless/02560_with_fill_int256_int.sql index 77e621c84c7..a8876629ef2 100644 --- a/tests/queries/0_stateless/02560_with_fill_int256_int.sql +++ b/tests/queries/0_stateless/02560_with_fill_int256_int.sql @@ -2,3 +2,8 @@ SELECT (number * 2)::Int128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO SELECT (number * 2)::Int256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; SELECT (number * 2)::UInt128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO 8; + +SELECT (number * 2)::Int128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; +SELECT (number * 2)::Int256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; +SELECT (number * 2)::UInt128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 475 } +SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 475 } From 4f2a58ddbda895d9b2f4bc2f7305fe2ab40ba64d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 08:44:31 +0100 Subject: [PATCH 055/116] Fix style --- src/Common/FieldVisitorConvertToNumber.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index b55f3ff23b7..df884d7eebf 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_CONVERT_TYPE; - extern const int NOT_IMPLEMENTED; } From 8385263ba97571fbacd7ee0308ac1cb3be99df4d Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 8 Feb 2023 11:04:58 +0100 Subject: [PATCH 056/116] Add in code doc to sipHash128Reference --- src/Functions/FunctionsHashingMisc.cpp | 12 ++++++++++-- ...15_all_new_functions_must_be_documented.reference | 2 -- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index f544960b7b4..c1c3df1d53e 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -15,8 +15,16 @@ REGISTER_FUNCTION(Hashing) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction({ + "Like [sipHash128](#hash_functions-siphash128) but implements the 128-bit algorithm from the original authors of SipHash.", + Documentation::Examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))"}}, + Documentation::Categories{"Hash"} + }); + factory.registerFunction({ + "Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key.", + Documentation::Examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));"}}, + Documentation::Categories{"Hash"} + }); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); 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 dc106dc5bd9..e41249af54c 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 @@ -647,8 +647,6 @@ sin sinh sipHash128 sipHash128Keyed -sipHash128Reference -sipHash128ReferenceKeyed sipHash64 sipHash64Keyed sleep From f002698e0959d0b5bfbd7584dedbd634b4609040 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:16:59 +0000 Subject: [PATCH 057/116] More fine-granular warning relaxation --- base/poco/CMakeLists.txt | 3 --- base/poco/Crypto/CMakeLists.txt | 3 +++ base/poco/Data/CMakeLists.txt | 3 +++ base/poco/Foundation/CMakeLists.txt | 3 +++ base/poco/JSON/CMakeLists.txt | 3 +++ base/poco/MongoDB/CMakeLists.txt | 3 +++ base/poco/Net/CMakeLists.txt | 3 +++ base/poco/NetSSL_OpenSSL/CMakeLists.txt | 3 +++ base/poco/Redis/CMakeLists.txt | 3 +++ base/poco/Util/CMakeLists.txt | 3 +++ base/poco/XML/CMakeLists.txt | 3 +++ 11 files changed, 30 insertions(+), 3 deletions(-) diff --git a/base/poco/CMakeLists.txt b/base/poco/CMakeLists.txt index 0432ce196bf..82c48b5b622 100644 --- a/base/poco/CMakeLists.txt +++ b/base/poco/CMakeLists.txt @@ -1,6 +1,3 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - add_subdirectory (Crypto) add_subdirectory (Data) add_subdirectory (Data/ODBC) diff --git a/base/poco/Crypto/CMakeLists.txt b/base/poco/Crypto/CMakeLists.txt index 3753c5eed7b..205b1200054 100644 --- a/base/poco/Crypto/CMakeLists.txt +++ b/base/poco/Crypto/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + if (ENABLE_SSL) file (GLOB SRCS src/*.cpp) diff --git a/base/poco/Data/CMakeLists.txt b/base/poco/Data/CMakeLists.txt index 142d3592b69..d3c7e5fe772 100644 --- a/base/poco/Data/CMakeLists.txt +++ b/base/poco/Data/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS src/*.cpp) add_library (_poco_data ${SRCS}) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 3e7c7ae3e3e..340a8e99965 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -1,5 +1,8 @@ # Foundation (pcre) +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS_PCRE src/pcre_*.c) add_library (_poco_foundation_pcre ${SRCS_PCRE}) diff --git a/base/poco/JSON/CMakeLists.txt b/base/poco/JSON/CMakeLists.txt index 96fa00d098d..af04982817d 100644 --- a/base/poco/JSON/CMakeLists.txt +++ b/base/poco/JSON/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + # Poco::JSON (pdjson) set (SRCS_PDJSON diff --git a/base/poco/MongoDB/CMakeLists.txt b/base/poco/MongoDB/CMakeLists.txt index 92589fe438f..64798d65d6f 100644 --- a/base/poco/MongoDB/CMakeLists.txt +++ b/base/poco/MongoDB/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS src/*.cpp) add_library (_poco_mongodb ${SRCS}) diff --git a/base/poco/Net/CMakeLists.txt b/base/poco/Net/CMakeLists.txt index da58f20e07b..9f35ca5543b 100644 --- a/base/poco/Net/CMakeLists.txt +++ b/base/poco/Net/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS src/*.cpp) add_library (_poco_net ${SRCS}) diff --git a/base/poco/NetSSL_OpenSSL/CMakeLists.txt b/base/poco/NetSSL_OpenSSL/CMakeLists.txt index 9ec5acaab02..70b57331ee7 100644 --- a/base/poco/NetSSL_OpenSSL/CMakeLists.txt +++ b/base/poco/NetSSL_OpenSSL/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + if (ENABLE_SSL) file (GLOB SRCS src/*.cpp) diff --git a/base/poco/Redis/CMakeLists.txt b/base/poco/Redis/CMakeLists.txt index 562957140ba..c91d5e878b3 100644 --- a/base/poco/Redis/CMakeLists.txt +++ b/base/poco/Redis/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS src/*.cpp) add_library (_poco_redis ${SRCS}) diff --git a/base/poco/Util/CMakeLists.txt b/base/poco/Util/CMakeLists.txt index f37e099e648..65e4ca6e879 100644 --- a/base/poco/Util/CMakeLists.txt +++ b/base/poco/Util/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + file (GLOB SRCS src/*.cpp) add_library (_poco_util ${SRCS}) diff --git a/base/poco/XML/CMakeLists.txt b/base/poco/XML/CMakeLists.txt index 992cfbe763c..e82f85d138e 100644 --- a/base/poco/XML/CMakeLists.txt +++ b/base/poco/XML/CMakeLists.txt @@ -1,3 +1,6 @@ +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") + # Poco::XML (expat) file (GLOB SRCS_EXPAT src/xml*.c) From ba15cf834db1ba63561cceedc40191e61c26e4e6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:27:27 +0000 Subject: [PATCH 058/116] Fix warnings in poco foundation --- base/poco/Foundation/CMakeLists.txt | 47 +++++++++++++++++++++++------ 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 340a8e99965..9048b73d0b3 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -1,14 +1,25 @@ # Foundation (pcre) -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS_PCRE src/pcre_*.c) add_library (_poco_foundation_pcre ${SRCS_PCRE}) add_library (Poco::Foundation::PCRE ALIAS _poco_foundation_pcre) -target_compile_options (_poco_foundation_pcre PRIVATE -Wno-sign-compare) +# TODO: remove these warning exclusions +target_compile_options (_poco_foundation_pcre + PRIVATE + -Wno-cast-align + -Wno-cast-qual + -Wno-comma + -Wno-conditional-uninitialized + -Wno-extra-semi-stmt + -Wno-implicit-fallthrough + -Wno-reserved-identifier + -Wno-sign-compare + -Wno-unknown-pragmas + -Wno-unreachable-code-break + -Wno-unused-macros +) # Foundation @@ -172,21 +183,37 @@ set (SRCS add_library (_poco_foundation ${SRCS}) add_library (Poco::Foundation ALIAS _poco_foundation) +# TODO: remove these warning exclusions target_compile_options (_poco_foundation PRIVATE + -Wno-array-bounds -Wno-atomic-implicit-seq-cst + -Wno-cast-align + -Wno-cast-qual + -Wno-class-varargs + -Wno-covered-switch-default -Wno-deprecated -Wno-extra-semi-stmt - -Wno-zero-as-null-pointer-constant + -Wno-implicit-fallthrough -Wno-implicit-int-float-conversion + -Wno-misleading-indentation + -Wno-missing-noreturn + -Wno-old-style-cast + -Wno-redundant-parens + -Wno-reserved-identifier + -Wno-reserved-macro-identifier + -Wno-shadow + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-suggest-destructor-override + -Wno-suggest-override -Wno-thread-safety-analysis -Wno-thread-safety-negative -) - -target_compile_options (_poco_foundation - PRIVATE - -Wno-sign-compare + -Wno-unreachable-code-return + -Wno-unused-exception-parameter + -Wno-unused-macros -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant ) target_compile_definitions (_poco_foundation From 8dd0a465f50b75ad6fafb8931ab88ab1f0ed89d1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:31:44 +0000 Subject: [PATCH 059/116] Fix warnings in poco redis --- base/poco/Redis/CMakeLists.txt | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/base/poco/Redis/CMakeLists.txt b/base/poco/Redis/CMakeLists.txt index c91d5e878b3..f906b2d6183 100644 --- a/base/poco/Redis/CMakeLists.txt +++ b/base/poco/Redis/CMakeLists.txt @@ -1,11 +1,16 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS src/*.cpp) add_library (_poco_redis ${SRCS}) add_library (Poco::Redis ALIAS _poco_redis) -target_compile_options (_poco_redis PRIVATE -Wno-shadow) +# TODO: remove these warning exclusions +target_compile_options (_poco_redis + PRIVATE + -Wno-deprecated-dynamic-exception-spec + -Wno-shadow + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-zero-as-null-pointer-constant +) target_include_directories (_poco_redis SYSTEM PUBLIC "include") target_link_libraries (_poco_redis PUBLIC Poco::Net) From 3a339f8b029dca8284d0012770b34118f228322f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:36:13 +0000 Subject: [PATCH 060/116] Fix warnings in poco crypto --- base/poco/Crypto/CMakeLists.txt | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/base/poco/Crypto/CMakeLists.txt b/base/poco/Crypto/CMakeLists.txt index 205b1200054..d11a437fc02 100644 --- a/base/poco/Crypto/CMakeLists.txt +++ b/base/poco/Crypto/CMakeLists.txt @@ -1,13 +1,27 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - if (ENABLE_SSL) file (GLOB SRCS src/*.cpp) add_library (_poco_crypto ${SRCS}) add_library (Poco::Crypto ALIAS _poco_crypto) - target_compile_options (_poco_crypto PRIVATE -Wno-newline-eof) + # TODO: remove these warning exclusions + target_compile_options (_poco_crypto + PRIVATE + -Wno-covered-switch-default + -Wno-deprecated-dynamic-exception-spec + -Wno-extra-semi-stmt + -Wno-missing-noreturn + -Wno-newline-eof + -Wno-old-style-cast + -Wno-shadow + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-suggest-destructor-override + -Wno-suggest-override + -Wno-unreachable-code-return + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant + ) target_include_directories (_poco_crypto SYSTEM PUBLIC "include") target_link_libraries (_poco_crypto PUBLIC Poco::Foundation OpenSSL::SSL OpenSSL::Crypto) From b0f2c9894083d7038f25a684826073cd0a132264 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:40:38 +0000 Subject: [PATCH 061/116] Fix warnings in poco net --- base/poco/Net/CMakeLists.txt | 29 ++++++++++++++++++++--------- 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/base/poco/Net/CMakeLists.txt b/base/poco/Net/CMakeLists.txt index 9f35ca5543b..792045c9b43 100644 --- a/base/poco/Net/CMakeLists.txt +++ b/base/poco/Net/CMakeLists.txt @@ -1,6 +1,3 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS src/*.cpp) add_library (_poco_net ${SRCS}) @@ -12,17 +9,31 @@ elseif (OS_DARWIN OR OS_FREEBSD) target_compile_definitions (_poco_net PUBLIC POCO_HAVE_FD_POLL) endif () +# TODO: remove these warning exclusions target_compile_options (_poco_net PRIVATE -Wno-atomic-implicit-seq-cst - -Wno-extra-semi-stmt - -Wno-extra-semi -) - -target_compile_options (_poco_net - PRIVATE + -Wno-cast-align + -Wno-cast-qual + -Wno-comma + -Wno-covered-switch-default -Wno-deprecated -Wno-extra-semi + -Wno-extra-semi-stmt + -Wno-missing-noreturn + -Wno-old-style-cast + -Wno-shadow + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-suggest-destructor-override + -Wno-suggest-override + -Wno-undef + -Wno-unreachable-code + -Wno-unreachable-code-return + -Wno-unused-macros + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant ) + target_include_directories (_poco_net SYSTEM PUBLIC "include") target_link_libraries (_poco_net PUBLIC Poco::Foundation) From c0ca2db7abb5a04a28e7ef944eeff795590f67bf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:43:29 +0000 Subject: [PATCH 062/116] Fix warnings in poco data --- base/poco/Data/CMakeLists.txt | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/base/poco/Data/CMakeLists.txt b/base/poco/Data/CMakeLists.txt index d3c7e5fe772..dfdf3194f3c 100644 --- a/base/poco/Data/CMakeLists.txt +++ b/base/poco/Data/CMakeLists.txt @@ -1,10 +1,20 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS src/*.cpp) add_library (_poco_data ${SRCS}) add_library (Poco::Data ALIAS _poco_data) +# TODO: remove these warning exclusions +target_compile_options (_poco_data + PRIVATE + -Wno-comma + -Wno-covered-switch-default + -Wno-deprecated-dynamic-exception-spec + -Wno-extra-semi-stmt + -Wno-old-style-cast + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant +) target_include_directories (_poco_data SYSTEM PUBLIC "include") target_link_libraries (_poco_data PUBLIC Poco::Foundation) From 4ee6ac8deccd00b45de145519130d836104b5782 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:45:48 +0000 Subject: [PATCH 063/116] Fix warnings in poco utils --- base/poco/Util/CMakeLists.txt | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/base/poco/Util/CMakeLists.txt b/base/poco/Util/CMakeLists.txt index 65e4ca6e879..b359f672ef7 100644 --- a/base/poco/Util/CMakeLists.txt +++ b/base/poco/Util/CMakeLists.txt @@ -1,10 +1,20 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS src/*.cpp) add_library (_poco_util ${SRCS}) add_library (Poco::Util ALIAS _poco_util) +# TODO: remove these warning exclusions +target_compile_options (_poco_util + PRIVATE + -Wno-deprecated-dynamic-exception-spec + -Wno-newline-eof + -Wno-old-style-cast + -Wno-shadow + -Wno-sign-compare + -Wno-suggest-destructor-override + -Wno-suggest-override + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant +) target_include_directories (_poco_util SYSTEM PUBLIC "include") target_link_libraries (_poco_util PUBLIC Poco::JSON Poco::XML) From 353ef8b451a80ceaed134e475e983793333f1163 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:48:11 +0000 Subject: [PATCH 064/116] Fix warnings in poco mongodb --- base/poco/MongoDB/CMakeLists.txt | 11 ++++++++--- base/poco/Util/CMakeLists.txt | 1 + 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/base/poco/MongoDB/CMakeLists.txt b/base/poco/MongoDB/CMakeLists.txt index 64798d65d6f..8f5c6be2cae 100644 --- a/base/poco/MongoDB/CMakeLists.txt +++ b/base/poco/MongoDB/CMakeLists.txt @@ -1,10 +1,15 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - file (GLOB SRCS src/*.cpp) add_library (_poco_mongodb ${SRCS}) add_library (Poco::MongoDB ALIAS _poco_mongodb) +# TODO: remove these warning exclusions +target_compile_options (_poco_mongodb + PRIVATE + -Wno-old-style-cast + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant +) + target_include_directories (_poco_mongodb SYSTEM PUBLIC "include") target_link_libraries (_poco_mongodb PUBLIC Poco::Net) diff --git a/base/poco/Util/CMakeLists.txt b/base/poco/Util/CMakeLists.txt index b359f672ef7..52da9730d67 100644 --- a/base/poco/Util/CMakeLists.txt +++ b/base/poco/Util/CMakeLists.txt @@ -7,6 +7,7 @@ add_library (Poco::Util ALIAS _poco_util) target_compile_options (_poco_util PRIVATE -Wno-deprecated-dynamic-exception-spec + -Wno-extra-semi-stmt -Wno-newline-eof -Wno-old-style-cast -Wno-shadow From 1838c8912a62af74550906f13b8d3f779830e9fe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:51:41 +0000 Subject: [PATCH 065/116] Fix warnings in poco netssl_openssl --- base/poco/NetSSL_OpenSSL/CMakeLists.txt | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/CMakeLists.txt b/base/poco/NetSSL_OpenSSL/CMakeLists.txt index 70b57331ee7..0c200184932 100644 --- a/base/poco/NetSSL_OpenSSL/CMakeLists.txt +++ b/base/poco/NetSSL_OpenSSL/CMakeLists.txt @@ -1,12 +1,27 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - if (ENABLE_SSL) file (GLOB SRCS src/*.cpp) add_library (_poco_net_ssl ${SRCS}) add_library (Poco::Net::SSL ALIAS _poco_net_ssl) + # TODO: remove these warning exclusions + target_compile_options (_poco_net_ssl + PRIVATE + -Wno-cast-qual + -Wno-covered-switch-default + -Wno-deprecated-copy-with-user-provided-dtor + -Wno-deprecated-dynamic-exception-spec + -Wno-extra-semi + -Wno-extra-semi-stmt + -Wno-implicit-fallthrough + -Wno-old-style-cast + -Wno-shorten-64-to-32 + -Wno-sign-compare + -Wno-unused-exception-parameter + -Wno-unused-macros + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant + ) target_include_directories (_poco_net_ssl SYSTEM PUBLIC "include") target_link_libraries (_poco_net_ssl PUBLIC Poco::Crypto Poco::Net Poco::Util) else () From 64afe8be0b6c1fc2cfabf03c88e07f0641f7e200 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 17:58:15 +0000 Subject: [PATCH 066/116] Fix warnings in poco xml --- base/poco/XML/CMakeLists.txt | 34 ++++++++++++++++++++++++++++++---- 1 file changed, 30 insertions(+), 4 deletions(-) diff --git a/base/poco/XML/CMakeLists.txt b/base/poco/XML/CMakeLists.txt index e82f85d138e..102da8f1f5f 100644 --- a/base/poco/XML/CMakeLists.txt +++ b/base/poco/XML/CMakeLists.txt @@ -1,6 +1,3 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - # Poco::XML (expat) file (GLOB SRCS_EXPAT src/xml*.c) @@ -8,6 +5,17 @@ file (GLOB SRCS_EXPAT src/xml*.c) add_library (_poco_xml_expat ${SRCS_EXPAT}) add_library (Poco::XML::Expat ALIAS _poco_xml_expat) +# TODO: remove these warning exclusions +target_compile_options (_poco_xml_expat + PRIVATE + -Wno-cast-qual + -Wno-empty-translation-unit + -Wno-extra-semi-stmt + -Wno-implicit-fallthrough + -Wno-reserved-identifier + -Wno-unused-macros +) + target_include_directories (_poco_xml_expat PUBLIC "include") target_include_directories (_poco_xml_expat PRIVATE "../Foundation/include") @@ -17,6 +25,24 @@ file (GLOB SRCS src/*.cpp) add_library (_poco_xml ${SRCS}) add_library (Poco::XML ALIAS _poco_xml) -target_compile_options (_poco_xml PRIVATE -Wno-old-style-cast) +# TODO: remove these warning exclusions +target_compile_options (_poco_xml + PRIVATE + -Wno-cast-qual + -Wno-deprecated-dynamic-exception-spec + -Wno-implicit-fallthrough + -Wno-missing-noreturn + -Wno-old-style-cast + -Wno-reserved-identifier + -Wno-shadow + -Wno-shorten-64-to-32 + -Wno-suggest-destructor-override + -Wno-suggest-override + -Wno-tautological-type-limit-compare + -Wno-unreachable-code + -Wno-unused-macros + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant +) target_include_directories (_poco_xml SYSTEM PUBLIC "include") target_link_libraries (_poco_xml PUBLIC Poco::Foundation Poco::XML::Expat) From 65cbfdf5c63f51193e36e25e6484efc341ec99bd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 18:35:25 +0000 Subject: [PATCH 067/116] Fix warnings in poco json --- base/poco/JSON/CMakeLists.txt | 25 ++++++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/base/poco/JSON/CMakeLists.txt b/base/poco/JSON/CMakeLists.txt index af04982817d..482399cba28 100644 --- a/base/poco/JSON/CMakeLists.txt +++ b/base/poco/JSON/CMakeLists.txt @@ -1,6 +1,3 @@ -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") - # Poco::JSON (pdjson) set (SRCS_PDJSON @@ -10,6 +7,17 @@ set (SRCS_PDJSON add_library (_poco_json_pdjson ${SRCS_PDJSON}) add_library (Poco::JSON::Pdjson ALIAS _poco_json_pdjson) +# TODO: remove these warning exclusions +target_compile_options (_poco_json_pdjson + PRIVATE + -Wno-cast-qual + -Wno-declaration-after-statement + -Wno-extra-semi-stmt + -Wno-implicit-fallthrough + -Wno-shorten-64-to-32 + -Wno-unreachable-code-return +) + # Poco::JSON file (GLOB SRCS src/*.cpp) @@ -17,5 +25,16 @@ file (GLOB SRCS src/*.cpp) add_library (_poco_json ${SRCS}) add_library (Poco::JSON ALIAS _poco_json) +# TODO: remove these warning exclusions +target_compile_options (_poco_json + PRIVATE + -Wno-deprecated-dynamic-exception-spec + -Wno-sign-compare + -Wno-suggest-destructor-override + -Wno-suggest-override + -Wno-unused-parameter + -Wno-zero-as-null-pointer-constant +) + target_include_directories (_poco_json SYSTEM PUBLIC "include") target_link_libraries (_poco_json PUBLIC Poco::Foundation Poco::JSON::Pdjson) From cf2587cd1be8b76f22d0c46d75e207440f355bc0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 8 Feb 2023 12:32:26 +0100 Subject: [PATCH 068/116] Add docs --- docs/en/operations/settings/settings-formats.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index fd727704710..3f81dc528f5 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1001,6 +1001,12 @@ Use Arrow String type instead of Binary for String columns. Disabled by default. +### output_format_arrow_fixed_string_as_fixed_byte_array (#output_format_arrow_fixed_string_as_fixed_byte_array) + +Use Arrow FIXED_SIZE_BINARY type instead of Binary/String for FixedString columns. + +Enabled by default. + ## ORC format settings {#orc-format-settings} ### input_format_orc_import_nested {#input_format_orc_import_nested} @@ -1087,6 +1093,12 @@ Use Parquet String type instead of Binary for String columns. Disabled by default. +### output_format_parquet_fixed_string_as_fixed_byte_array (#output_format_parquet_fixed_string_as_fixed_byte_array) + +Use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. + +Enabled by default. + ## Hive format settings {#hive-format-settings} ### input_format_hive_text_fields_delimiter {#input_format_hive_text_fields_delimiter} From 537a697f65f632b3d57ac459fe4b8f38961f0f71 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 21:28:27 +0000 Subject: [PATCH 069/116] Fix whitespaces --- .../ODBC/include/Poco/Data/ODBC/Preparator.h | 2 +- base/poco/Data/include/Poco/Data/RecordSet.h | 2 +- .../include/Poco/HexBinaryDecoder.h | 4 +- .../include/Poco/HexBinaryEncoder.h | 6 +- .../include/Poco/UnicodeConverter.h | 6 +- base/poco/Foundation/src/DateTimeParser.cpp | 6 +- base/poco/Foundation/src/MD5Engine.cpp | 4 +- base/poco/Foundation/src/UnicodeConverter.cpp | 6 +- base/poco/Net/include/Poco/Net/ServerSocket.h | 2 +- base/poco/Net/include/Poco/Net/SocketImpl.h | 2 +- base/poco/Util/include/Poco/Util/Units.h | 2 +- base/poco/Util/src/ServerApplication.cpp | 2 +- base/poco/XML/include/Poco/XML/expat.h | 30 ++++----- .../XML/include/Poco/XML/expat_external.h | 28 ++++----- base/poco/XML/src/ascii.h | 28 ++++----- base/poco/XML/src/asciitab.h | 26 ++++---- base/poco/XML/src/iasciitab.h | 26 ++++---- base/poco/XML/src/internal.h | 26 ++++---- base/poco/XML/src/latin1tab.h | 26 ++++---- base/poco/XML/src/nametab.h | 20 +++--- base/poco/XML/src/utf8tab.h | 26 ++++---- base/poco/XML/src/xmlparse.cpp | 62 +++++++++---------- base/poco/XML/src/xmlrole.h | 28 ++++----- base/poco/XML/src/xmltok.h | 26 ++++---- base/poco/XML/src/xmltok_impl.h | 22 +++---- utils/check-style/check-whitespaces | 2 +- 26 files changed, 210 insertions(+), 210 deletions(-) diff --git a/base/poco/Data/ODBC/include/Poco/Data/ODBC/Preparator.h b/base/poco/Data/ODBC/include/Poco/Data/ODBC/Preparator.h index aa38ea28ce9..ac771b625b3 100644 --- a/base/poco/Data/ODBC/include/Poco/Data/ODBC/Preparator.h +++ b/base/poco/Data/ODBC/include/Poco/Data/ODBC/Preparator.h @@ -570,7 +570,7 @@ private: if (Utility::isError(SQLBindCol(_rStmt, (SQLUSMALLINT) pos + 1, valueType, - (SQLPOINTER) pVal, + (SQLPOINTER) pVal, (SQLINTEGER) dataSize, &_lengths[pos]))) { diff --git a/base/poco/Data/include/Poco/Data/RecordSet.h b/base/poco/Data/include/Poco/Data/RecordSet.h index 050b3892725..91046c36b9f 100644 --- a/base/poco/Data/include/Poco/Data/RecordSet.h +++ b/base/poco/Data/include/Poco/Data/RecordSet.h @@ -460,7 +460,7 @@ private: } else { - throw Poco::BadCastException(Poco::format("Type cast failed!\nColumn: %z\nTarget type:\t%s", + throw Poco::BadCastException(Poco::format("Type cast failed!\nColumn: %z\nTarget type:\t%s", pos, std::string(typeid(T).name()))); } diff --git a/base/poco/Foundation/include/Poco/HexBinaryDecoder.h b/base/poco/Foundation/include/Poco/HexBinaryDecoder.h index 7aab0f6cd22..7878cdede20 100644 --- a/base/poco/Foundation/include/Poco/HexBinaryDecoder.h +++ b/base/poco/Foundation/include/Poco/HexBinaryDecoder.h @@ -29,7 +29,7 @@ namespace Poco { class Foundation_API HexBinaryDecoderBuf: public UnbufferedStreamBuf /// This streambuf decodes all hexBinary-encoded data read /// from the istream connected to it. - /// In hexBinary encoding, each binary octet is encoded as a character tuple, + /// In hexBinary encoding, each binary octet is encoded as a character tuple, /// consisting of two hexadecimal digits ([0-9a-fA-F]) representing the octet code. /// See also: XML Schema Part 2: Datatypes (http://www.w3.org/TR/xmlschema-2/), /// section 3.2.15. @@ -71,7 +71,7 @@ protected: class Foundation_API HexBinaryDecoder: public HexBinaryDecoderIOS, public std::istream /// This istream decodes all hexBinary-encoded data read /// from the istream connected to it. - /// In hexBinary encoding, each binary octet is encoded as a character tuple, + /// In hexBinary encoding, each binary octet is encoded as a character tuple, /// consisting of two hexadecimal digits ([0-9a-fA-F]) representing the octet code. /// See also: XML Schema Part 2: Datatypes (http://www.w3.org/TR/xmlschema-2/), /// section 3.2.15. diff --git a/base/poco/Foundation/include/Poco/HexBinaryEncoder.h b/base/poco/Foundation/include/Poco/HexBinaryEncoder.h index 453c3cd821c..f2670d5d2a5 100644 --- a/base/poco/Foundation/include/Poco/HexBinaryEncoder.h +++ b/base/poco/Foundation/include/Poco/HexBinaryEncoder.h @@ -29,8 +29,8 @@ namespace Poco { class Foundation_API HexBinaryEncoderBuf: public UnbufferedStreamBuf /// This streambuf encodes all data written /// to it in hexBinary encoding and forwards it to a connected - /// ostream. - /// In hexBinary encoding, each binary octet is encoded as a character tuple, + /// ostream. + /// In hexBinary encoding, each binary octet is encoded as a character tuple, /// consisting of two hexadecimal digits ([0-9a-fA-F]) representing the octet code. /// See also: XML Schema Part 2: Datatypes (http://www.w3.org/TR/xmlschema-2/), /// section 3.2.15. @@ -95,7 +95,7 @@ class Foundation_API HexBinaryEncoder: public HexBinaryEncoderIOS, public std::o /// Always call close() when done /// writing data, to ensure proper /// completion of the encoding operation. - /// In hexBinary encoding, each binary octet is encoded as a character tuple, + /// In hexBinary encoding, each binary octet is encoded as a character tuple, /// consisting of two hexadecimal digits ([0-9a-fA-F]) representing the octet code. /// See also: XML Schema Part 2: Datatypes (http://www.w3.org/TR/xmlschema-2/), /// section 3.2.15. diff --git a/base/poco/Foundation/include/Poco/UnicodeConverter.h b/base/poco/Foundation/include/Poco/UnicodeConverter.h index 0733b67b7d9..da37e89d847 100644 --- a/base/poco/Foundation/include/Poco/UnicodeConverter.h +++ b/base/poco/Foundation/include/Poco/UnicodeConverter.h @@ -37,7 +37,7 @@ public: static void convert(const std::string& utf8String, UTF32String& utf32String); /// Converts the given UTF-8 encoded string into an UTF-32 encoded wide string. - static void convert(const char* utf8String, std::size_t length, UTF32String& utf32String); + static void convert(const char* utf8String, std::size_t length, UTF32String& utf32String); /// Converts the given UTF-8 encoded character sequence into an UTF-32 encoded wide string. static void convert(const char* utf8String, UTF32String& utf32String); @@ -46,7 +46,7 @@ public: static void convert(const std::string& utf8String, UTF16String& utf16String); /// Converts the given UTF-8 encoded string into an UTF-16 encoded wide string. - static void convert(const char* utf8String, std::size_t length, UTF16String& utf16String); + static void convert(const char* utf8String, std::size_t length, UTF16String& utf16String); /// Converts the given UTF-8 encoded character sequence into an UTF-16 encoded wide string. static void convert(const char* utf8String, UTF16String& utf16String); @@ -58,7 +58,7 @@ public: static void convert(const UTF32String& utf32String, std::string& utf8String); /// Converts the given UTF-32 encoded wide string into an UTF-8 encoded string. - static void convert(const UTF16Char* utf16String, std::size_t length, std::string& utf8String); + static void convert(const UTF16Char* utf16String, std::size_t length, std::string& utf8String); /// Converts the given zero-terminated UTF-16 encoded wide character sequence into an UTF-8 encoded string. static void convert(const UTF32Char* utf16String, std::size_t length, std::string& utf8String); diff --git a/base/poco/Foundation/src/DateTimeParser.cpp b/base/poco/Foundation/src/DateTimeParser.cpp index b58482c7b33..c89c232f4a0 100644 --- a/base/poco/Foundation/src/DateTimeParser.cpp +++ b/base/poco/Foundation/src/DateTimeParser.cpp @@ -276,11 +276,11 @@ int DateTimeParser::parseTZD(std::string::const_iterator& it, const std::string: {"PDT", -7*3600}, {"AKST", -9*3600}, {"AKDT", -8*3600}, - {"HST", -10*3600}, + {"HST", -10*3600}, {"AEST", 10*3600}, {"AEDT", 11*3600}, - {"ACST", 9*3600+1800}, - {"ACDT", 10*3600+1800}, + {"ACST", 9*3600+1800}, + {"ACDT", 10*3600+1800}, {"AWST", 8*3600}, {"AWDT", 9*3600} }; diff --git a/base/poco/Foundation/src/MD5Engine.cpp b/base/poco/Foundation/src/MD5Engine.cpp index 7ac89f0bd37..06a549ec192 100644 --- a/base/poco/Foundation/src/MD5Engine.cpp +++ b/base/poco/Foundation/src/MD5Engine.cpp @@ -224,7 +224,7 @@ void MD5Engine::transform (UInt32 state[4], const unsigned char block[64]) GG (c, d, a, b, x[11], S23, 0x265e5a51); /* 19 */ GG (b, c, d, a, x[ 0], S24, 0xe9b6c7aa); /* 20 */ GG (a, b, c, d, x[ 5], S21, 0xd62f105d); /* 21 */ - GG (d, a, b, c, x[10], S22, 0x2441453); /* 22 */ + GG (d, a, b, c, x[10], S22, 0x2441453); /* 22 */ GG (c, d, a, b, x[15], S23, 0xd8a1e681); /* 23 */ GG (b, c, d, a, x[ 4], S24, 0xe7d3fbc8); /* 24 */ GG (a, b, c, d, x[ 9], S21, 0x21e1cde6); /* 25 */ @@ -248,7 +248,7 @@ void MD5Engine::transform (UInt32 state[4], const unsigned char block[64]) HH (a, b, c, d, x[13], S31, 0x289b7ec6); /* 41 */ HH (d, a, b, c, x[ 0], S32, 0xeaa127fa); /* 42 */ HH (c, d, a, b, x[ 3], S33, 0xd4ef3085); /* 43 */ - HH (b, c, d, a, x[ 6], S34, 0x4881d05); /* 44 */ + HH (b, c, d, a, x[ 6], S34, 0x4881d05); /* 44 */ HH (a, b, c, d, x[ 9], S31, 0xd9d4d039); /* 45 */ HH (d, a, b, c, x[12], S32, 0xe6db99e5); /* 46 */ HH (c, d, a, b, x[15], S33, 0x1fa27cf8); /* 47 */ diff --git a/base/poco/Foundation/src/UnicodeConverter.cpp b/base/poco/Foundation/src/UnicodeConverter.cpp index 992c644c614..f0a7742d00c 100644 --- a/base/poco/Foundation/src/UnicodeConverter.cpp +++ b/base/poco/Foundation/src/UnicodeConverter.cpp @@ -86,7 +86,7 @@ void UnicodeConverter::convert(const std::string& utf8String, UTF16String& utf16 } -void UnicodeConverter::convert(const char* utf8String, std::size_t length, UTF16String& utf16String) +void UnicodeConverter::convert(const char* utf8String, std::size_t length, UTF16String& utf16String) { if (!utf8String || !length) { @@ -130,7 +130,7 @@ void UnicodeConverter::convert(const UTF32String& utf32String, std::string& utf8 } -void UnicodeConverter::convert(const UTF16Char* utf16String, std::size_t length, std::string& utf8String) +void UnicodeConverter::convert(const UTF16Char* utf16String, std::size_t length, std::string& utf8String) { utf8String.clear(); UTF8Encoding utf8Encoding; @@ -140,7 +140,7 @@ void UnicodeConverter::convert(const UTF16Char* utf16String, std::size_t length } -void UnicodeConverter::convert(const UTF32Char* utf32String, std::size_t length, std::string& utf8String) +void UnicodeConverter::convert(const UTF32Char* utf32String, std::size_t length, std::string& utf8String) { toUTF8(UTF32String(utf32String, length), utf8String); } diff --git a/base/poco/Net/include/Poco/Net/ServerSocket.h b/base/poco/Net/include/Poco/Net/ServerSocket.h index d8aca6790ae..d206dd534ce 100644 --- a/base/poco/Net/include/Poco/Net/ServerSocket.h +++ b/base/poco/Net/include/Poco/Net/ServerSocket.h @@ -131,7 +131,7 @@ public: /// If the library has not been built with IPv6 support, /// a Poco::NotImplementedException will be thrown. - virtual void bind6(const SocketAddress& address, bool reuseAddress, bool reusePort, bool ipV6Only); + virtual void bind6(const SocketAddress& address, bool reuseAddress, bool reusePort, bool ipV6Only); /// Binds a local IPv6 address to the socket. /// /// This is usually only done when establishing a server diff --git a/base/poco/Net/include/Poco/Net/SocketImpl.h b/base/poco/Net/include/Poco/Net/SocketImpl.h index d1fec1de7d6..b88f620dd65 100644 --- a/base/poco/Net/include/Poco/Net/SocketImpl.h +++ b/base/poco/Net/include/Poco/Net/SocketImpl.h @@ -114,7 +114,7 @@ public: /// If the library has not been built with IPv6 support, /// a Poco::NotImplementedException will be thrown. - virtual void bind6(const SocketAddress& address, bool reuseAddress, bool reusePort, bool ipV6Only); + virtual void bind6(const SocketAddress& address, bool reuseAddress, bool reusePort, bool ipV6Only); /// Bind a local IPv6 address to the socket. /// /// This is usually only done when establishing a server diff --git a/base/poco/Util/include/Poco/Util/Units.h b/base/poco/Util/include/Poco/Util/Units.h index e7f3869dda4..47e8fe3c3aa 100644 --- a/base/poco/Util/include/Poco/Util/Units.h +++ b/base/poco/Util/include/Poco/Util/Units.h @@ -848,7 +848,7 @@ UNIT_DISPLAY_NAME(Units::T, "T"); UNIT_DISPLAY_NAME(Units::H, "H"); UNIT_DISPLAY_NAME(Units::lx, "lx"); UNIT_DISPLAY_NAME(Units::Gy, "Gy"); -UNIT_DISPLAY_NAME(Units::kat, "kat"); +UNIT_DISPLAY_NAME(Units::kat, "kat"); namespace Units diff --git a/base/poco/Util/src/ServerApplication.cpp b/base/poco/Util/src/ServerApplication.cpp index 99107151e3b..dbfa5d703d9 100644 --- a/base/poco/Util/src/ServerApplication.cpp +++ b/base/poco/Util/src/ServerApplication.cpp @@ -143,7 +143,7 @@ void ServerApplication::ServiceControlHandler(DWORD control) case SERVICE_CONTROL_INTERROGATE: break; } - SetServiceStatus(_serviceStatusHandle, &_serviceStatus); + SetServiceStatus(_serviceStatusHandle, &_serviceStatus); } diff --git a/base/poco/XML/include/Poco/XML/expat.h b/base/poco/XML/include/Poco/XML/expat.h index c9214f64070..93a5d19d53f 100644 --- a/base/poco/XML/include/Poco/XML/expat.h +++ b/base/poco/XML/include/Poco/XML/expat.h @@ -7,33 +7,33 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper + Copyright (c) 2000 Clark Cooper Copyright (c) 2000-2005 Fred L. Drake, Jr. Copyright (c) 2001-2002 Greg Stein Copyright (c) 2002-2016 Karl Waclawek Copyright (c) 2016-2022 Sebastian Pipping - Copyright (c) 2016 Cristian Rodríguez - Copyright (c) 2016 Thomas Beutlich - Copyright (c) 2017 Rhodri James - Copyright (c) 2022 Thijs Schreijer + Copyright (c) 2016 Cristian Rodríguez + Copyright (c) 2016 Thomas Beutlich + Copyright (c) 2017 Rhodri James + Copyright (c) 2022 Thijs Schreijer Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/include/Poco/XML/expat_external.h b/base/poco/XML/include/Poco/XML/expat_external.h index 8829f770910..9c5fc8a2e8b 100644 --- a/base/poco/XML/include/Poco/XML/expat_external.h +++ b/base/poco/XML/include/Poco/XML/expat_external.h @@ -7,32 +7,32 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper + Copyright (c) 2000 Clark Cooper Copyright (c) 2000-2004 Fred L. Drake, Jr. Copyright (c) 2001-2002 Greg Stein Copyright (c) 2002-2006 Karl Waclawek - Copyright (c) 2016 Cristian Rodríguez + Copyright (c) 2016 Cristian Rodríguez Copyright (c) 2016-2019 Sebastian Pipping - Copyright (c) 2017 Rhodri James - Copyright (c) 2018 Yury Gribov + Copyright (c) 2017 Rhodri James + Copyright (c) 2018 Yury Gribov Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/ascii.h b/base/poco/XML/src/ascii.h index 1f594d2e54b..5e696150817 100644 --- a/base/poco/XML/src/ascii.h +++ b/base/poco/XML/src/ascii.h @@ -7,28 +7,28 @@ |_| XML parser Copyright (c) 1999-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2007 Karl Waclawek - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2007 Karl Waclawek + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/asciitab.h b/base/poco/XML/src/asciitab.h index 1f6453a4d0b..86df212f5a1 100644 --- a/base/poco/XML/src/asciitab.h +++ b/base/poco/XML/src/asciitab.h @@ -7,27 +7,27 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/iasciitab.h b/base/poco/XML/src/iasciitab.h index 29853841fc6..44bd93f691e 100644 --- a/base/poco/XML/src/iasciitab.h +++ b/base/poco/XML/src/iasciitab.h @@ -7,27 +7,27 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/internal.h b/base/poco/XML/src/internal.h index ea906711448..d9e592052f9 100644 --- a/base/poco/XML/src/internal.h +++ b/base/poco/XML/src/internal.h @@ -27,28 +27,28 @@ Copyright (c) 2002-2003 Fred L. Drake, Jr. Copyright (c) 2002-2006 Karl Waclawek - Copyright (c) 2003 Greg Stein + Copyright (c) 2003 Greg Stein Copyright (c) 2016-2021 Sebastian Pipping - Copyright (c) 2018 Yury Gribov - Copyright (c) 2019 David Loffredo + Copyright (c) 2018 Yury Gribov + Copyright (c) 2019 David Loffredo Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/latin1tab.h b/base/poco/XML/src/latin1tab.h index 34ab055f703..a63ebd6a122 100644 --- a/base/poco/XML/src/latin1tab.h +++ b/base/poco/XML/src/latin1tab.h @@ -7,27 +7,27 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/nametab.h b/base/poco/XML/src/nametab.h index 63485446b96..e8cf8d7d9d2 100644 --- a/base/poco/XML/src/nametab.h +++ b/base/poco/XML/src/nametab.h @@ -10,22 +10,22 @@ Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/utf8tab.h b/base/poco/XML/src/utf8tab.h index 51387789ed6..4e28ec2e9a9 100644 --- a/base/poco/XML/src/utf8tab.h +++ b/base/poco/XML/src/utf8tab.h @@ -7,27 +7,27 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/xmlparse.cpp b/base/poco/XML/src/xmlparse.cpp index 99570a1fb12..3207ac3ddbd 100644 --- a/base/poco/XML/src/xmlparse.cpp +++ b/base/poco/XML/src/xmlparse.cpp @@ -7,52 +7,52 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper + Copyright (c) 2000 Clark Cooper Copyright (c) 2000-2006 Fred L. Drake, Jr. Copyright (c) 2001-2002 Greg Stein Copyright (c) 2002-2016 Karl Waclawek Copyright (c) 2005-2009 Steven Solie - Copyright (c) 2016 Eric Rahm + Copyright (c) 2016 Eric Rahm Copyright (c) 2016-2022 Sebastian Pipping - Copyright (c) 2016 Gaurav - Copyright (c) 2016 Thomas Beutlich - Copyright (c) 2016 Gustavo Grieco - Copyright (c) 2016 Pascal Cuoq - Copyright (c) 2016 Ed Schouten + Copyright (c) 2016 Gaurav + Copyright (c) 2016 Thomas Beutlich + Copyright (c) 2016 Gustavo Grieco + Copyright (c) 2016 Pascal Cuoq + Copyright (c) 2016 Ed Schouten Copyright (c) 2017-2018 Rhodri James - Copyright (c) 2017 Václav Slavík - Copyright (c) 2017 Viktor Szakats - Copyright (c) 2017 Chanho Park - Copyright (c) 2017 Rolf Eike Beer - Copyright (c) 2017 Hans Wennborg - Copyright (c) 2018 Anton Maklakov - Copyright (c) 2018 Benjamin Peterson - Copyright (c) 2018 Marco Maggi - Copyright (c) 2018 Mariusz Zaborski - Copyright (c) 2019 David Loffredo + Copyright (c) 2017 Václav Slavík + Copyright (c) 2017 Viktor Szakats + Copyright (c) 2017 Chanho Park + Copyright (c) 2017 Rolf Eike Beer + Copyright (c) 2017 Hans Wennborg + Copyright (c) 2018 Anton Maklakov + Copyright (c) 2018 Benjamin Peterson + Copyright (c) 2018 Marco Maggi + Copyright (c) 2018 Mariusz Zaborski + Copyright (c) 2019 David Loffredo Copyright (c) 2019-2020 Ben Wagner - Copyright (c) 2019 Vadim Zeitlin - Copyright (c) 2021 Dong-hee Na - Copyright (c) 2022 Samanta Navarro - Copyright (c) 2022 Jeffrey Walton + Copyright (c) 2019 Vadim Zeitlin + Copyright (c) 2021 Dong-hee Na + Copyright (c) 2022 Samanta Navarro + Copyright (c) 2022 Jeffrey Walton Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/xmlrole.h b/base/poco/XML/src/xmlrole.h index d6e1fa150a1..0262f7b24c8 100644 --- a/base/poco/XML/src/xmlrole.h +++ b/base/poco/XML/src/xmlrole.h @@ -7,28 +7,28 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Karl Waclawek - Copyright (c) 2002 Fred L. Drake, Jr. - Copyright (c) 2017 Sebastian Pipping + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Karl Waclawek + Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2017 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/xmltok.h b/base/poco/XML/src/xmltok.h index 6f630c2f9ba..4e786a14307 100644 --- a/base/poco/XML/src/xmltok.h +++ b/base/poco/XML/src/xmltok.h @@ -7,29 +7,29 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper - Copyright (c) 2002 Fred L. Drake, Jr. + Copyright (c) 2000 Clark Cooper + Copyright (c) 2002 Fred L. Drake, Jr. Copyright (c) 2002-2005 Karl Waclawek Copyright (c) 2016-2017 Sebastian Pipping - Copyright (c) 2017 Rhodri James + Copyright (c) 2017 Rhodri James Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/base/poco/XML/src/xmltok_impl.h b/base/poco/XML/src/xmltok_impl.h index c518aada013..97e222b00c7 100644 --- a/base/poco/XML/src/xmltok_impl.h +++ b/base/poco/XML/src/xmltok_impl.h @@ -7,26 +7,26 @@ |_| XML parser Copyright (c) 1997-2000 Thai Open Source Software Center Ltd - Copyright (c) 2000 Clark Cooper + Copyright (c) 2000 Clark Cooper Copyright (c) 2017-2019 Sebastian Pipping Licensed under the MIT license: - Permission is hereby granted, free of charge, to any person obtaining - a copy of this software and associated documentation files (the - "Software"), to deal in the Software without restriction, including - without limitation the rights to use, copy, modify, merge, publish, + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit - persons to whom the Software is furnished to do so, subject to the + persons to whom the Software is furnished to do so, subject to the following conditions: - The above copyright notice and this permission notice shall be included + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. - THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, + DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ diff --git a/utils/check-style/check-whitespaces b/utils/check-style/check-whitespaces index 331190f7804..5a20569868d 100755 --- a/utils/check-style/check-whitespaces +++ b/utils/check-style/check-whitespaces @@ -1,7 +1,7 @@ #!/usr/bin/env bash ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|poco/|memcpy/|consistent-hashing/|Parsers/New' +EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing/|Parsers/New' # Double whitespaces find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' 2>/dev/null | From fbea504ea1b79e818c7ae1be732663de1a018959 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 8 Feb 2023 16:12:21 +0300 Subject: [PATCH 070/116] Remove excessive license notices from preciseExp10.cpp --- base/base/preciseExp10.cpp | 149 ++----------------------------------- base/base/preciseExp10.h | 4 +- 2 files changed, 6 insertions(+), 147 deletions(-) diff --git a/base/base/preciseExp10.cpp b/base/base/preciseExp10.cpp index 6a4025889b7..53d6ebd9f1b 100644 --- a/base/base/preciseExp10.cpp +++ b/base/base/preciseExp10.cpp @@ -1,8 +1,9 @@ /* - -https://www.musl-libc.org/ -http://git.musl-libc.org/cgit/musl/tree/src/math/exp10.c - + Origin: musl-libc /src/math/exp10.c + https://www.musl-libc.org/ + http://git.musl-libc.org/cgit/musl/tree/src/math/exp10.c +*/ +/* musl as a whole is licensed under the following standard MIT license: ---------------------------------------------------------------------- @@ -27,146 +28,6 @@ CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. ---------------------------------------------------------------------- - -Authors/contributors include: - -Alex Dowad -Alexander Monakov -Anthony G. Basile -Arvid Picciani -Bobby Bingham -Boris Brezillon -Brent Cook -Chris Spiegel -Clément Vasseur -Daniel Micay -Denys Vlasenko -Emil Renner Berthing -Felix Fietkau -Felix Janda -Gianluca Anzolin -Hauke Mehrtens -Hiltjo Posthuma -Isaac Dunham -Jaydeep Patil -Jens Gustedt -Jeremy Huntwork -Jo-Philipp Wich -Joakim Sindholt -John Spencer -Josiah Worcester -Justin Cormack -Khem Raj -Kylie McClain -Luca Barbato -Luka Perkov -M Farkas-Dyck (Strake) -Mahesh Bodapati -Michael Forney -Natanael Copa -Nicholas J. Kain -orc -Pascal Cuoq -Petr Hosek -Pierre Carrier -Rich Felker -Richard Pennington -Shiz -sin -Solar Designer -Stefan Kristiansson -Szabolcs Nagy -Timo Teräs -Trutz Behn -Valentin Ochs -William Haddon - -Portions of this software are derived from third-party works licensed -under terms compatible with the above MIT license: - -The TRE regular expression implementation (src/regex/reg* and -src/regex/tre*) is Copyright © 2001-2008 Ville Laurikari and licensed -under a 2-clause BSD license (license text in the source files). The -included version has been heavily modified by Rich Felker in 2012, in -the interests of size, simplicity, and namespace cleanliness. - -Much of the math library code (src/math/ * and src/complex/ *) is -Copyright © 1993,2004 Sun Microsystems or -Copyright © 2003-2011 David Schultz or -Copyright © 2003-2009 Steven G. Kargl or -Copyright © 2003-2009 Bruce D. Evans or -Copyright © 2008 Stephen L. Moshier -and labelled as such in comments in the individual source files. All -have been licensed under extremely permissive terms. - -The ARM memcpy code (src/string/arm/memcpy_el.S) is Copyright © 2008 -The Android Open Source Project and is licensed under a two-clause BSD -license. It was taken from Bionic libc, used on Android. - -The implementation of DES for crypt (src/crypt/crypt_des.c) is -Copyright © 1994 David Burren. It is licensed under a BSD license. - -The implementation of blowfish crypt (src/crypt/crypt_blowfish.c) was -originally written by Solar Designer and placed into the public -domain. The code also comes with a fallback permissive license for use -in jurisdictions that may not recognize the public domain. - -The smoothsort implementation (src/stdlib/qsort.c) is Copyright © 2011 -Valentin Ochs and is licensed under an MIT-style license. - -The BSD PRNG implementation (src/prng/random.c) and XSI search API -(src/search/ *.c) functions are Copyright © 2011 Szabolcs Nagy and -licensed under following terms: "Permission to use, copy, modify, -and/or distribute this code for any purpose with or without fee is -hereby granted. There is no warranty." - -The x86_64 port was written by Nicholas J. Kain and is licensed under -the standard MIT terms. - -The mips and microblaze ports were originally written by Richard -Pennington for use in the ellcc project. The original code was adapted -by Rich Felker for build system and code conventions during upstream -integration. It is licensed under the standard MIT terms. - -The mips64 port was contributed by Imagination Technologies and is -licensed under the standard MIT terms. - -The powerpc port was also originally written by Richard Pennington, -and later supplemented and integrated by John Spencer. It is licensed -under the standard MIT terms. - -All other files which have no copyright comments are original works -produced specifically for use as part of this library, written either -by Rich Felker, the main author of the library, or by one or more -contibutors listed above. Details on authorship of individual files -can be found in the git version control history of the project. The -omission of copyright and license comments in each file is in the -interest of source tree size. - -In addition, permission is hereby granted for all public header files -(include/ * and arch/ * /bits/ *) and crt files intended to be linked into -applications (crt/ *, ldso/dlstart.c, and arch/ * /crt_arch.h) to omit -the copyright notice and permission notice otherwise required by the -license, and to use these files without any requirement of -attribution. These files include substantial contributions from: - -Bobby Bingham -John Spencer -Nicholas J. Kain -Rich Felker -Richard Pennington -Stefan Kristiansson -Szabolcs Nagy - -all of whom have explicitly granted such permission. - -This file previously contained text expressing a belief that most of -the files covered by the above exception were sufficiently trivial not -to be subject to copyright, resulting in confusion over whether it -negated the permissions granted in the license. In the spirit of -permissive licensing, and of not having licensing issues being an -obstacle to adoption, that text has been removed. - */ #include diff --git a/base/base/preciseExp10.h b/base/base/preciseExp10.h index 26a88318172..7e16aa25d24 100644 --- a/base/base/preciseExp10.h +++ b/base/base/preciseExp10.h @@ -3,9 +3,7 @@ /** exp10 from GNU libm fails to give precise result for integer arguments. * For example, exp10(3) gives 1000.0000000000001 * despite the fact that 1000 is exactly representable in double and float. - * Better to always use implementation from MUSL. - * - * Note: the function names are different to avoid confusion with symbols from the system libm. + * Better to always use our own implementation based on MUSL implementation. */ double preciseExp10(double x); From 53f8ee85e5b4d489a9eb9520985268ee9b38a0f7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 8 Feb 2023 15:30:05 +0000 Subject: [PATCH 071/116] Fix warnings in poco odbc --- base/poco/Data/ODBC/CMakeLists.txt | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/base/poco/Data/ODBC/CMakeLists.txt b/base/poco/Data/ODBC/CMakeLists.txt index 4a49f915b67..62f0e9f1969 100644 --- a/base/poco/Data/ODBC/CMakeLists.txt +++ b/base/poco/Data/ODBC/CMakeLists.txt @@ -23,7 +23,18 @@ if (ENABLE_ODBC) add_library (_poco_data_odbc ${SRCS}) add_library (Poco::Data::ODBC ALIAS _poco_data_odbc) - target_compile_options (_poco_data_odbc PRIVATE -Wno-unused-variable) + # TODO: remove these warning exclusions + target_compile_options (_poco_data_odbc + PRIVATE + -Wno-cast-qual + -Wno-deprecated-dynamic-exception-spec + -Wno-extra-semi-stmt + -Wno-old-style-cast + -Wno-sign-compare + -Wno-unused-parameter + -Wno-unused-variable + -Wno-zero-as-null-pointer-constant + ) target_include_directories (_poco_data_odbc SYSTEM PUBLIC "include") target_link_libraries (_poco_data_odbc PUBLIC Poco::Data ch_contrib::unixodbc) From 74b892c0b9bf49204e976e75aa8f32dafa3c9c7a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 8 Feb 2023 15:40:49 +0000 Subject: [PATCH 072/116] Fix FreeBSD build --- base/poco/Foundation/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 9048b73d0b3..0dcc954cb98 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -209,6 +209,7 @@ target_compile_options (_poco_foundation -Wno-suggest-override -Wno-thread-safety-analysis -Wno-thread-safety-negative + -Wno-undef -Wno-unreachable-code-return -Wno-unused-exception-parameter -Wno-unused-macros From cb45d3aec1c7d761629d091360e1674dd285e48e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Feb 2023 19:00:17 +0300 Subject: [PATCH 073/116] Update Dockerfile.ubuntu --- docker/server/Dockerfile.ubuntu | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index beb70cbe662..ba2d7430e06 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -16,7 +16,6 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list gnupg \ locales \ wget \ - yasm \ tzdata \ && apt-get clean From 9314c90b0531a49daa4912d4c2137a5cc6fb3535 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Feb 2023 12:28:02 +0100 Subject: [PATCH 074/116] add 00002_log_and_exception_messages_formatting back --- ...nd_exception_messages_formatting.reference | 15 +++++ ..._log_and_exception_messages_formatting.sql | 64 +++++++++++++++++++ 2 files changed, 79 insertions(+) create mode 100644 tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference create mode 100644 tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference new file mode 100644 index 00000000000..032d7e396ff --- /dev/null +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -0,0 +1,15 @@ +runtime messages 0.001 +runtime exceptions 0.05 +messages shorter than 10 10 +messages shorter than 16 40 +exceptions shorter than 30 125 +noisy messages 0.3 +noisy Trace messages 0.16 +noisy Debug messages 0.09 +noisy Info messages 0.05 +noisy Warning messages 0.01 +noisy Error messages 0.02 +no Fatal messages 0 +number of too noisy messages 3 +number of noisy messages 10 +incorrect patterns 15 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql new file mode 100644 index 00000000000..bb2a8ce8b98 --- /dev/null +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -0,0 +1,64 @@ +-- Tags: no-parallel, no-fasttest +-- no-parallel because we want to run this test when most of the other tests already passed + +-- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log + +system flush logs; +drop table if exists logs; +create view logs as select * from system.text_log where now() - toIntervalMinute(120) < event_time; + +-- Check that we don't have too many messages formatted with fmt::runtime or strings concatenation. +-- 0.001 threshold should be always enough, the value was about 0.00025 +select 'runtime messages', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.001) from logs; + +-- Check the same for exceptions. The value was 0.03 +select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where message like '%DB::Exception%'; + +-- Check that we don't have too many short meaningless message patterns. +select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 10) from logs where length(message_format_string) < 10; + +-- Same as above. Feel free to update the threshold or remove this query if really necessary +select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 40) from logs where length(message_format_string) < 16; + +-- Same as above, but exceptions must be more informative. Feel free to update the threshold or remove this query if really necessary +select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 125) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%'; + + +-- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold +select 'noisy messages', max2((select count() from logs group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.30); + +-- Same as above, but excluding Test level (actually finds top 1 Trace message) +with ('Access granted: {}{}', '{} -> {}') as frequent_in_tests +select 'noisy Trace messages', max2((select count() from logs where level!='Test' and message_format_string not in frequent_in_tests + group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.16); + +-- Same as above for Debug +select 'noisy Debug messages', max2((select count() from logs where level <= 'Debug' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.09); + +-- Same as above for Info +select 'noisy Info messages', max2((select count() from logs where level <= 'Information' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.05); + +-- Same as above for Warning +with ('Not enabled four letter command {}') as frequent_in_tests +select 'noisy Warning messages', max2((select countOrDefault() from logs where level = 'Warning' and message_format_string not in frequent_in_tests + group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.01); + +-- Same as above for Error +select 'noisy Error messages', max2((select countOrDefault() from logs where level = 'Error' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.02); + +select 'no Fatal messages', count() from logs where level = 'Fatal'; + + +-- Avoid too noisy messages: limit the number of messages with high frequency +select 'number of too noisy messages', max2(count(), 3) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.10); +select 'number of noisy messages', max2(count(), 10) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.05); + +-- Each message matches its pattern (returns 0 rows) +-- FIXME maybe we should make it stricter ('Code:%Exception: '||s||'%'), but it's not easy because of addMessage +select 'incorrect patterns', max2(countDistinct(message_format_string), 15) from ( + select message_format_string, any(message) as any_message from logs + where message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s) + and message not like ('%Exception: '||s||'%') group by message_format_string +) where any_message not like '%Poco::Exception%'; + +drop table logs; From 28ddee7b58c7d34886a0612671d1c01d218eec8a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 8 Feb 2023 18:34:56 +0000 Subject: [PATCH 075/116] More warning exclusions in poco odbc --- base/poco/Data/ODBC/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/base/poco/Data/ODBC/CMakeLists.txt b/base/poco/Data/ODBC/CMakeLists.txt index 62f0e9f1969..73f2c81ea6c 100644 --- a/base/poco/Data/ODBC/CMakeLists.txt +++ b/base/poco/Data/ODBC/CMakeLists.txt @@ -31,6 +31,8 @@ if (ENABLE_ODBC) -Wno-extra-semi-stmt -Wno-old-style-cast -Wno-sign-compare + -Wno-tautological-constant-out-of-range-compare + -Wno-tautological-unsigned-zero-compare -Wno-unused-parameter -Wno-unused-variable -Wno-zero-as-null-pointer-constant From 4869d3806c29e8706c3d0dd40e44313404ac0e0c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 2 Feb 2023 00:30:49 +0100 Subject: [PATCH 076/116] Add setting check_referential_table_dependencies to check referential dependencies on DROP TABLE. --- src/Core/Settings.h | 1 + src/Databases/DatabaseAtomic.cpp | 6 +- src/Databases/DatabaseMemory.cpp | 7 +- src/Databases/DatabaseOrdinary.cpp | 7 +- src/Databases/TablesLoader.cpp | 27 +++-- src/Databases/TablesLoader.h | 1 + src/Interpreters/DatabaseCatalog.cpp | 102 +++++++++++++----- src/Interpreters/DatabaseCatalog.h | 27 +++-- src/Interpreters/InterpreterCreateQuery.cpp | 6 +- src/Interpreters/InterpreterDropQuery.cpp | 13 +-- src/Interpreters/InterpreterRenameQuery.cpp | 16 ++- src/Interpreters/loadMetadata.cpp | 1 + src/Storages/System/StorageSystemTables.cpp | 4 +- ...k_referential_table_dependencies.reference | 0 ...2_check_referential_table_dependencies.sql | 29 +++++ 15 files changed, 178 insertions(+), 69 deletions(-) create mode 100644 tests/queries/0_stateless/02552_check_referential_table_dependencies.reference create mode 100644 tests/queries/0_stateless/02552_check_referential_table_dependencies.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 63c5d818593..6e085fd27ac 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -641,6 +641,7 @@ class IColumn; \ M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \ M(Bool, check_table_dependencies, true, "Check that DDL query (such as DROP TABLE or RENAME) will not break dependencies", 0) \ + M(Bool, check_referential_table_dependencies, false, "Check that DDL query (such as DROP TABLE or RENAME) will not break referential dependencies", 0) \ M(Bool, use_local_cache_for_remote_storage, true, "Use local cache for remote storage like HDFS or S3, it's used for remote table engine only", 0) \ \ M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (without condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 255c3e1f012..ee4ed319abe 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -522,11 +522,13 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new { /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard - if (query_context->getSettingsRef().check_table_dependencies) + bool check_ref_deps = query_context->getSettingsRef().check_referential_table_dependencies; + bool check_loading_deps = !check_ref_deps && query_context->getSettingsRef().check_table_dependencies; + if (check_ref_deps || check_loading_deps) { std::lock_guard lock(mutex); for (auto & table : tables) - DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed({database_name, table.first}); + DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed({database_name, table.first}, check_ref_deps, check_loading_deps); } try diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 31ef526b408..fda0bbe8032 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -143,8 +144,10 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl applyMetadataChangesToCreateQuery(it->second, metadata); - auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); - DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies); + /// The create query of the table has been just changed, we need to update dependencies too. + auto ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); + auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); + DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies); } std::vector> DatabaseMemory::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 87f91856c1b..49250602132 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -312,8 +313,10 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta out.close(); } - auto new_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); - DatabaseCatalog::instance().updateDependencies(table_id, new_dependencies); + /// The create query of the table has been just changed, we need to update dependencies too. + auto ref_dependencies = getDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); + auto loading_dependencies = getLoadingDependenciesFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); + DatabaseCatalog::instance().updateDependencies(table_id, ref_dependencies, loading_dependencies); commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context); } diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index fbb5b1f17d9..5d66f49554d 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -36,6 +36,7 @@ TablesLoader::TablesLoader(ContextMutablePtr global_context_, Databases database , strictness_mode(strictness_mode_) , referential_dependencies("ReferentialDeps") , loading_dependencies("LoadingDeps") +, all_loading_dependencies("LoadingDeps") { metadata.default_database = global_context->getCurrentDatabase(); log = &Poco::Logger::get("TablesLoader"); @@ -74,7 +75,7 @@ void TablesLoader::loadTables() buildDependencyGraph(); /// Update existing info (it's important for ATTACH DATABASE) - DatabaseCatalog::instance().addDependencies(referential_dependencies); + DatabaseCatalog::instance().addDependencies(referential_dependencies, loading_dependencies); /// Remove tables that do not exist removeUnresolvableDependencies(); @@ -95,18 +96,22 @@ void TablesLoader::buildDependencyGraph() { for (const auto & [table_name, table_metadata] : metadata.parsed_tables) { + auto new_ref_dependencies = getDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast); auto new_loading_dependencies = getLoadingDependenciesFromCreateQuery(global_context, table_name, table_metadata.ast); + if (!new_ref_dependencies.empty()) + referential_dependencies.addDependencies(table_name, new_ref_dependencies); + if (!new_loading_dependencies.empty()) - referential_dependencies.addDependencies(table_name, new_loading_dependencies); + loading_dependencies.addDependencies(table_name, new_loading_dependencies); /// We're adding `new_loading_dependencies` to the graph here even if they're empty because /// we need to have all tables from `metadata.parsed_tables` in the graph. - loading_dependencies.addDependencies(table_name, new_loading_dependencies); + all_loading_dependencies.addDependencies(table_name, new_loading_dependencies); } referential_dependencies.log(); - loading_dependencies.log(); + all_loading_dependencies.log(); } @@ -130,7 +135,7 @@ void TablesLoader::removeUnresolvableDependencies() log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently." "Consider converting it to DDL dictionary.", - fmt::join(loading_dependencies.getDependents(table_id), ", "), + fmt::join(all_loading_dependencies.getDependents(table_id), ", "), table_id); } else @@ -141,12 +146,12 @@ void TablesLoader::removeUnresolvableDependencies() LOG_WARNING( log, "Tables {} depend on {}, but seems like that does not exist. Will ignore it and try to load existing tables", - fmt::join(loading_dependencies.getDependents(table_id), ", "), + fmt::join(all_loading_dependencies.getDependents(table_id), ", "), table_id); } size_t num_dependencies, num_dependents; - loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents); + all_loading_dependencies.getNumberOfAdjacents(table_id, num_dependencies, num_dependents); if (num_dependencies || !num_dependents) throw Exception(ErrorCodes::LOGICAL_ERROR, "Table {} does not have dependencies and dependent tables as it expected to." "It's a bug", table_id); @@ -154,13 +159,13 @@ void TablesLoader::removeUnresolvableDependencies() return true; /// Exclude this dependency. }; - loading_dependencies.removeTablesIf(need_exclude_dependency); + all_loading_dependencies.removeTablesIf(need_exclude_dependency); - if (loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size()) + if (all_loading_dependencies.getNumberOfTables() != metadata.parsed_tables.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of tables to be loaded is not as expected. It's a bug"); /// Cannot load tables with cyclic dependencies. - loading_dependencies.checkNoCyclicDependencies(); + all_loading_dependencies.checkNoCyclicDependencies(); } @@ -172,7 +177,7 @@ void TablesLoader::loadTablesInTopologicalOrder(ThreadPool & pool) load_context->setSetting("cast_ipv4_ipv6_default_on_conversion_error", 1); /// Load tables in parallel. - auto tables_to_load = loading_dependencies.getTablesSortedByDependencyForParallel(); + auto tables_to_load = all_loading_dependencies.getTablesSortedByDependencyForParallel(); for (size_t level = 0; level != tables_to_load.size(); ++level) { diff --git a/src/Databases/TablesLoader.h b/src/Databases/TablesLoader.h index 13d404b96ce..eb07351bd7f 100644 --- a/src/Databases/TablesLoader.h +++ b/src/Databases/TablesLoader.h @@ -69,6 +69,7 @@ private: ParsedTablesMetadata metadata; TablesDependencyGraph referential_dependencies; TablesDependencyGraph loading_dependencies; + TablesDependencyGraph all_loading_dependencies; Poco::Logger * log; std::atomic tables_processed{0}; AtomicStopwatch stopwatch; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index ed8545d6d01..ad5d9d4d325 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -224,6 +224,7 @@ void DatabaseCatalog::shutdownImpl() }) == uuid_map.end()); databases.clear(); referential_dependencies.clear(); + loading_dependencies.clear(); view_dependencies.clear(); } @@ -471,10 +472,13 @@ void DatabaseCatalog::updateDatabaseName(const String & old_name, const String & databases.erase(it); databases.emplace(new_name, db); + /// Update dependencies. for (const auto & table_name : tables_in_database) { - auto dependencies = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true); - referential_dependencies.addDependencies(StorageID{new_name, table_name}, dependencies); + auto removed_ref_deps = referential_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true); + auto removed_loading_deps = loading_dependencies.removeDependencies(StorageID{old_name, table_name}, /* remove_isolated_tables= */ true); + referential_dependencies.addDependencies(StorageID{new_name, table_name}, removed_ref_deps); + loading_dependencies.addDependencies(StorageID{new_name, table_name}, removed_loading_deps); } } @@ -645,6 +649,7 @@ std::unique_ptr DatabaseCatalog::database_catalog; DatabaseCatalog::DatabaseCatalog(ContextMutablePtr global_context_) : WithMutableContext(global_context_) , referential_dependencies{"ReferentialDeps"} + , loading_dependencies{"LoadingDeps"} , view_dependencies{"ViewDeps"} , log(&Poco::Logger::get("DatabaseCatalog")) { @@ -1040,53 +1045,107 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) }); } -void DatabaseCatalog::addDependencies(const StorageID & table_id, const std::vector & dependencies) +void DatabaseCatalog::addDependencies( + const StorageID & table_id, + const std::vector & new_referential_dependencies, + const std::vector & new_loading_dependencies) { + if (new_referential_dependencies.empty() && new_loading_dependencies.empty()) + return; std::lock_guard lock{databases_mutex}; - referential_dependencies.addDependencies(table_id, dependencies); + if (!new_referential_dependencies.empty()) + referential_dependencies.addDependencies(table_id, new_referential_dependencies); + if (!new_loading_dependencies.empty()) + loading_dependencies.addDependencies(table_id, new_loading_dependencies); } -void DatabaseCatalog::addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies) +void DatabaseCatalog::addDependencies( + const QualifiedTableName & table_name, + const TableNamesSet & new_referential_dependencies, + const TableNamesSet & new_loading_dependencies) { + if (new_referential_dependencies.empty() && new_loading_dependencies.empty()) + return; std::lock_guard lock{databases_mutex}; - referential_dependencies.addDependencies(table_name, dependencies); + if (!new_referential_dependencies.empty()) + referential_dependencies.addDependencies(table_name, new_referential_dependencies); + if (!new_loading_dependencies.empty()) + loading_dependencies.addDependencies(table_name, new_loading_dependencies); } -void DatabaseCatalog::addDependencies(const TablesDependencyGraph & extra_graph) +void DatabaseCatalog::addDependencies( + const TablesDependencyGraph & new_referential_dependencies, const TablesDependencyGraph & new_loading_dependencies) { std::lock_guard lock{databases_mutex}; - referential_dependencies.mergeWith(extra_graph); + referential_dependencies.mergeWith(new_referential_dependencies); + loading_dependencies.mergeWith(new_loading_dependencies); } -std::vector DatabaseCatalog::getDependencies(const StorageID & table_id) const +std::vector DatabaseCatalog::getReferentialDependencies(const StorageID & table_id) const { std::lock_guard lock{databases_mutex}; return referential_dependencies.getDependencies(table_id); } -std::vector DatabaseCatalog::getDependents(const StorageID & table_id) const +std::vector DatabaseCatalog::getReferentialDependents(const StorageID & table_id) const { std::lock_guard lock{databases_mutex}; return referential_dependencies.getDependents(table_id); } -std::vector DatabaseCatalog::removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database) +std::vector DatabaseCatalog::getLoadingDependencies(const StorageID & table_id) const { std::lock_guard lock{databases_mutex}; - if (check_dependencies) - checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database); - return referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); + return loading_dependencies.getDependencies(table_id); } -void DatabaseCatalog::checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database) const +std::vector DatabaseCatalog::getLoadingDependents(const StorageID & table_id) const { std::lock_guard lock{databases_mutex}; - return checkTableCanBeRemovedOrRenamedUnlocked(table_id, is_drop_database); + return loading_dependencies.getDependents(table_id); } -void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const +std::pair, std::vector> DatabaseCatalog::removeDependencies( + const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) { - const auto & dependents = referential_dependencies.getDependents(removing_table); + std::lock_guard lock{databases_mutex}; + checkTableCanBeRemovedOrRenamedUnlocked(table_id, check_referential_dependencies, check_loading_dependencies, is_drop_database); + return {referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true), + loading_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true)}; +} + +void DatabaseCatalog::updateDependencies( + const StorageID & table_id, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies) +{ + std::lock_guard lock{databases_mutex}; + referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); + loading_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); + if (!new_referential_dependencies.empty()) + referential_dependencies.addDependencies(table_id, new_referential_dependencies); + if (!new_loading_dependencies.empty()) + loading_dependencies.addDependencies(table_id, new_loading_dependencies); +} + +void DatabaseCatalog::checkTableCanBeRemovedOrRenamed( + const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const +{ + if (!check_referential_dependencies && !check_loading_dependencies) + return; + std::lock_guard lock{databases_mutex}; + return checkTableCanBeRemovedOrRenamedUnlocked(table_id, check_referential_dependencies, check_loading_dependencies, is_drop_database); +} + +void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked( + const StorageID & removing_table, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const +{ + chassert(!check_referential_dependencies || !check_loading_dependencies); /// These flags must not be both set. + std::vector dependents; + if (check_referential_dependencies) + dependents = referential_dependencies.getDependents(removing_table); + else if (check_loading_dependencies) + dependents = loading_dependencies.getDependents(removing_table); + else + return; if (!is_drop_database) { @@ -1108,13 +1167,6 @@ void DatabaseCatalog::checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, fmt::join(from_other_databases, ", ")); } -void DatabaseCatalog::updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies) -{ - std::lock_guard lock{databases_mutex}; - referential_dependencies.removeDependencies(table_id, /* remove_isolated_tables= */ true); - referential_dependencies.addDependencies(table_id, new_dependencies); -} - void DatabaseCatalog::cleanupStoreDirectoryTask() { for (const auto & [disk_name, disk] : getContext()->getDisksMap()) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 5dc3f90b7f4..ba3625626da 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -220,17 +220,19 @@ public: /// Referential dependencies between tables: table "A" depends on table "B" /// if "B" is referenced in the definition of "A". - void addDependencies(const StorageID & table_id, const std::vector & dependencies); - void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & dependencies); - void addDependencies(const TablesDependencyGraph & extra_graph); - std::vector removeDependencies(const StorageID & table_id, bool check_dependencies, bool is_drop_database = false); + /// Loading dependencies were used to check whether a table can be removed before we had those referential dependencies. + /// Now we support this mode (see `check_table_referential_dependencies` in Setting.h) for compatibility. + void addDependencies(const StorageID & table_id, const std::vector & new_referential_dependencies, const std::vector & new_loading_dependencies); + void addDependencies(const QualifiedTableName & table_name, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies); + void addDependencies(const TablesDependencyGraph & new_referential_dependencies, const TablesDependencyGraph & new_loading_dependencies); + std::pair, std::vector> removeDependencies(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false); + std::vector getReferentialDependencies(const StorageID & table_id) const; + std::vector getReferentialDependents(const StorageID & table_id) const; + std::vector getLoadingDependencies(const StorageID & table_id) const; + std::vector getLoadingDependents(const StorageID & table_id) const; + void updateDependencies(const StorageID & table_id, const TableNamesSet & new_referential_dependencies, const TableNamesSet & new_loading_dependencies); - std::vector getDependencies(const StorageID & table_id) const; - std::vector getDependents(const StorageID & table_id) const; - - void updateDependencies(const StorageID & table_id, const TableNamesSet & new_dependencies); - - void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool is_drop_database = false) const; + void checkTableCanBeRemovedOrRenamed(const StorageID & table_id, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database = false) const; private: // The global instance of database catalog. unique_ptr is to allow @@ -244,7 +246,7 @@ private: void shutdownImpl(); - void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool is_drop_database) const TSA_REQUIRES(databases_mutex); + void checkTableCanBeRemovedOrRenamedUnlocked(const StorageID & removing_table, bool check_referential_dependencies, bool check_loading_dependencies, bool is_drop_database) const TSA_REQUIRES(databases_mutex); struct UUIDToStorageMapPart { @@ -287,6 +289,9 @@ private: /// if the table "B" is referenced in the definition of the table "A". TablesDependencyGraph referential_dependencies TSA_GUARDED_BY(databases_mutex); + /// Loading dependencies were used to check whether a table can be removed before we had referential dependencies. + TablesDependencyGraph loading_dependencies TSA_GUARDED_BY(databases_mutex); + /// View dependencies between a source table and its view. TablesDependencyGraph view_dependencies TSA_GUARDED_BY(databases_mutex); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e1a2c5775b7..7e5a4688034 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1243,9 +1243,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// If table has dependencies - add them to the graph QualifiedTableName qualified_name{database_name, create.getTable()}; - TableNamesSet dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); - if (!dependencies.empty()) - DatabaseCatalog::instance().addDependencies(qualified_name, dependencies); + auto ref_dependencies = getDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); + auto loading_dependencies = getLoadingDependenciesFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); + DatabaseCatalog::instance().addDependencies(qualified_name, ref_dependencies, loading_dependencies); return fillTableIfNeeded(create); } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index e4bf22730b3..e2484a48da4 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -187,8 +187,9 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue if (query.permanently) { /// Server may fail to restart of DETACH PERMANENTLY if table has dependent ones - DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, - is_drop_or_detach_database); + bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies; + bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies; + DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database); /// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart database->detachTablePermanently(context_, table_id.table_name); } @@ -233,8 +234,9 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue table->checkTableCanBeDropped(); /// Check dependencies before shutting table down - if (context_->getSettingsRef().check_table_dependencies) - DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed(table_id, is_drop_or_detach_database); + bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies; + bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies; + DatabaseCatalog::instance().checkTableCanBeRemovedOrRenamed(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database); table->flushAndShutdown(); @@ -242,8 +244,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue if (database->getUUID() == UUIDHelpers::Nil) table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); - DatabaseCatalog::instance().removeDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, - is_drop_or_detach_database); + DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database); database->dropTable(context_, table_id.table_name, query.sync); /// We have to drop mmapio cache when dropping table from Ordinary database diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 8cf39d2c850..3dfdd867ae7 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -124,10 +124,17 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c } else { - std::vector dependencies; + StorageID from_table_id{elem.from_database_name, elem.from_table_name}; + StorageID to_table_id{elem.to_database_name, elem.to_table_name}; + std::vector ref_dependencies; + std::vector loading_dependencies; + if (!exchange_tables) - dependencies = database_catalog.removeDependencies(StorageID(elem.from_database_name, elem.from_table_name), - getContext()->getSettingsRef().check_table_dependencies); + { + bool check_ref_deps = getContext()->getSettingsRef().check_referential_table_dependencies; + bool check_loading_deps = !check_ref_deps && getContext()->getSettingsRef().check_table_dependencies; + std::tie(ref_dependencies, loading_dependencies) = database_catalog.removeDependencies(from_table_id, check_ref_deps, check_loading_deps); + } database->renameTable( getContext(), @@ -137,8 +144,7 @@ BlockIO InterpreterRenameQuery::executeToTables(const ASTRenameQuery & rename, c exchange_tables, rename.dictionary); - if (!dependencies.empty()) - DatabaseCatalog::instance().addDependencies(StorageID(elem.to_database_name, elem.to_table_name), dependencies); + DatabaseCatalog::instance().addDependencies(to_table_id, ref_dependencies, loading_dependencies); } } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index fa87e66e196..837d0f1789d 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -348,6 +348,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons InterpreterSystemQuery::startStopActionInDatabase(action, /* start */ false, database_name, database, context, log); local_context->setSetting("check_table_dependencies", false); + local_context->setSetting("check_referential_table_dependencies", false); convertOrdinaryDatabaseToAtomic(log, local_context, database, database_name, tmp_name); LOG_INFO(log, "Will start background operations after renaming tables in database {}", database_name); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 71d465dfb1b..532d1ba5e64 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -491,8 +491,8 @@ protected: if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2] || columns_mask[src_index + 3]) { - auto dependencies = DatabaseCatalog::instance().getDependencies(StorageID{database_name, table_name}); - auto dependents = DatabaseCatalog::instance().getDependents(StorageID{database_name, table_name}); + auto dependencies = DatabaseCatalog::instance().getLoadingDependencies(StorageID{database_name, table_name}); + auto dependents = DatabaseCatalog::instance().getLoadingDependents(StorageID{database_name, table_name}); Array dependencies_databases; Array dependencies_tables; diff --git a/tests/queries/0_stateless/02552_check_referential_table_dependencies.reference b/tests/queries/0_stateless/02552_check_referential_table_dependencies.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02552_check_referential_table_dependencies.sql b/tests/queries/0_stateless/02552_check_referential_table_dependencies.sql new file mode 100644 index 00000000000..e83006eca5f --- /dev/null +++ b/tests/queries/0_stateless/02552_check_referential_table_dependencies.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS mv; +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; + +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv TO dst AS SELECT x FROM src; + +SET check_referential_table_dependencies = 1; + +-- Can't drop because of referential dependencies +DROP TABLE src; -- { serverError HAVE_DEPENDENT_OBJECTS } +DROP TABLE dst; -- { serverError HAVE_DEPENDENT_OBJECTS } + +-- Ok to drop in the correct order +DROP TABLE mv; +DROP TABLE src; +DROP TABLE dst; + +-- Check again with check_referential_table_dependencies = 0 +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv TO dst AS SELECT x FROM src; + +SET check_referential_table_dependencies = 0; + +DROP TABLE src; +DROP TABLE dst; +DROP TABLE mv; From 8392b35d5062eb4b62b3041fbb01f58885155d0d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 04:39:12 +0100 Subject: [PATCH 077/116] Speed up countDigits --- base/base/itoa.h | 68 ++++++++++++++++++----------------- src/Functions/countDigits.cpp | 59 +++++++++--------------------- 2 files changed, 51 insertions(+), 76 deletions(-) diff --git a/base/base/itoa.h b/base/base/itoa.h index dd3e3cc96fe..513070c99d9 100644 --- a/base/base/itoa.h +++ b/base/base/itoa.h @@ -33,6 +33,41 @@ #include +template +inline int digits10(T x) +{ + if (x < 10ULL) + return 1; + if (x < 100ULL) + return 2; + if (x < 1000ULL) + return 3; + + if (x < 1000000000000ULL) + { + if (x < 100000000ULL) + { + if (x < 1000000ULL) + { + if (x < 10000ULL) + return 4; + else + return 5 + (x >= 100000ULL); + } + + return 7 + (x >= 10000000ULL); + } + + if (x < 10000000000ULL) + return 9 + (x >= 1000000000ULL); + + return 11 + (x >= 100000000000ULL); + } + + return 12 + digits10(x / 1000000000000ULL); +} + + namespace impl { @@ -312,39 +347,6 @@ namespace convert } } -template -static inline int digits10(T x) -{ - if (x < 10ULL) - return 1; - if (x < 100ULL) - return 2; - if (x < 1000ULL) - return 3; - - if (x < 1000000000000ULL) - { - if (x < 100000000ULL) - { - if (x < 1000000ULL) - { - if (x < 10000ULL) - return 4; - else - return 5 + (x >= 100000ULL); - } - - return 7 + (x >= 10000000ULL); - } - - if (x < 10000000000ULL) - return 9 + (x >= 1000000000ULL); - - return 11 + (x >= 100000000000ULL); - } - - return 12 + digits10(x / 1000000000000ULL); -} template static inline char * writeUIntText(T x, char * p) diff --git a/src/Functions/countDigits.cpp b/src/Functions/countDigits.cpp index aefe0d92d94..2ca8d944b0a 100644 --- a/src/Functions/countDigits.cpp +++ b/src/Functions/countDigits.cpp @@ -2,9 +2,10 @@ #include #include #include -#include #include #include +#include +#include namespace DB @@ -83,7 +84,7 @@ private: template static void execute(const ColVecType & col, ColumnUInt8 & result_column, size_t rows_count) { - using NativeT = NativeType; + using NativeT = make_unsigned_t>; const auto & src_data = col.getData(); auto & dst_data = result_column.getData(); @@ -92,50 +93,22 @@ private: for (size_t i = 0; i < rows_count; ++i) { if constexpr (is_decimal) - dst_data[i] = digits(src_data[i].value); - else - dst_data[i] = digits(src_data[i]); - } - } - - template - static UInt32 digits(T value) - { - static_assert(!is_decimal); - using DivT = std::conditional_t, Int32, UInt32>; - - UInt32 res = 0; - T tmp; - - if constexpr (sizeof(T) > sizeof(Int32)) - { - static constexpr const DivT e9 = 1000000000; - - tmp = value / e9; - while (tmp != 0) { - value = tmp; - tmp /= e9; - res += 9; + auto value = src_data[i].value; + if (unlikely(value < 0)) + dst_data[i] = digits10(-static_cast(value)); + else + dst_data[i] = digits10(value); + } + else + { + auto value = src_data[i]; + if (unlikely(value < 0)) + dst_data[i] = digits10(-static_cast(value)); + else + dst_data[i] = digits10(value); } } - - static constexpr const DivT e3 = 1000; - - tmp = value / e3; - while (tmp != 0) - { - value = tmp; - tmp /= e3; - res += 3; - } - - while (value != 0) - { - value /= 10; - ++res; - } - return res; } }; From e3fc7662c1247e9e6c22b1847cfbddd190fca984 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 04:51:01 +0100 Subject: [PATCH 078/116] Add test --- .../0_stateless/02560_count_digits.reference | 19 ++++++++++++++++ .../0_stateless/02560_count_digits.sql | 22 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/02560_count_digits.reference create mode 100644 tests/queries/0_stateless/02560_count_digits.sql diff --git a/tests/queries/0_stateless/02560_count_digits.reference b/tests/queries/0_stateless/02560_count_digits.reference new file mode 100644 index 00000000000..836975f7db4 --- /dev/null +++ b/tests/queries/0_stateless/02560_count_digits.reference @@ -0,0 +1,19 @@ +1 +1 +1 +5 +5 +20 +19 +39 +78 +39 +77 +8 +9 +10 +11 +8 +9 +10 +11 diff --git a/tests/queries/0_stateless/02560_count_digits.sql b/tests/queries/0_stateless/02560_count_digits.sql new file mode 100644 index 00000000000..19f5403bdf5 --- /dev/null +++ b/tests/queries/0_stateless/02560_count_digits.sql @@ -0,0 +1,22 @@ +SELECT countDigits(0); +SELECT countDigits(1); +SELECT countDigits(-1); +SELECT countDigits(12345); +SELECT countDigits(-12345); +SELECT countDigits(0xFFFFFFFFFFFFFFFF); +SELECT countDigits(CAST(0x8000000000000000 AS Int64)); +SELECT countDigits(CAST(-1 AS UInt128)); +SELECT countDigits(CAST(-1 AS UInt256)); +SELECT countDigits(CAST(CAST(-1 AS UInt128) DIV 2 + 1 AS Int128)); +SELECT countDigits(CAST(CAST(-1 AS UInt256) DIV 2 + 1 AS Int256)); + +SELECT countDigits(-123.45678::Decimal32(5)); +SELECT countDigits(-123.456789::Decimal64(6)); +SELECT countDigits(-123.4567890::Decimal128(7)); +SELECT countDigits(-123.45678901::Decimal256(8)); + +-- this behavior can be surprising, but actually reasonable: +SELECT countDigits(-123.456::Decimal32(5)); +SELECT countDigits(-123.4567::Decimal64(6)); +SELECT countDigits(-123.45678::Decimal128(7)); +SELECT countDigits(-123.456789::Decimal256(8)); From 9a86d0087c229482a70fab6a1872f3e1f4e52f42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 04:52:33 +0100 Subject: [PATCH 079/116] Add performance test --- tests/performance/countDigits.xml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 tests/performance/countDigits.xml diff --git a/tests/performance/countDigits.xml b/tests/performance/countDigits.xml new file mode 100644 index 00000000000..5e822b3c4fd --- /dev/null +++ b/tests/performance/countDigits.xml @@ -0,0 +1,6 @@ + + SELECT count() FROM test.hits WHERE NOT ignore(countDigits(RegionID)) + SELECT count() FROM test.hits WHERE NOT ignore(countDigits(AdvEngineID)) + SELECT count() FROM test.hits WHERE NOT ignore(countDigits(ClientIP)) + SELECT count() FROM test.hits WHERE NOT ignore(countDigits(WatchID)) + From 78446b46d993252027552626204311de4de30d36 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 9 Feb 2023 07:29:51 +0300 Subject: [PATCH 080/116] Return note about function name in preciseExp10.h --- base/base/preciseExp10.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/base/base/preciseExp10.h b/base/base/preciseExp10.h index 7e16aa25d24..051c07cc2e2 100644 --- a/base/base/preciseExp10.h +++ b/base/base/preciseExp10.h @@ -3,7 +3,9 @@ /** exp10 from GNU libm fails to give precise result for integer arguments. * For example, exp10(3) gives 1000.0000000000001 * despite the fact that 1000 is exactly representable in double and float. - * Better to always use our own implementation based on MUSL implementation. + * Better to always use our own implementation based on a MUSL's one. + * + * Note: the function name is different to avoid confusion with symbols from the system libm. */ double preciseExp10(double x); From a40ef2bf183829166e4bbd15115a0acbe8f8cfcc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 08:09:19 +0100 Subject: [PATCH 081/116] Make it better --- src/Processors/Transforms/FillingTransform.cpp | 17 ++++++++++------- .../0_stateless/02560_with_fill_int256_int.sql | 4 ++-- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index c569dff15c1..e3bf12ea525 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -27,7 +27,7 @@ Block FillingTransform::transformHeader(Block header, const SortDescription & so /// Columns which are not from sorting key may not be constant anymore. for (auto & column : header) if (column.column && isColumnConst(*column.column) && !sort_keys.contains(column.name)) - column.column = column.type->createColumn(); + column.column = column.column->convertToFullColumnIfConst(); return header; } @@ -76,12 +76,12 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & if (which.isInt128() || which.isUInt128()) { max_type = Field::Types::Int128; - to_type = std::make_shared(); + to_type = type; } else if (which.isInt256() || which.isUInt256()) { max_type = Field::Types::Int256; - to_type = std::make_shared(); + to_type = type; } else if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) { @@ -109,9 +109,12 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & || descr.fill_step.getType() > max_type) return false; - descr.fill_from = convertFieldToType(descr.fill_from, *to_type); - descr.fill_to = convertFieldToType(descr.fill_to, *to_type); - descr.fill_step = convertFieldToType(descr.fill_step, *to_type); + if (!descr.fill_from.isNull()) + descr.fill_from = convertFieldToTypeOrThrow(descr.fill_from, *to_type); + if (!descr.fill_to.isNull()) + descr.fill_to = convertFieldToTypeOrThrow(descr.fill_to, *to_type); + if (!descr.fill_step.isNull()) + descr.fill_step = convertFieldToTypeOrThrow(descr.fill_step, *to_type); if (descr.step_kind) { @@ -201,7 +204,7 @@ FillingTransform::FillingTransform( throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, "Incompatible types of WITH FILL expression values with column type {}", type->getName()); - if (type->isValueRepresentedByUnsignedInteger() && + if (isUnsignedInteger(type) && ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) { diff --git a/tests/queries/0_stateless/02560_with_fill_int256_int.sql b/tests/queries/0_stateless/02560_with_fill_int256_int.sql index a8876629ef2..2039f7ec233 100644 --- a/tests/queries/0_stateless/02560_with_fill_int256_int.sql +++ b/tests/queries/0_stateless/02560_with_fill_int256_int.sql @@ -5,5 +5,5 @@ SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM 3 TO SELECT (number * 2)::Int128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; SELECT (number * 2)::Int256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -SELECT (number * 2)::UInt128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 475 } -SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 475 } +SELECT (number * 2)::UInt128 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 69 } +SELECT (number * 2)::UInt256 FROM numbers(10) ORDER BY 1 ASC WITH FILL FROM -3 TO 5; -- { serverError 69 } From 327be4dcfc6c17c647e384ac1f531d77ab086bd0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 08:14:36 +0100 Subject: [PATCH 082/116] Add a test for #30421 --- .../02561_with_fill_date_datetime_incompatible.reference | 0 .../0_stateless/02561_with_fill_date_datetime_incompatible.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.reference create mode 100644 tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.sql diff --git a/tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.reference b/tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.sql b/tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.sql new file mode 100644 index 00000000000..458e5047a63 --- /dev/null +++ b/tests/queries/0_stateless/02561_with_fill_date_datetime_incompatible.sql @@ -0,0 +1,2 @@ +SELECT today() AS a +ORDER BY a ASC WITH FILL FROM now() - toIntervalMonth(1) TO now() + toIntervalDay(1) STEP 82600; -- { serverError 475 } From ad2dd144c4e10a113102546b45cc787299aed2e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 08:22:40 +0100 Subject: [PATCH 083/116] Fix test --- tests/queries/0_stateless/01458_count_digits.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01458_count_digits.reference b/tests/queries/0_stateless/01458_count_digits.reference index 80c210bc0af..93cb52f7352 100644 --- a/tests/queries/0_stateless/01458_count_digits.reference +++ b/tests/queries/0_stateless/01458_count_digits.reference @@ -1,7 +1,7 @@ -0 2 2 0 2 3 0 2 4 +1 2 2 1 2 3 1 2 4 2 3 4 10 10 19 19 39 39 2 2 2 2 2 2 2 2 2 2 2 2 -0 0 0 0 0 0 0 0 0 0 0 0 +1 1 1 1 1 1 1 1 1 1 1 1 3 3 3 5 5 5 10 10 10 19 19 20 2 3 4 5 6 7 From cb847172ea0425da5d750a9e7a8d20ff3307810c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 08:39:46 +0100 Subject: [PATCH 084/116] Fix something --- src/Processors/Transforms/FillingTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index e3bf12ea525..a73ace852c1 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -61,7 +61,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { WhichDataType which_from(descr.fill_from_type); if ((which_from.isDateOrDate32() || which_from.isDateTime() || which_from.isDateTime64()) && - !descr.fill_from_type->equals(*type)) + !descr.fill_from_type->equals(*removeNullable(type))) return false; } @@ -198,7 +198,7 @@ FillingTransform::FillingTransform( auto & descr = filling_row.getFillDescription(i); const Block & output_header = getOutputPort().getHeader(); - const DataTypePtr & type = output_header.getByPosition(block_position).type; + const DataTypePtr & type = removeNullable(output_header.getByPosition(block_position).type); if (!tryConvertFields(descr, type)) throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, @@ -349,8 +349,8 @@ void FillingTransform::transform(Chunk & chunk) interpolate(); while (filling_row.next(next_row)) { - insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); - interpolate(); + insertFromFillingRow(res_fill_columns, res_interpolate_columns, res_other_columns, filling_row, interpolate_block); + interpolate(); } setResultColumns(chunk, res_fill_columns, res_interpolate_columns, res_other_columns); From 3a75ede60a9f4dfebb442978d091f8b24e3b5f23 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 08:41:47 +0100 Subject: [PATCH 085/116] Implement #16733 --- src/Processors/Transforms/FillingTransform.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index a73ace852c1..bfee5ed8bf9 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -366,7 +366,7 @@ void FillingTransform::transform(Chunk & chunk) if (first) { - for (size_t i = 0; i < filling_row.size(); ++i) + for (size_t i = 0, size = filling_row.size(); i < size; ++i) { auto current_value = (*old_fill_columns[i])[0]; const auto & fill_from = filling_row.getFillDescription(i).fill_from; From ae731b739acb40de487cd28a5c4a4caa015904fb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 9 Feb 2023 08:13:16 +0000 Subject: [PATCH 086/116] Prefer explicitly defined creds for S3 --- src/IO/S3Common.cpp | 9 +- tests/integration/test_storage_s3/test.py | 8 +- .../test_invalid_env_credentials.py | 126 ++++++++++++++++++ 3 files changed, 140 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_storage_s3/test_invalid_env_credentials.py diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 50fee97870d..6a696f6c989 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -599,7 +599,13 @@ public: { auto * logger = &Poco::Logger::get("S3CredentialsProviderChain"); - if (use_environment_credentials) + /// add explicit credentials to the front of the chain + /// because it's manually defined by the user + if (!credentials.IsEmpty()) + { + AddProvider(std::make_shared(credentials)); + } + else if (use_environment_credentials) { static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; @@ -693,7 +699,6 @@ public: } } - AddProvider(std::make_shared(credentials)); /// Quite verbose provider (argues if file with credentials doesn't exist) so iut's the last one /// in chain. AddProvider(std::make_shared()); diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ac6eee11892..002f587113b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1699,7 +1699,6 @@ def test_ast_auth_headers(started_cluster): def test_environment_credentials(started_cluster): - filename = "test.csv" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["s3_with_environment_credentials"] @@ -1713,6 +1712,13 @@ def test_environment_credentials(started_cluster): ).strip() ) + # manually defined access key should override from env + with pytest.raises(helpers.client.QueryRuntimeException) as ei: + instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', 'aws', 'aws123')") + + assert ei.value.returncode == 243 + assert "HTTP response code: 403" in ei.value.stderr def test_s3_list_objects_failure(started_cluster): bucket = started_cluster.minio_bucket diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py new file mode 100644 index 00000000000..b16b8f4485a --- /dev/null +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -0,0 +1,126 @@ +import json +import os +import logging + +import helpers.client +from helpers.mock_servers import start_mock_servers +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +MINIO_INTERNAL_PORT = 9001 + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +# Creates S3 bucket for tests and allows anonymous read-write access to it. +def prepare_s3_bucket(started_cluster): + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + + started_cluster.minio_restricted_bucket = "{}-with-auth".format( + started_cluster.minio_bucket + ) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) + + minio_client.make_bucket(started_cluster.minio_restricted_bucket) + +def run_s3_mocks(started_cluster): + script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") + start_mock_servers( + started_cluster, + script_dir, + [ + ("mock_s3.py", "resolver", "8080"), + ("unstable_server.py", "resolver", "8081"), + ("echo.py", "resolver", "8082"), + ("no_list_objects.py", "resolver", "8083"), + ], + ) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "s3_with_invalid_environment_credentials", + with_minio=True, + env_variables={ + "AWS_ACCESS_KEY_ID": "aws", + "AWS_SECRET_ACCESS_KEY": "aws123", + }, + main_configs=["configs/use_environment_credentials.xml"], + ) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + run_s3_mocks(cluster) + + yield cluster + finally: + cluster.shutdown() + +def test_with_invalid_environment_credentials(started_cluster): + auth = "'minio','minio123'" + bucket = started_cluster.minio_restricted_bucket + + instance = started_cluster.instances["s3_with_invalid_environment_credentials"] + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth}) select * from numbers(100) settings s3_truncate_on_insert=1" + ) + + with pytest.raises(helpers.client.QueryRuntimeException) as ei: + instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')") + + assert ei.value.returncode == 243 + assert "HTTP response code: 403" in ei.value.stderr + + assert ( + "100" + == instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth})" + ).strip() + ) + + From 2f9ad860bd7e8ce59a996c762f855b33699f5473 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 9 Feb 2023 08:26:36 +0000 Subject: [PATCH 087/116] Automatic style fix --- tests/integration/test_storage_s3/test.py | 4 +++- .../test_storage_s3/test_invalid_env_credentials.py | 8 +++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 002f587113b..54944b56919 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1715,11 +1715,13 @@ def test_environment_credentials(started_cluster): # manually defined access key should override from env with pytest.raises(helpers.client.QueryRuntimeException) as ei: instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', 'aws', 'aws123')") + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', 'aws', 'aws123')" + ) assert ei.value.returncode == 243 assert "HTTP response code: 403" in ei.value.stderr + def test_s3_list_objects_failure(started_cluster): bucket = started_cluster.minio_bucket instance = started_cluster.instances["dummy"] # type: ClickHouseInstance diff --git a/tests/integration/test_storage_s3/test_invalid_env_credentials.py b/tests/integration/test_storage_s3/test_invalid_env_credentials.py index b16b8f4485a..2f5d9349904 100644 --- a/tests/integration/test_storage_s3/test_invalid_env_credentials.py +++ b/tests/integration/test_storage_s3/test_invalid_env_credentials.py @@ -61,6 +61,7 @@ def prepare_s3_bucket(started_cluster): minio_client.make_bucket(started_cluster.minio_restricted_bucket) + def run_s3_mocks(started_cluster): script_dir = os.path.join(os.path.dirname(__file__), "s3_mocks") start_mock_servers( @@ -74,6 +75,7 @@ def run_s3_mocks(started_cluster): ], ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -100,6 +102,7 @@ def started_cluster(): finally: cluster.shutdown() + def test_with_invalid_environment_credentials(started_cluster): auth = "'minio','minio123'" bucket = started_cluster.minio_restricted_bucket @@ -111,7 +114,8 @@ def test_with_invalid_environment_credentials(started_cluster): with pytest.raises(helpers.client.QueryRuntimeException) as ei: instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')") + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl')" + ) assert ei.value.returncode == 243 assert "HTTP response code: 403" in ei.value.stderr @@ -122,5 +126,3 @@ def test_with_invalid_environment_credentials(started_cluster): f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_cache4.jsonl', {auth})" ).strip() ) - - From 4c7f30ef56882a83ee66cfcb503fb706bafc8337 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 11:55:28 +0300 Subject: [PATCH 088/116] Update preciseExp10.cpp --- base/base/preciseExp10.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/base/base/preciseExp10.cpp b/base/base/preciseExp10.cpp index 53d6ebd9f1b..1cd660dc569 100644 --- a/base/base/preciseExp10.cpp +++ b/base/base/preciseExp10.cpp @@ -1,8 +1,7 @@ -/* - Origin: musl-libc /src/math/exp10.c - https://www.musl-libc.org/ - http://git.musl-libc.org/cgit/musl/tree/src/math/exp10.c -*/ +/** Origin: musl-libc /src/math/exp10.c + * https://www.musl-libc.org/ + * http://git.musl-libc.org/cgit/musl/tree/src/math/exp10.c + */ /* musl as a whole is licensed under the following standard MIT license: From 1c553a2201a1561725675e9628ecfe88b7dd8eb6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 9 Feb 2023 09:55:23 +0000 Subject: [PATCH 089/116] Fix ARM and PPC builds --- base/poco/Foundation/CMakeLists.txt | 1 + base/poco/XML/CMakeLists.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 0dcc954cb98..dceb18e68cc 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -207,6 +207,7 @@ target_compile_options (_poco_foundation -Wno-sign-compare -Wno-suggest-destructor-override -Wno-suggest-override + -Wno-tautological-unsigned-zero-compare -Wno-thread-safety-analysis -Wno-thread-safety-negative -Wno-undef diff --git a/base/poco/XML/CMakeLists.txt b/base/poco/XML/CMakeLists.txt index 102da8f1f5f..4b875e7071e 100644 --- a/base/poco/XML/CMakeLists.txt +++ b/base/poco/XML/CMakeLists.txt @@ -39,6 +39,7 @@ target_compile_options (_poco_xml -Wno-suggest-destructor-override -Wno-suggest-override -Wno-tautological-type-limit-compare + -Wno-tautological-unsigned-zero-compare -Wno-unreachable-code -Wno-unused-macros -Wno-unused-parameter From 65ec9bd7d02ce9146249cabcea0e951aae89a04b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Feb 2023 10:58:49 +0000 Subject: [PATCH 090/116] Fix write buffer destruction order for vertical merge. --- src/Storages/MergeTree/MergeTask.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 46af2e1563a..d3549bcc2ff 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -259,9 +259,9 @@ private: struct VerticalMergeRuntimeContext : public IStageRuntimeContext //-V730 { /// Begin dependencies from previous stage - std::unique_ptr rows_sources_write_buf{nullptr}; - std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; std::unique_ptr rows_sources_file; + std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; + std::unique_ptr rows_sources_write_buf{nullptr}; std::optional column_sizes; CompressionCodecPtr compression_codec; DiskPtr tmp_disk{nullptr}; From ffc97908ef278678e3b713817cdb5e1251464622 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Thu, 9 Feb 2023 15:12:21 +0100 Subject: [PATCH 091/116] increase a time gap between insert and ttl move --- tests/integration/test_merge_tree_s3_failover/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index d4c691fdb55..c61cacc9d8c 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -182,7 +182,7 @@ def test_move_failover(cluster): data String ) ENGINE=MergeTree() ORDER BY id - TTL dt + INTERVAL 3 SECOND TO VOLUME 'external' + TTL dt + INTERVAL 4 SECOND TO VOLUME 'external' SETTINGS storage_policy='s3_cold' """ ) @@ -191,7 +191,7 @@ def test_move_failover(cluster): fail_request(cluster, 1) node.query( - "INSERT INTO s3_failover_test VALUES (now() - 2, 0, 'data'), (now() - 2, 1, 'data')" + "INSERT INTO s3_failover_test VALUES (now() - 1, 0, 'data'), (now() - 1, 1, 'data')" ) # Wait for part move to S3. From 612fe0bd00c12ae33b038cea5ef0b2e182a71287 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Feb 2023 15:42:46 +0100 Subject: [PATCH 092/116] Revert "Allow vertical merges from compact to wide parts" --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + src/Storages/MergeTree/MergeTask.cpp | 8 ++-- .../MergeTree/MergeTreeDataPartWide.h | 2 + .../MergeTreeDataPartWriterOnDisk.cpp | 13 ++---- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 10 ++++- ...539_vertical_merge_compact_parts.reference | 2 - .../02539_vertical_merge_compact_parts.sql | 41 ------------------- 7 files changed, 19 insertions(+), 59 deletions(-) delete mode 100644 tests/queries/0_stateless/02539_vertical_merge_compact_parts.reference delete mode 100644 tests/queries/0_stateless/02539_vertical_merge_compact_parts.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ea1fd209a20..9d0252bd625 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -110,6 +110,8 @@ public: virtual bool isStoredOnRemoteDiskWithZeroCopySupport() const = 0; + virtual bool supportsVerticalMerge() const { return false; } + /// NOTE: Returns zeros if column files are not found in checksums. /// Otherwise return information about column size on disk. ColumnSize getColumnSize(const String & column_name) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 02165d244ff..5874c257ad0 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -953,10 +953,10 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm return MergeAlgorithm::Horizontal; if (ctx->need_remove_expired_values) return MergeAlgorithm::Horizontal; - if (global_ctx->future_part->part_format.part_type != MergeTreeDataPartType::Wide) - return MergeAlgorithm::Horizontal; - if (global_ctx->future_part->part_format.storage_type != MergeTreeDataPartStorageType::Full) - return MergeAlgorithm::Horizontal; + + for (const auto & part : global_ctx->future_part->parts) + if (!part->supportsVerticalMerge() || !isFullPartStorage(part->getDataPartStorage())) + return MergeAlgorithm::Horizontal; bool is_supported_storage = ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary || diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 3f0d52d831e..0d68334a623 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -47,6 +47,8 @@ public: bool isStoredOnRemoteDiskWithZeroCopySupport() const override; + bool supportsVerticalMerge() const override { return true; } + String getFileNameForColumn(const NameAndTypePair & column) const override; ~MergeTreeDataPartWide() override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 1dec7c2cd7c..1f40177d0fa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -135,7 +135,6 @@ static size_t computeIndexGranularityImpl( { size_t rows_in_block = block.rows(); size_t index_granularity_for_block; - if (!can_use_adaptive_index_granularity) { index_granularity_for_block = fixed_index_granularity_rows; @@ -144,9 +143,7 @@ static size_t computeIndexGranularityImpl( { size_t block_size_in_memory = block.bytes(); if (blocks_are_granules) - { index_granularity_for_block = rows_in_block; - } else if (block_size_in_memory >= index_granularity_bytes) { size_t granules_in_block = block_size_in_memory / index_granularity_bytes; @@ -158,14 +155,10 @@ static size_t computeIndexGranularityImpl( index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes; } } + /// We should be less or equal than fixed index granularity + index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block); - /// We should be less or equal than fixed index granularity. - /// But if block size is a granule size then do not adjust it. - /// Granularity greater than fixed granularity might come from compact part. - if (!blocks_are_granules) - index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block); - - /// Very rare case when index granularity bytes less than single row. + /// very rare case when index granularity bytes less then single row if (index_granularity_for_block == 0) index_granularity_for_block = 1; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f9fe6f2c8ab..cce459c1ba8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -484,8 +484,15 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows); } - if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) + if (index_granularity_rows > data_part->index_granularity_info.fixed_index_granularity) { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Mark #{} has {} rows, but max fixed granularity is {}, index granularity size {}", + mark_num, index_granularity_rows, data_part->index_granularity_info.fixed_index_granularity, + index_granularity.getMarksCount()); + } + + if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" @@ -494,7 +501,6 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai mark_num, offset_in_compressed_file, offset_in_decompressed_block, index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity.getMarksCount()); - } auto column = type->createColumn(); diff --git a/tests/queries/0_stateless/02539_vertical_merge_compact_parts.reference b/tests/queries/0_stateless/02539_vertical_merge_compact_parts.reference deleted file mode 100644 index 685d3f3140d..00000000000 --- a/tests/queries/0_stateless/02539_vertical_merge_compact_parts.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 2 MergeParts Horizontal Compact -1 3 MergeParts Vertical Wide diff --git a/tests/queries/0_stateless/02539_vertical_merge_compact_parts.sql b/tests/queries/0_stateless/02539_vertical_merge_compact_parts.sql deleted file mode 100644 index 4140a73b8bc..00000000000 --- a/tests/queries/0_stateless/02539_vertical_merge_compact_parts.sql +++ /dev/null @@ -1,41 +0,0 @@ -DROP TABLE IF EXISTS t_compact_vertical_merge; - -CREATE TABLE t_compact_vertical_merge (id UInt64, s LowCardinality(String), arr Array(UInt64)) -ENGINE MergeTree ORDER BY id -SETTINGS - index_granularity = 16, - min_bytes_for_wide_part = 0, - min_rows_for_wide_part = 100, - vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1; - -INSERT INTO t_compact_vertical_merge SELECT number, toString(number), range(number % 10) FROM numbers(40); -INSERT INTO t_compact_vertical_merge SELECT number, toString(number), range(number % 10) FROM numbers(40); - -OPTIMIZE TABLE t_compact_vertical_merge FINAL; -SYSTEM FLUSH LOGS; - -WITH splitByChar('_', part_name) AS name_parts, - name_parts[2]::UInt64 AS min_block, - name_parts[3]::UInt64 AS max_block -SELECT min_block, max_block, event_type, merge_algorithm, part_type FROM system.part_log -WHERE - database = currentDatabase() AND - table = 't_compact_vertical_merge' AND - min_block = 1 AND max_block = 2; - -INSERT INTO t_compact_vertical_merge SELECT number, toString(number), range(number % 10) FROM numbers(40); - -OPTIMIZE TABLE t_compact_vertical_merge FINAL; -SYSTEM FLUSH LOGS; - -WITH splitByChar('_', part_name) AS name_parts, - name_parts[2]::UInt64 AS min_block, - name_parts[3]::UInt64 AS max_block -SELECT min_block, max_block, event_type, merge_algorithm, part_type FROM system.part_log -WHERE - database = currentDatabase() AND - table = 't_compact_vertical_merge' AND - min_block = 1 AND max_block = 3; - -DROP TABLE t_compact_vertical_merge; From 74937cf27bbaf3723ee095e702697a04d68e2479 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 8 Feb 2023 13:07:27 +0000 Subject: [PATCH 093/116] Reject DoS-prone hyperscan regexes --- .../functions/string-search-functions.md | 12 ++- src/Core/Settings.h | 1 + src/Dictionaries/RegExpTreeDictionary.cpp | 54 ++----------- .../FunctionsMultiStringFuzzySearch.h | 10 ++- src/Functions/FunctionsMultiStringSearch.h | 10 ++- src/Functions/MatchImpl.h | 2 +- src/Functions/MultiMatchAllIndicesImpl.h | 36 +++++++-- src/Functions/MultiMatchAnyImpl.h | 43 ++++++++--- src/Functions/MultiSearchFirstIndexImpl.h | 6 +- src/Functions/MultiSearchFirstPositionImpl.h | 6 +- src/Functions/MultiSearchImpl.h | 6 +- src/Functions/checkHyperscanRegexp.cpp | 75 +++++++++++++++++++ src/Functions/checkHyperscanRegexp.h | 18 +++++ src/Functions/extract.cpp | 6 +- .../02560_regexp_denial_of_service.reference | 0 .../02560_regexp_denial_of_service.sql | 58 ++++++++++++++ 16 files changed, 258 insertions(+), 85 deletions(-) create mode 100644 tests/queries/0_stateless/02560_regexp_denial_of_service.reference create mode 100644 tests/queries/0_stateless/02560_regexp_denial_of_service.sql diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 7b14b0e96de..b6b70c7795b 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -391,10 +391,18 @@ For patterns to search for substrings in a string, it is better to use LIKE or ## multiMatchAny(haystack, \[pattern1, pattern2, …, patternn\]) -The same as `match`, but returns 0 if none of the regular expressions are matched and 1 if any of the patterns matches. It uses [hyperscan](https://github.com/intel/hyperscan) library. For patterns to search substrings in a string, it is better to use `multiSearchAny` since it works much faster. +The same as `match`, but returns 0 if none of the regular expressions are matched and 1 if any of the patterns matches. For patterns to search substrings in a string, it is better to use `multiSearchAny` since it works much faster. :::note -The length of any of the `haystack` string must be less than 232 bytes otherwise the exception is thrown. This restriction takes place because of hyperscan API. +Functions `multiMatchAny`, `multiMatchAnyIndex`, `multiMatchAllIndices` and their fuzzy equivalents (`multiFuzzyMatchAny`, +`multiFuzzyMatchAnyIndex`, `multiFuzzyMatchAllIndices`) use the (Vectorscan)[https://github.com/VectorCamp/vectorscan] library. As such, +they are only enabled if ClickHouse is compiled with support for vectorscan. + +Due to restrictions of vectorscan, the length of the `haystack` string must be less than 232 bytes. + +Hyperscan is generally vulnerable to regular expression denial of service (ReDoS) attacks (e.g. see +(here)[https://www.usenix.org/conference/usenixsecurity22/presentation/turonova], (here)[https://doi.org/10.1007/s10664-021-10033-1] and +(here)[ https://doi.org/10.1145/3236024.3236027]. Users are adviced to check the provided patterns carefully. ::: ## multiMatchAnyIndex(haystack, \[pattern1, pattern2, …, patternn\]) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 983ec4d6416..e378de77875 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -446,6 +446,7 @@ class IColumn; M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \ M(UInt64, max_hyperscan_regexp_total_length, 0, "Max total length of all regexps than can be used in hyperscan multi-match functions (per every function). Zero means unlimited.", 0) \ + M(Bool, reject_expensive_hyperscan_regexps, true, "Reject patterns which will likely be expensive to evaluate with hyperscan (due to NFA state explosion)", 0) \ M(Bool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ M(Bool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index fdda2cd407d..c636f200324 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include @@ -152,53 +153,6 @@ void RegExpTreeDictionary::calculateBytesAllocated() bytes_allocated += 2 * sizeof(UInt64) * topology_order.size(); } -namespace -{ - /// hyper scan is not good at processing regex containing {0, 200} - /// This will make re compilation slow and failed. So we select this heavy regular expressions and - /// process it with re2. - struct RegexChecker - { - re2_st::RE2 searcher; - RegexChecker() : searcher(R"(\{([\d]+),([\d]+)\})") {} - - static bool isFigureLargerThanFifty(const String & str) - try - { - auto number = std::stoi(str); - return number > 50; - } - catch (std::exception &) - { - return false; - } - - [[maybe_unused]] - bool isSimpleRegex(const String & regex) const - { - - re2_st::StringPiece haystack(regex.data(), regex.size()); - re2_st::StringPiece matches[10]; - size_t start_pos = 0; - while (start_pos < regex.size()) - { - if (searcher.Match(haystack, start_pos, regex.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 10)) - { - const auto & match = matches[0]; - start_pos += match.length(); - const auto & match1 = matches[1]; - const auto & match2 = matches[2]; - if (isFigureLargerThanFifty(match1.ToString()) || isFigureLargerThanFifty(match2.ToString())) - return false; - } - else - break; - } - return true; - } - }; -} - void RegExpTreeDictionary::initRegexNodes(Block & block) { auto id_column = block.getByName(kId).column; @@ -207,7 +161,9 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) auto keys_column = block.getByName(kKeys).column; auto values_column = block.getByName(kValues).column; - RegexChecker checker; +#ifdef USE_VECTORSCAN + SlowWithHyperscanChecker checker; +#endif size_t size = block.rows(); for (size_t i = 0; i < size; i++) @@ -253,7 +209,7 @@ void RegExpTreeDictionary::initRegexNodes(Block & block) } regex_nodes.emplace(id, node); #if USE_VECTORSCAN - if (use_vectorscan && checker.isSimpleRegex(regex)) + if (use_vectorscan && !checker.isSlow(regex)) { simple_regexps.push_back(regex); regexp_ids.push_back(id); diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index 5d86dd4dada..00d989f388e 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -39,13 +39,14 @@ public: static FunctionPtr create(ContextPtr context) { const auto & settings = context->getSettingsRef(); - return std::make_shared(settings.allow_hyperscan, settings.max_hyperscan_regexp_length, settings.max_hyperscan_regexp_total_length); + return std::make_shared(settings.allow_hyperscan, settings.max_hyperscan_regexp_length, settings.max_hyperscan_regexp_total_length, settings.reject_expensive_hyperscan_regexps); } - FunctionsMultiStringFuzzySearch(bool allow_hyperscan_, size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + FunctionsMultiStringFuzzySearch(bool allow_hyperscan_, size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_, bool reject_expensive_hyperscan_regexps_) : allow_hyperscan(allow_hyperscan_) , max_hyperscan_regexp_length(max_hyperscan_regexp_length_) , max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + , reject_expensive_hyperscan_regexps(reject_expensive_hyperscan_regexps_) {} String getName() const override { return name; } @@ -112,14 +113,14 @@ public: col_needles_const->getValue(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, edit_distance, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to @@ -135,6 +136,7 @@ private: const bool allow_hyperscan; const size_t max_hyperscan_regexp_length; const size_t max_hyperscan_regexp_total_length; + const bool reject_expensive_hyperscan_regexps; }; } diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 2465567b883..c0ed90aa042 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -53,13 +53,14 @@ public: static FunctionPtr create(ContextPtr context) { const auto & settings = context->getSettingsRef(); - return std::make_shared(settings.allow_hyperscan, settings.max_hyperscan_regexp_length, settings.max_hyperscan_regexp_total_length); + return std::make_shared(settings.allow_hyperscan, settings.max_hyperscan_regexp_length, settings.max_hyperscan_regexp_total_length, settings.reject_expensive_hyperscan_regexps); } - FunctionsMultiStringSearch(bool allow_hyperscan_, size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_) + FunctionsMultiStringSearch(bool allow_hyperscan_, size_t max_hyperscan_regexp_length_, size_t max_hyperscan_regexp_total_length_, bool reject_expensive_hyperscan_regexps_) : allow_hyperscan(allow_hyperscan_) , max_hyperscan_regexp_length(max_hyperscan_regexp_length_) , max_hyperscan_regexp_total_length(max_hyperscan_regexp_total_length_) + , reject_expensive_hyperscan_regexps(reject_expensive_hyperscan_regexps_) {} String getName() const override { return name; } @@ -108,13 +109,13 @@ public: col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_const->getValue(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); else Impl::vectorVector( col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needles_vector->getData(), col_needles_vector->getOffsets(), vec_res, offsets_res, - allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); // the combination of const haystack + const needle is not implemented because // useDefaultImplementationForConstants() == true makes upper layers convert both to @@ -130,6 +131,7 @@ private: const bool allow_hyperscan; const size_t max_hyperscan_regexp_length; const size_t max_hyperscan_regexp_total_length; + const bool reject_expensive_hyperscan_regexps; }; } diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 00c08ea8531..db8dd55474e 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -6,7 +6,7 @@ #include #include #include -#include "Regexps.h" +#include #include "config.h" #include diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index dec8349e693..d655311f532 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -6,7 +6,7 @@ #include #include #include -#include "Regexps.h" +#include #include "config.h" @@ -51,9 +51,10 @@ struct MultiMatchAllIndicesImpl PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); } static void vectorConstant( @@ -65,7 +66,8 @@ struct MultiMatchAllIndicesImpl std::optional edit_distance, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -77,6 +79,14 @@ struct MultiMatchAllIndicesImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (reject_expensive_hyperscan_regexps) + { + SlowWithHyperscanChecker checker; + for (auto needle : needles) + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } + offsets.resize(haystack_offsets.size()); if (needles_arr.empty()) @@ -135,6 +145,7 @@ struct MultiMatchAllIndicesImpl (void)edit_distance; (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; + (void)reject_expensive_hyperscan_regexps; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } @@ -148,9 +159,10 @@ struct MultiMatchAllIndicesImpl PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); } static void vectorVector( @@ -163,7 +175,8 @@ struct MultiMatchAllIndicesImpl std::optional edit_distance, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -195,6 +208,14 @@ struct MultiMatchAllIndicesImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (reject_expensive_hyperscan_regexps) + { + SlowWithHyperscanChecker checker; + for (auto needle : needles) + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } + MultiRegexps::DeferredConstructedRegexpsPtr deferred_constructed_regexps = MultiRegexps::getOrSet(needles, edit_distance); MultiRegexps::Regexps * regexps = deferred_constructed_regexps->get(); hs_scratch_t * scratch = nullptr; @@ -249,6 +270,7 @@ struct MultiMatchAllIndicesImpl (void)edit_distance; (void)max_hyperscan_regexp_length; (void)max_hyperscan_regexp_total_length; + (void)reject_expensive_hyperscan_regexps; throw Exception(ErrorCodes::NOT_IMPLEMENTED, "multi-search all indices is not implemented when vectorscan is off"); #endif // USE_VECTORSCAN } diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 0e42d4f6b58..7a35fcebc24 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -5,7 +5,7 @@ #include #include #include -#include "Regexps.h" +#include #include "config.h" @@ -65,9 +65,10 @@ struct MultiMatchAnyImpl PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { - vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorConstant(haystack_data, haystack_offsets, needles_arr, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); } static void vectorConstant( @@ -79,7 +80,8 @@ struct MultiMatchAnyImpl [[maybe_unused]] std::optional edit_distance, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -91,6 +93,14 @@ struct MultiMatchAnyImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (reject_expensive_hyperscan_regexps) + { + SlowWithHyperscanChecker checker; + for (auto needle : needles) + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } + res.resize(haystack_offsets.size()); if (needles_arr.empty()) @@ -175,9 +185,10 @@ struct MultiMatchAnyImpl PaddedPODArray & offsets, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { - vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + vectorVector(haystack_data, haystack_offsets, needles_data, needles_offsets, res, offsets, std::nullopt, allow_hyperscan, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length, reject_expensive_hyperscan_regexps); } static void vectorVector( @@ -190,7 +201,8 @@ struct MultiMatchAnyImpl std::optional edit_distance, bool allow_hyperscan, size_t max_hyperscan_regexp_length, - size_t max_hyperscan_regexp_total_length) + size_t max_hyperscan_regexp_total_length, + bool reject_expensive_hyperscan_regexps) { if (!allow_hyperscan) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0"); @@ -209,9 +221,7 @@ struct MultiMatchAnyImpl needles.reserve(needles_offsets[i] - prev_needles_offset); for (size_t j = prev_needles_offset; j < needles_offsets[i]; ++j) - { needles.emplace_back(needles_data_string->getDataAt(j).toView()); - } if (needles.empty()) { @@ -223,6 +233,14 @@ struct MultiMatchAnyImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + if (reject_expensive_hyperscan_regexps) + { + SlowWithHyperscanChecker checker; + for (auto needle : needles) + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } + MultiRegexps::DeferredConstructedRegexpsPtr deferred_constructed_regexps = MultiRegexps::getOrSet(needles, edit_distance); MultiRegexps::Regexps * regexps = deferred_constructed_regexps->get(); hs_scratch_t * scratch = nullptr; @@ -309,6 +327,13 @@ struct MultiMatchAnyImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); + for (auto needle : needles) + { + SlowWithHyperscanChecker checker; + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } + for (size_t j = 0; j < needles.size(); ++j) { String needle(needles[j]); diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index f108f2ca41e..73f3c92adfb 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -32,7 +32,8 @@ struct MultiSearchFirstIndexImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -78,7 +79,8 @@ struct MultiSearchFirstIndexImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 69cab478d13..99dd3f9d394 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -32,7 +32,8 @@ struct MultiSearchFirstPositionImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -87,7 +88,8 @@ struct MultiSearchFirstPositionImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index d42c2ca43e4..fb7d56f302a 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -32,7 +32,8 @@ struct MultiSearchImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) @@ -77,7 +78,8 @@ struct MultiSearchImpl PaddedPODArray & /*offsets*/, bool /*allow_hyperscan*/, size_t /*max_hyperscan_regexp_length*/, - size_t /*max_hyperscan_regexp_total_length*/) + size_t /*max_hyperscan_regexp_total_length*/, + bool /*reject_expensive_hyperscan_regexps*/) { const size_t haystack_size = haystack_offsets.size(); res.resize(haystack_size); diff --git a/src/Functions/checkHyperscanRegexp.cpp b/src/Functions/checkHyperscanRegexp.cpp index 4a1bc4f9031..e6fbc3baa1a 100644 --- a/src/Functions/checkHyperscanRegexp.cpp +++ b/src/Functions/checkHyperscanRegexp.cpp @@ -1,6 +1,7 @@ #include #include +#include namespace DB { @@ -27,4 +28,78 @@ void checkHyperscanRegexp(const std::vector & regexps, size_t } } +namespace +{ + +bool isLargerThanFifty(std::string_view str) +{ + int number; + auto [_, ec] = std::from_chars(str.begin(), str.end(), number); + if (ec != std::errc()) + return false; + return number > 50; +} + +} + +/// Check for sub-patterns of the form x{n} or x{n,} can be expensive. Ignore spaces before/after n and m. +bool SlowWithHyperscanChecker::isSlowOneRepeat(std::string_view regexp) +{ + re2_st::StringPiece haystack(regexp.data(), regexp.size()); + re2_st::StringPiece matches[2]; + size_t start_pos = 0; + while (start_pos < regexp.size()) + { + if (searcher_one_repeat.Match(haystack, start_pos, regexp.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 2)) + { + const auto & match = matches[0]; + start_pos += (matches[0].data() - haystack.data()) + match.length(); // fwd by prefix + match length + const auto & submatch = matches[1]; + if (isLargerThanFifty({submatch.data(), submatch.size()})) + return true; + } + else + break; + } + return false; +} + +/// Check if sub-patterns of the form x{n,m} can be expensive. Ignore spaces before/after n and m. +bool SlowWithHyperscanChecker::isSlowTwoRepeats(std::string_view regexp) +{ + re2_st::StringPiece haystack(regexp.data(), regexp.size()); + re2_st::StringPiece matches[3]; + size_t start_pos = 0; + while (start_pos < regexp.size()) + { + if (searcher_two_repeats.Match(haystack, start_pos, regexp.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 3)) + { + const auto & match = matches[0]; + start_pos += (matches[0].data() - haystack.data()) + match.length(); // fwd by prefix + match length + const auto & submatch1 = matches[1]; + const auto & submatch2 = matches[2]; + if (isLargerThanFifty({submatch1.data(), submatch1.size()}) + || isLargerThanFifty({submatch2.data(), submatch2.size()})) + return true; + } + else + break; + } + return false; +} + +SlowWithHyperscanChecker::SlowWithHyperscanChecker() + : searcher_one_repeat(R"(\{\s*([\d]+)\s*,?\s*})") + , searcher_two_repeats(R"(\{\s*([\d]+)\s*,\s*([\d]+)\s*\})") +{} + +bool SlowWithHyperscanChecker::isSlow(std::string_view regexp) +{ + if (isSlowOneRepeat(regexp)) + return true; + else if (isSlowTwoRepeats(regexp)) + return true; + return false; +} + } diff --git a/src/Functions/checkHyperscanRegexp.h b/src/Functions/checkHyperscanRegexp.h index e5f1165a949..8bb76e6ebc4 100644 --- a/src/Functions/checkHyperscanRegexp.h +++ b/src/Functions/checkHyperscanRegexp.h @@ -3,9 +3,27 @@ #include #include +#include + namespace DB { void checkHyperscanRegexp(const std::vector & regexps, size_t max_hyperscan_regexp_length, size_t max_hyperscan_regexp_total_length); +/// Regexp evaluation with hyperscan can be slow for certain patterns due to NFA state explosion. Try to identify such patterns on a +/// best-effort basis. + +class SlowWithHyperscanChecker +{ +public: + SlowWithHyperscanChecker(); + bool isSlow(std::string_view regexp); + +private: + bool isSlowOneRepeat(std::string_view regexp); + bool isSlowTwoRepeats(std::string_view regexp); + re2_st::RE2 searcher_one_repeat; + re2_st::RE2 searcher_two_repeats; +}; + } diff --git a/src/Functions/extract.cpp b/src/Functions/extract.cpp index 5d539e03dae..74c5a2fdd36 100644 --- a/src/Functions/extract.cpp +++ b/src/Functions/extract.cpp @@ -1,6 +1,6 @@ -#include "FunctionsStringSearchToString.h" -#include "FunctionFactory.h" -#include "Regexps.h" +#include +#include +#include #include diff --git a/tests/queries/0_stateless/02560_regexp_denial_of_service.reference b/tests/queries/0_stateless/02560_regexp_denial_of_service.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02560_regexp_denial_of_service.sql b/tests/queries/0_stateless/02560_regexp_denial_of_service.sql new file mode 100644 index 00000000000..3a02c12c679 --- /dev/null +++ b/tests/queries/0_stateless/02560_regexp_denial_of_service.sql @@ -0,0 +1,58 @@ +-- Tags: no-fasttest, use-vectorscan + +DROP TABLE IF EXISTS t; + +-- test that the check which rejects hyperscan regexes with too big bounded repeats works + +-- {n} +SELECT multiMatchAny('test', ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{ 51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51 }']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['prefix.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51}.suffix']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{4,4}midfix{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +-- {n,} +SELECT multiMatchAny('test', ['.{51,}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{ 51,}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51 ,}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51, }']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['prefix.{51,}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51,}.suffix']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{4,4}midfix{51,}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +-- {n,m} +SELECT multiMatchAny('test', ['.{1,51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51,52}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{ 51,52}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51 ,52}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51, 52}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{51,52 }']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['prefix.{1,51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{1,51}.suffix']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny('test', ['.{4,4}midfix{1,51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +-- test that the check is implemented in all functions which use vectorscan + +CREATE TABLE t(c String) Engine=MergeTree() ORDER BY c; +INSERT INTO t VALUES('Hallo Welt'); + +SELECT multiMatchAny('Hallo Welt', ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAny(c, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +SELECT multiMatchAnyIndex('Hallo Welt', ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAnyIndex(c, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +SELECT multiMatchAllIndices('Hallo Welt', ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiMatchAllIndices(c, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +SELECT multiFuzzyMatchAny('Hallo Welt', 1, ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiFuzzyMatchAny(c, 1, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +SELECT multiFuzzyMatchAnyIndex('Hallo Welt', 1, ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiFuzzyMatchAnyIndex(c, 1, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +SELECT multiFuzzyMatchAllIndices('Hallo Welt', 1, ['.{51}']); -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } +SELECT multiFuzzyMatchAllIndices(c, 1, ['.{51}']) FROM t; -- { serverError HYPERSCAN_CANNOT_SCAN_TEXT } + +DROP TABLE t; From 0d61ab29f71682548db3cd5e9beec1b7eeb516ae Mon Sep 17 00:00:00 2001 From: bkuschel Date: Thu, 9 Feb 2023 12:55:05 -0500 Subject: [PATCH 094/116] Update Curl to 7.87.0 --- contrib/curl | 2 +- contrib/curl-cmake/CMakeLists.txt | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/contrib/curl b/contrib/curl index 462196e6b4a..c12fb3ddaf4 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit 462196e6b4a47f924293a0e26b8e9c23d37ac26f +Subproject commit c12fb3ddaf48e709a7a4deaa55ec485e4df163ee diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 207b7c66371..8a570bd267c 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -8,6 +8,10 @@ endif() set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") set (SRCS + "${LIBRARY_DIR}/lib/fopen.c" + "${LIBRARY_DIR}/lib/noproxy.c" + "${LIBRARY_DIR}/lib/idn.c" + "${LIBRARY_DIR}/lib/cfilters.c" "${LIBRARY_DIR}/lib/file.c" "${LIBRARY_DIR}/lib/timeval.c" "${LIBRARY_DIR}/lib/base64.c" @@ -82,7 +86,6 @@ set (SRCS "${LIBRARY_DIR}/lib/openldap.c" "${LIBRARY_DIR}/lib/curl_gethostname.c" "${LIBRARY_DIR}/lib/gopher.c" - "${LIBRARY_DIR}/lib/idn_win32.c" "${LIBRARY_DIR}/lib/http_proxy.c" "${LIBRARY_DIR}/lib/asyn-thread.c" "${LIBRARY_DIR}/lib/curl_gssapi.c" @@ -93,7 +96,6 @@ set (SRCS "${LIBRARY_DIR}/lib/rand.c" "${LIBRARY_DIR}/lib/curl_multibyte.c" "${LIBRARY_DIR}/lib/conncache.c" - "${LIBRARY_DIR}/lib/dotdot.c" "${LIBRARY_DIR}/lib/http2.c" "${LIBRARY_DIR}/lib/smb.c" "${LIBRARY_DIR}/lib/curl_endian.c" @@ -103,7 +105,6 @@ set (SRCS "${LIBRARY_DIR}/lib/sha256.c" "${LIBRARY_DIR}/lib/setopt.c" "${LIBRARY_DIR}/lib/curl_path.c" - "${LIBRARY_DIR}/lib/curl_ctype.c" "${LIBRARY_DIR}/lib/curl_range.c" "${LIBRARY_DIR}/lib/psl.c" "${LIBRARY_DIR}/lib/doh.c" From acabee8554e212a5c83fe35e7f81e22b7c82bc2c Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 9 Feb 2023 20:17:46 +0000 Subject: [PATCH 095/116] review fixes --- src/IO/WriteBufferFromS3.cpp | 2 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index f78e35854cd..86a4cebb5d8 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -356,7 +356,7 @@ void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) } else { - write_settings.resource_link.adjust(cost, 0); // We assume no resource was used in case of failure + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 58bcef38413..c5573a1ed86 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -99,7 +99,16 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); + int bytes_read; + try + { + bytes_read = hdfsRead(fs.get(), fin, internal_buffer.begin(), safe_cast(num_bytes_to_read)); + } + catch (...) + { + read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure + throw; + } rlock.unlock(); if (bytes_read < 0) From 59d380c4a2a1f0417a9b3fceb0e725728cf45027 Mon Sep 17 00:00:00 2001 From: Larry Luo Date: Thu, 9 Feb 2023 12:29:42 -0800 Subject: [PATCH 096/116] Avoid CVE-2022-40303 CVE-2022-40304 --- contrib/libxml2 | 2 +- contrib/libxml2-cmake/CMakeLists.txt | 2 - .../linux_x86_64/include/libxml/xmlversion.h | 66 ++++++++++++------- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/contrib/libxml2 b/contrib/libxml2 index 7846b0a677f..f507d167f17 160000 --- a/contrib/libxml2 +++ b/contrib/libxml2 @@ -1 +1 @@ -Subproject commit 7846b0a677f8d3ce72486125fa281e92ac9970e8 +Subproject commit f507d167f1755b7eaea09fb1a44d29aab828b6d1 diff --git a/contrib/libxml2-cmake/CMakeLists.txt b/contrib/libxml2-cmake/CMakeLists.txt index a84936f8e3a..b7b3f7037ec 100644 --- a/contrib/libxml2-cmake/CMakeLists.txt +++ b/contrib/libxml2-cmake/CMakeLists.txt @@ -24,7 +24,6 @@ set(SRCS "${LIBXML2_SOURCE_DIR}/xinclude.c" "${LIBXML2_SOURCE_DIR}/nanohttp.c" "${LIBXML2_SOURCE_DIR}/nanoftp.c" - "${LIBXML2_SOURCE_DIR}/DOCBparser.c" "${LIBXML2_SOURCE_DIR}/catalog.c" "${LIBXML2_SOURCE_DIR}/globals.c" "${LIBXML2_SOURCE_DIR}/threads.c" @@ -36,7 +35,6 @@ set(SRCS "${LIBXML2_SOURCE_DIR}/xmlschemastypes.c" "${LIBXML2_SOURCE_DIR}/xmlunicode.c" "${LIBXML2_SOURCE_DIR}/triostr.c" - #"${LIBXML2_SOURCE_DIR}/trio.c" "${LIBXML2_SOURCE_DIR}/xmlreader.c" "${LIBXML2_SOURCE_DIR}/relaxng.c" "${LIBXML2_SOURCE_DIR}/dict.c" diff --git a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h index 52f62214324..9eabfaa50c8 100644 --- a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h +++ b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h @@ -1,6 +1,6 @@ /* - * Summary: compile-time version informations - * Description: compile-time version informations for the XML library + * Summary: compile-time version information + * Description: compile-time version information for the XML library * * Copy: See Copyright for the status of this software. * @@ -29,28 +29,28 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * the version string like "1.2.3" */ -#define LIBXML_DOTTED_VERSION "2.9.8" +#define LIBXML_DOTTED_VERSION "2.10.3" /** * LIBXML_VERSION: * * the version number: 1.2.3 value is 10203 */ -#define LIBXML_VERSION 20908 +#define LIBXML_VERSION 21003 /** * LIBXML_VERSION_STRING: * * the version number string, 1.2.3 value is "10203" */ -#define LIBXML_VERSION_STRING "20908" +#define LIBXML_VERSION_STRING "21003" /** * LIBXML_VERSION_EXTRA: * - * extra version information, used to show a CVS compilation + * extra version information, used to show a git commit description */ -#define LIBXML_VERSION_EXTRA "-GITv2.9.9-rc2-1-g6fc04d71" +#define LIBXML_VERSION_EXTRA "" /** * LIBXML_TEST_VERSION: @@ -58,7 +58,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * Macro to check that the libxml version in use is compatible with * the version the software has been compiled against */ -#define LIBXML_TEST_VERSION xmlCheckVersion(20908); +#define LIBXML_TEST_VERSION xmlCheckVersion(21003); #ifndef VMS #if 0 @@ -90,7 +90,9 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the thread support is configured in */ -#define LIBXML_THREAD_ENABLED 1 +#if 1 +#define LIBXML_THREAD_ENABLED +#endif /** * LIBXML_THREAD_ALLOC_ENABLED: @@ -169,7 +171,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the FTP support is configured in */ -#if 1 +#if 0 #define LIBXML_FTP_ENABLED #endif @@ -205,7 +207,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the deprecated APIs are compiled in for compatibility */ -#if 1 +#if 0 #define LIBXML_LEGACY_ENABLED #endif @@ -227,15 +229,6 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); #define LIBXML_CATALOG_ENABLED #endif -/** - * LIBXML_DOCB_ENABLED: - * - * Whether the SGML Docbook support is configured in - */ -#if 1 -#define LIBXML_DOCB_ENABLED -#endif - /** * LIBXML_XPATH_ENABLED: * @@ -254,6 +247,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); #define LIBXML_XPTR_ENABLED #endif +/** + * LIBXML_XPTR_LOCS_ENABLED: + * + * Whether support for XPointer locations is configured in + */ +#if 0 +#define LIBXML_XPTR_LOCS_ENABLED +#endif + /** * LIBXML_XINCLUDE_ENABLED: * @@ -268,7 +270,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether iconv support is available */ -#if 0 +#if 1 #define LIBXML_ICONV_ENABLED #endif @@ -348,8 +350,10 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * LIBXML_EXPR_ENABLED: * * Whether the formal expressions interfaces are compiled in + * + * This code is unused and disabled unconditionally for now. */ -#if 1 +#if 0 #define LIBXML_EXPR_ENABLED #endif @@ -452,6 +456,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); # define LIBXML_ATTR_FORMAT(fmt,args) #endif +#ifndef XML_DEPRECATED +# ifdef IN_LIBXML +# define XML_DEPRECATED +# else +/* Available since at least GCC 3.1 */ +# define XML_DEPRECATED __attribute__((deprecated)) +# endif +#endif + #else /* ! __GNUC__ */ /** * ATTRIBUTE_UNUSED: @@ -471,6 +484,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * Macro used to indicate to GCC the parameter are printf like */ #define LIBXML_ATTR_FORMAT(fmt,args) +/** + * XML_DEPRECATED: + * + * Macro used to indicate that a function, variable, type or struct member + * is deprecated. + */ +#ifndef XML_DEPRECATED +#define XML_DEPRECATED +#endif #endif /* __GNUC__ */ #ifdef __cplusplus From 8c9be17b915402fcdf6e07b981b7e9381e72a452 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Feb 2023 21:49:11 +0100 Subject: [PATCH 097/116] Add a test --- .../02562_with_fill_nullable.reference | 19 +++++++++++++++++++ .../0_stateless/02562_with_fill_nullable.sql | 4 ++++ 2 files changed, 23 insertions(+) create mode 100644 tests/queries/0_stateless/02562_with_fill_nullable.reference create mode 100644 tests/queries/0_stateless/02562_with_fill_nullable.sql diff --git a/tests/queries/0_stateless/02562_with_fill_nullable.reference b/tests/queries/0_stateless/02562_with_fill_nullable.reference new file mode 100644 index 00000000000..d32f49569b4 --- /dev/null +++ b/tests/queries/0_stateless/02562_with_fill_nullable.reference @@ -0,0 +1,19 @@ +2023-02-09 +2023-02-10 +2023-02-11 +2023-02-12 +2023-02-13 +2023-02-14 +2023-02-15 +2023-02-16 +2023-02-17 +2023-02-18 +2023-02-19 +--- +2023-02-09 +2023-02-10 +2023-02-11 +2023-02-12 +2023-02-13 +\N +\N diff --git a/tests/queries/0_stateless/02562_with_fill_nullable.sql b/tests/queries/0_stateless/02562_with_fill_nullable.sql new file mode 100644 index 00000000000..d2ca09e1a45 --- /dev/null +++ b/tests/queries/0_stateless/02562_with_fill_nullable.sql @@ -0,0 +1,4 @@ +SELECT toNullable('2023-02-09'::Date + number * 10) AS d FROM numbers(2) ORDER BY d WITH FILL; +SELECT '---'; +SELECT number % 2 ? NULL : toNullable('2023-02-09'::Date + number) AS d FROM numbers(5) ORDER BY d ASC NULLS LAST WITH FILL; +-- TODO: NULLS FIRST does not work correctly with FILL. From c02adec4b83f7b38c491fb42b9f0805ba1c5790e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 Feb 2023 07:47:48 +0000 Subject: [PATCH 098/116] Early return --- src/IO/S3Common.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 6a696f6c989..eb6aff607d9 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -604,8 +604,10 @@ public: if (!credentials.IsEmpty()) { AddProvider(std::make_shared(credentials)); + return; } - else if (use_environment_credentials) + + if (use_environment_credentials) { static const char AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI[] = "AWS_CONTAINER_CREDENTIALS_RELATIVE_URI"; static const char AWS_ECS_CONTAINER_CREDENTIALS_FULL_URI[] = "AWS_CONTAINER_CREDENTIALS_FULL_URI"; From 8144c28c3b042f259eb4db3cbfbfb369539c281c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 9 Feb 2023 10:05:14 +0000 Subject: [PATCH 099/116] Fix spelling --- base/poco/CHANGELOG | 2791 ----------------- base/poco/CONTRIBUTORS | 52 - base/poco/Crypto/include/Poco/Crypto/ECKey.h | 3 +- .../poco/Crypto/include/Poco/Crypto/EVPPKey.h | 2 +- .../include/Poco/Crypto/OpenSSLInitializer.h | 2 +- base/poco/Crypto/src/CryptoStream.cpp | 2 +- base/poco/Crypto/src/X509Certificate.cpp | 2 +- .../Data/ODBC/include/Poco/Data/ODBC/Binder.h | 8 +- .../Data/ODBC/include/Poco/Data/ODBC/ODBC.h | 2 +- .../ODBC/include/Poco/Data/ODBC/SessionImpl.h | 2 +- .../ODBC/include/Poco/Data/ODBC/TypeInfo.h | 2 +- base/poco/Data/ODBC/src/ODBCStatementImpl.cpp | 2 +- base/poco/Data/ODBC/src/Preparator.cpp | 2 +- .../Data/include/Poco/Data/AbstractBinder.h | 2 +- .../include/Poco/Data/AbstractExtraction.h | 4 +- base/poco/Data/include/Poco/Data/Binding.h | 4 +- base/poco/Data/include/Poco/Data/Bulk.h | 4 +- .../Data/include/Poco/Data/BulkExtraction.h | 2 +- base/poco/Data/include/Poco/Data/Data.h | 2 +- base/poco/Data/include/Poco/Data/Extraction.h | 2 +- base/poco/Data/include/Poco/Data/RecordSet.h | 2 +- base/poco/Data/include/Poco/Data/Row.h | 6 +- .../Data/include/Poco/Data/RowFormatter.h | 6 +- base/poco/Data/include/Poco/Data/SQLChannel.h | 12 +- base/poco/Data/include/Poco/Data/Session.h | 4 +- .../poco/Data/include/Poco/Data/SessionImpl.h | 2 +- base/poco/Data/include/Poco/Data/Statement.h | 2 +- .../Data/include/Poco/Data/StatementCreator.h | 2 +- .../Data/include/Poco/Data/StatementImpl.h | 6 +- .../poco/Data/include/Poco/Data/Transaction.h | 6 +- .../Foundation/include/Poco/AbstractCache.h | 7 +- .../Foundation/include/Poco/AbstractEvent.h | 12 +- .../include/Poco/AbstractStrategy.h | 2 +- .../Foundation/include/Poco/AutoReleasePool.h | 2 +- .../Foundation/include/Poco/BinaryWriter.h | 2 +- base/poco/Foundation/include/Poco/Clock.h | 2 +- base/poco/Foundation/include/Poco/Config.h | 2 +- base/poco/Foundation/include/Poco/DateTime.h | 2 +- .../include/Poco/Dynamic/VarHolder.h | 68 +- .../include/Poco/FIFOBufferStream.h | 2 +- .../Foundation/include/Poco/FileChannel.h | 2 +- .../poco/Foundation/include/Poco/FileStream.h | 4 +- .../poco/Foundation/include/Poco/Foundation.h | 2 +- .../poco/Foundation/include/Poco/HMACEngine.h | 2 +- .../Foundation/include/Poco/LocalDateTime.h | 4 +- base/poco/Foundation/include/Poco/Logger.h | 10 +- base/poco/Foundation/include/Poco/MD4Engine.h | 2 +- base/poco/Foundation/include/Poco/MD5Engine.h | 2 +- base/poco/Foundation/include/Poco/Nullable.h | 2 +- .../Foundation/include/Poco/PBKDF2Engine.h | 2 +- base/poco/Foundation/include/Poco/Path.h | 2 +- .../include/Poco/RecursiveDirectoryIterator.h | 2 +- .../include/Poco/RegularExpression.h | 2 +- .../poco/Foundation/include/Poco/SHA1Engine.h | 2 +- .../include/Poco/SharedMemory_WIN32.h | 2 +- .../Foundation/include/Poco/SimpleHashTable.h | 2 +- .../Foundation/include/Poco/StringTokenizer.h | 2 +- .../Foundation/include/Poco/TextEncoding.h | 4 +- base/poco/Foundation/include/Poco/Thread.h | 2 +- base/poco/Foundation/include/Poco/Timestamp.h | 2 +- base/poco/Foundation/include/Poco/TypeList.h | 10 +- .../include/Poco/UniqueAccessExpireLRUCache.h | 2 +- .../include/Poco/UniqueExpireLRUCache.h | 2 +- base/poco/Foundation/src/DirectoryWatcher.cpp | 20 +- base/poco/Foundation/src/Logger.cpp | 2 +- base/poco/Foundation/src/Random.cpp | 4 +- base/poco/Foundation/src/bignum-dtoa.cc | 4 +- base/poco/Foundation/src/deflate.c | 4 +- base/poco/Foundation/src/deflate.h | 2 +- base/poco/Foundation/src/double-conversion.h | 6 +- base/poco/Foundation/src/fixed-dtoa.cc | 2 +- base/poco/Foundation/src/gzguts.h | 11 - base/poco/Foundation/src/inftrees.h | 2 +- base/poco/Foundation/src/pcre_compile.c | 2 +- base/poco/Foundation/src/pcre_exec.c | 6 +- base/poco/Foundation/src/pcre_internal.h | 2 +- base/poco/Foundation/src/pcre_jit_compile.c | 10 +- base/poco/Foundation/src/strtod.cc | 2 +- base/poco/Foundation/src/trees.c | 2 +- base/poco/Foundation/src/zutil.c | 2 +- base/poco/Foundation/src/zutil.h | 25 +- base/poco/JSON/include/Poco/JSON/Object.h | 2 +- .../Net/include/Poco/Net/ICMPPacketImpl.h | 12 +- base/poco/Net/include/Poco/Net/Net.h | 2 +- base/poco/Net/include/Poco/Net/PartSource.h | 2 +- .../poco/Net/include/Poco/Net/SocketReactor.h | 2 +- base/poco/Net/src/IPAddressImpl.cpp | 2 +- base/poco/Net/src/MessageHeader.cpp | 10 +- base/poco/Net/src/NetworkInterface.cpp | 2 +- base/poco/Net/src/SMTPClientSession.cpp | 2 +- base/poco/Net/src/Socket.cpp | 8 +- .../NetSSL_OpenSSL/include/Poco/Net/NetSSL.h | 2 +- base/poco/NetSSL_OpenSSL/src/SSLException.cpp | 2 +- base/poco/Redis/include/Poco/Redis/Redis.h | 2 +- .../poco/Util/include/Poco/Util/Application.h | 2 +- base/poco/Util/include/Poco/Util/Units.h | 1 - base/poco/XML/src/AbstractNode.cpp | 28 +- base/poco/XML/src/XMLStreamParser.cpp | 10 +- utils/check-style/check-typos | 2 +- utils/check-style/codespell-ignore-words.list | 7 + 100 files changed, 223 insertions(+), 3094 deletions(-) delete mode 100644 base/poco/CHANGELOG delete mode 100644 base/poco/CONTRIBUTORS diff --git a/base/poco/CHANGELOG b/base/poco/CHANGELOG deleted file mode 100644 index a10371b381c..00000000000 --- a/base/poco/CHANGELOG +++ /dev/null @@ -1,2791 +0,0 @@ -This is the changelog file for the POCO C++ Libraries. - -Release 1.9.3 (2019-08-20) -========================== - -- fixed GH #2603: Remove incorrect upper size limits for SSL certificates in NetSSL_Win -- fixed GH #2661: Poco::Zip::ZipArchive cannot load new tomcat.zip file (additional fix) -- fixed GH #2742: Support of vs150 & vs160 with the official Microsoft localization executable, - vswhere.exe, installed by MSVC starting from VS2017 -- Data/ODBC: make binding of std::string configurable (SQL_LONGVARCHAR - default or SQL_VARCHAR) - through a global setting (Poco::Data::ODBC::Connector::bindStringToLongVarChar()). -- added Poco::SharedLibrary::setSearchPath() (currently implemented on Windows only) -- Windows required minimum version is now Windows XP SP2 -- upgraded bundled SQLite to 3.29.0 -- CppParser now supports type aliases defined with using keyword. -- PageCompiler: added support for adding Content-Security-Policy and Cache-Control headers. - - -Release 1.9.2 (2019-07-02) -========================== - -- fixed GH #2736: Error using CMake gui - latest version -- fixed GH #2737: Bundles vulnerable copy of Expat — please upgrade to Expat 2.2.7 -- fixed GH #2738: Poco::AccessExpireStrategy::onGet() must not extend expiration time after expiration - - -Release 1.9.1 (2019-06-24) -========================== - -- Added support for building with different OpenSSL distributions on Windows. - See the POCO_EXTERNAL_OPENSSL macro defined in Foundation/include/Poco/Config.h - for options. -- Added Poco::Net::HTTPClientSession::flushRequest() -- Added Poco::Net::WebSocket::setMaxPayloadSize() and Poco::Net::WebSocket::getMaxPayloadSize() - to specify a maximum acceptable payload size for Poco::Net::WebSocket::receiveFrame(). -- Poco::Net::WebSocket: don't attempt to send empty credentials in response to 401 response. -- Redis: added support for additional commands (exists, expire, ping, multi, exec, discard) -- Redis: added Poco::Redis::Client::isConnected() -- Upgraded bundled PCRE to version 8.43 -- Upgraded bundled SQLite to version 3.28.0 -- Added project/solution files for Visual Studio 2019 -- Fixed Visual Studio project files (version information from DLLVersion.rc not included in DLLs) -- Include version resource in DLLs built with CMake -- Added HTTP*Credentials::empty() and HTTPCredentials::clear() -- fixed GH #2220: Encoding/DoubleByteEncoding.cpp fails to compile with VS2008 and _DEBUG -- fixed GH #2243: DLLVersion.rc is excluded from build, missing detail information in properties of *.dll -- fixed GH #2277: SQLite null pointer dereference occurs when exception is being thrown -- fixed GH #2313: PollSet behaves differently on windows -- fixed GH #2316: cmake can't find MySQL and ODBC libraries -- fixed GH #2336: Omit ContentLength in WebSocket accept response -- fixed GH #2358: Don't include for later OpenSSL -- fixed GH #2364: Stringify escapes every unicode symbol when object contain an array -- fixed GH #2380: Calling Poco::Net::X509Certificate::addChainCertificate() leads to double free. -- fixed GH #2492: Net::Socket::address() crash on Android -- fixed GH #2549: Fix keepAlive in http client session -- fixed GH #2565: HTMLForm: optional enforcement of Content-Length instead of Chunked Transfer-Encoding -- fixed GH #2570: DialogSocket: receiveStatusMessage() - line length limit applies to entire multi-line message -- fixed GH #2583: Crypto library does not build with OpenSSL 1.0.0 -- fixed GH #2655: MongoDB Binary element to string - bug -- fixed GH #2661: Poco::Zip::ZipArchive cannot load new tomcat.zip file -- fixed GH #2700: Invalid read of memory in Poco::Environment::set which may cause crashes. -- fixed GH #2712: File_WIN32.cpp(168): error C2065: "_upath":Undeclared identifier -- fixed GH #2723: Access violation when trying to decompress .zip file with unsupported compression method. - - -Release 1.9.0 (2018-03-07) -========================== - -- Added additional text encodings, available in the new PocoEncodings library (GH #2165) -- Added Punycode support for resolving Internationalized Domain Names to Poco::Net::DNS (GH #2164) -- Added XDG Base Directory Specification support in Poco::Path, Poco::Util::SystemConfiguration - and Poco::Util::Application (GH #1609, GH #561, GH #1609) -- Added support for GCM ciphers in Crypto library (GH #2129) -- Poco::Net::RemoteSyslogChannel and Poco::Net::RemoteSyslogListener now have basic - support for RFC 5424 structured data (GH #2173) -- Poco::File now has methods for obtaining partition space (GH #1545) -- Added Poco::Net::Context::addCertificateAuthority() (GH #2197) -- Added Poco::AutoPtr::reset() and Poco::SharedPtr::reset() to improve compatibility - with std::shared_ptr() (GH #2172) -- fixed GH #703: Poco::Data::SQLite data types (INTEGER is now mapped to Int64) -- fixed GH #1426: inttypes.h is available on sun solaris -- fixed GH #1912: Run ping with custom data size #1912 -- fixed GH #2177: Run ping with custom timeout instead of the default -- fixed GH #2058 and GH #2095: Synchronization issue/deadlock in Poco::Util::Timer at destruction -- fixed GH #2089: Allow hyphen in HTTP authentication scheme names -- fixed GH #2106: Undefined behavior in Delegate::equals() -- fixed GH #2108: POCO SQLite Data Connector hangs for 20 secs waiting for thread timeouts on Windows -- fixed GH #2142: JSON::Object preserveOrder keys not synced on assignment -- fixed GH #2199 and GH #2188: Illegal header in zip file exception/assertion failure during JAR file decompression -- fixed GH #2203: Use MAX_ADDRESS_LENGTH to determine buffer size - - -Release 1.8.1 (2018-01-09) -========================== - -- Added Poco::File::linkTo() -- fixed GH #2044: Poco::Net::NetworkInterface::list does not list inactive interfaces - even when explicitly being asked for it. -- fixed GH #2042: Android abstract namespace local socket address -- fixed GH #2038: Poco::Net::MultipartWriter::createBoundary() always returns the same string. -- fixed GH #2020: SQLite not handling parameter count mismatch correctly. -- fixed GH #2012: Data/SQLite: Exception messages contain duplicate text -- fixed GH #2005: Upgraded bundled PCRE to 8.41 -- fixed GH #2000: Fix building XMLStreamParser with unbundled expat -- fixed GH #1603: fix MinGW 4.8.2 Compilation -- fixed GH #1991: Support building poco 1.8.0 as cmake sub-project -- fixed GH #2080: Bugs in Poco::Net::Socket::select when POCO_HAVE_FD_POLL is defined - - -Release 1.8.0.1 (2017-11-11) -============================ - -- Reverted change for GH #1828; DeflatingStreamBuf::sync() no longer flushes - underlying stream as this causes corruption for some Zip files. -- PocoDoc: fix for handling compiler configuration for Gradle builds. - - -Release 1.8.0 (2017-11-10) -========================== - -- Poco::Base64Encoder: add support for base64url encoding (GH #1967) -- Add Poco::Net::PollSet class to Net library (GH #1763) -- The Net library now supports Unix Domain Sockets, where available. -- Added stream parser (Poco::XML::XMLStreamParser) to XML library (GH #1697) -- Added Poco::Net::TCPServerConnectionFilter and Poco::Net::TCPServer::setConnectionFilter() - to support connection filtering and IP blacklisting (GH #1485) -- Added Redis library (GH #1383) -- Added Zip64 support to Zip library (GH #1356) -- Upgraded bundled SQLite to 3.21.0 -- Removed OpenVMS support (GH #1988) -- fixed GH #271: NamedMutex_UNIX.cpp must remove semid -- fixed GH #739: Add WebSocket::receiveFrame() that appends to a Poco::Buffer -- fixed GH #749: NTP Packet impl not according to RFC958 -- fixed GH #896: Sample "TwitterClient" of NetSSL_OpenSSL can't be build -- fixed GH #1172: Poco::Data default storage should be std::vector -- fixed GH #1337: Poco::HTMLForm throws exception HTMLFormException("Form must be prepared") - even after form is prepared. -- fixed GH #1373: SessionImpl::close() does not check return code of close handle specific function -- fixed GH #1425: Workaround bug in SolarisStudio 12.4 on RVO-ed objects. -- fixed GH #1614: Problematic license for JSON component: the previously used JSON.org parser - has been replaced with pdjson -- fixed GH #1659: wrong field size calculation in ODBC code -- fixed GH #1683: Poco::Data ODBC impl doesn't bind to unsigned numeric types properly -- fixed GH #1705: MongoDB: support URI in Connection -- fixed GH #1708: "SocketReactor::addEventHandler" and "SocketReactor::removeEventHandler" - must protect the access to "NotifierPtr pNotifier" -- fixed GH #1729: getConnectionTimeout of SQLite DB wrapper returns wrong value - (in milliseconds, should be in seconds) -- fixed GH #1739: OpenSSLInitializer isn't threadsafe -- fixed GH #1750: double_conversion in NumericString is in conflict with Qt5 Core -- fixed GH #1804 and GH #1805: Integer Overflow or Wraparound -- fixed GH #1828: DeflatingStreamBuf::sync() should also flush underlying stream. -- fixed GH #1880: FTPClientSession::close() error -- fixed GH #1897: DateTime wrong binding/extraction for MySQL database -- fixed GH #1905: Compiling Foundation library with POCO_NO_FPENVIRONMENT in Config.h fails -- fixed GH #1906: Race condition in ThreadPool -- fixed GH #1913: Message Doesn't Support 64-bit Thread IDs -- fixed GH #1921: ICMPSocket does not check reply address -- fixed GH #1926: Exception when using SortedDirectoryIterator -- fixed GH #1934: File::setExecutable() on POSIX should set executable bit for group and - other if corresponding readable bit is set -- fixed GH #1950: Net Exception: Address family not supported with clang -- fixed GH #1964: Buffer<> swap miss ownMem - - -Release 1.7.9p2 (2017-11-06) -============================ - -- fixed GH #1628: Export Poco::Zip::ZipUtil class - - -Release 1.7.9p1 (2017-11-02) -============================ - -- fixed GH #1968: Zip Decompress Parent Path Injection - - -Release 1.7.9 (2017-09-11) -========================== - -- fixed GH #1813: xmlparse.cpp doesn't compile in WinCE (poco 1.7.8p3) -- fixed GH #1826: XPath query error -- fixed GH #1834: Visual Studio 2008 cannot find stdint.h -- fixed GH #1842: Upgrade bundled expat to 2.2.3 -- fixed GH #1843: Use random salt for Poco::XML::NamePool -- fixed GH #1865: AbstractEvent::hasDelegates() is not thread-safe -- improved/fixed QNX support -- Poco::Util::LayeredConfiguration: added support for labelling configurations and - finding them by their label -- upgraded bundled SQLite to 3.20.1 -- PageCompiler: support <%@ include file="" %> syntax for includes, in addition - to <%@ include page="" %> -- PageCompiler: optimize generated request handler code by removing useless - statements, e.g. writing empty strings. -- added POCO_DEPRECATED macro which will be used in the future to deprecate - classes and methods. -- Poco::NamedMutex and Poco::NamedEvent (System V Semaphores implementation): files are - now opened with O_RDONLY | O_CREAT instead of O_WRONLY | O_CREAT, allowing sharing - between different users. Furthermore, ftok() is called with 'p' as project ID - argument. - - -Release 1.7.8p3 (2017-06-22) -============================ - -- fixed GH #1760: Upgrade bundled expat to 2.2.1 which fixes some vulnerabilities: - http://seclists.org/oss-sec/2017/q2/499 - - -Release 1.7.8p2 (2017-04-18) -============================ - -- fixed GH #1655: CipherImpl memory leak with OpenSSL 1.1 - - -Release 1.7.8 (2017-02-21) -========================== - -- fixed GH #1212: Lost WebSocket Frames after Client Websocket Handshake is complete -- fixed GH #1260: URI encoding -- fixed GH #1501: Alpine 3.4 trouble with Foundation/src/Error.cpp -- fixed GH #1523: Long path names under Windows -- fixed GH #1536: Building with OS X 10.12 SDK and 10.7 deployment target without libc++ fails -- fixed GH #1537: Need to add multiple cflags parameters to configure -- fixed GH #1539: Allow overriding POCO_TARGET_OSARCH for iPhoneSimulator -- fixed GH #1546: Enable bitcode for iPhone build config -- fixed GH #1549: Latin2Encoding and 0xFF -- fixed GH #1551: Unable to use Poco on macOS 10.12 -- fixed GH #1552: IPv6 & operator throws an exception when scope = 0 -- fixed GH #1566: Poco/Zip issue with some CM_DEFLATE archives -- fixed GH #1567: Poco/ZIP issue with uncompressed archives -- fixed GH #1570: IPv6AddressImpl::toString() returns wrong output for IPv6 address "::" -- fixed GH #1571: ODBC Preparator memory leak -- fixed GH #1573: Poco::File::createDirectories() should not throw Poco::FileExistsException -- fixed GH #1580: Unable to unzip zip file created using non-seeking stream -- fixed GH #1581: Cannot find 'pcre.h' when using POCO_UNBUNDLED, a non-system PCRE, and CMake -- fixed GH #1588: Poco::Net::HTTPChunkedStreamBuf::readFromDevice(): restrict maximum - size of chunk length -- fixed GH #1589: Poco::Net::HTMLForm: restrict maximum field and value length -- fixed GH #1590: Poco::Net::DialogSocket: restrict maximum line length -- fixed GH #1591: Poco::Net::MultipartReader: restrict maximum boundary string length -- fixed GH #1597: adding empty file to zip leads to archive that can't be unzipped by windows -- fixed GH #1599: readFromDevice() in AutoDetectStream.cpp in Poco Zip cannot detect signature -- fixed GH #1534: Upgraded bundled zlib to 1.2.11 -- fixed GH #1558: Upgraded bundled SQLite to 3.16.2 -- fixed GH #1586: Upgraded bundled PCRE to 8.40 -- fixed GH #1538: Upgraded bundled double-conversion to 1.1.5 -- MongoDB: added support for authentication using "MONGODB-CR" and "SCRAM-SHA-1" - authentication schemes. -- MongoDB: additional documentation and fixes for style and consistency and minor - API improvements (e.g., Poco::MongoDB::Binary) - Note: some flag enumeration values have been renamed for better consistency - and readability; existing code using these will have to be updated. - - -Release 1.7.7 (2016-12-31) -========================== - -- fixed GH #865: FileChannel compress fails leaving empty .gz files -- fixed GH #990: Potential race condition in Poco::File on Windows -- fixed GH #1157: Fixing a bug in the NetSSL_Win module (Host name verification failed error) -- fixed GH #1351: Fix for android include pthread.h from /usr/include -- fixed GH #1436: ODBC Bug: Unicode text(NVARCHAT) read from DB is truncated to half -- fixed GH #1453: _clock_gettime Symbol not found on Mac 10.11 -- fixed GH #1460: POCO does not build with OpenSSL 1.1 -- fixed GH #1461: Poco::Data::SQLite::SQLiteStatementImpl::next() error -- fixed GH #1462: AbstractConfiguration::getUInt does not parse hex numbers -- fixed GH #1464: ODBCMetaColumn::init() always maps integer NUMERIC/DECIMAL to Int32 -- fixed GH #1465: Assertion violation in DateTime.cpp using ZipArchive -- fixed GH #1472: HTTP(S)StreamFactory should send a User-Agent header. -- fixed GH #1476: Fixed error with Poco::UTF8Encoding::isLegal() -- fixed GH #1484: ODBC: fix uninitialized variable -- fixed GH #1486: Support ODBC GUID data type as string -- fixed GH #1488: Poco::ObjectPool shrinks if returned object is not valid -- fixed GH #1515: Detection of closed websocket connection -- fixed GH #1521: bug in JSON ParseHandler.cpp (empty keys should be valid) -- fixed GH #1526: iOS app rejected, IPv6 not working -- fixed GH #1532: RecordSet and RowFilter: bad use of reference counter - - -Release 1.7.6 (2016-10-18) -========================== - -- fixed GH #1298: ZipFileInfo: Assertion violation when reading ods files -- fixed GH #1315: Redefine Poco assertions for static analysis -- fixed GH #1397: Fix issues reported by static source code analysis -- fixed GH #1403: Android compile with poco-1.7.5 no 'pthread_condattr_setclock' error -- fixed GH #1416: Assertion violation when unzipping -- fixed GH #1418: Poco::Delegate assignment operator fails to compile for some specializations -- fixed GH #1422: Can't build poco 1.7.4 or 1.7.5 on centos5 32 bit -- fixed GH #1429: exception thrown in MongoDB when using replicaset -- fixed GH #1431: Poco/FIFOBuffer.h copy issue -- fixed GH #1445: Use stable_sort to preserve order of IP addresses from DNS -- fixed GH #1456: better handle leap seconds in Poco::DateTime and Poco::LocalDateTime -- fixed GH #1458: Probably invalid epoll_create() usage inside Poco/Socket.cpp -- Poco::XML::NamePool: increased default size from 251 to 509. Default size can now - be changed by defining the POCO_XML_NAMEPOOL_DEFAULT_SIZE macro accordingly. -- Enchancements: Poco::XML::Document and Poco::XML::DOMParser have new constructors - taking a NamePool size. Poco::Util::XMLConfiguration::load() also has a new overload - for that purpose. -- Improved error handling in the Zip library (getting rid of some poco_assert macros - and did proper error handling instead). -- Added Poco::URISyntaxException (subclass of Poco::SyntaxException), which is now - thrown by Poco::URI. -- Improved error handling in Poco::URIStreamOpener::open(). -- Poco::Data::MySQL: Handle connection lost/server gone error when starting a transaction - and retry. -- XMLConfiguration default (and single-argument delimiter) constructor now loads an empty - XML document with "config" root element to make the configuration usable without an - additional call to load() or loadEmpty(). - - -Release 1.7.5 (2016-08-29) -========================== - -- fixed GH #1252: Unable to compile Poco::Data for Windows Compact Embedded 2013 -- fixed GH #1344: Poco::Event::wait(timeout) should use CLOCK_MONOTONIC on Linux -- fixed GH #1355: [JSON::Object] After copy-ctor, JSON::Object::_keys still points to - keys in map of copied object -- GH #1361: Shell expansion rules say that tilde must be replaced with $HOME before - calling getpwuid -- Poco::SingletonHolder: added reset() method -- prefer clock_getttime() over gettimeofday() if available -- Upgraded bundled SQLite to 3.14.1 - - -Release 1.7.4 (2016-07-20) -========================== - -- fixed GH #1300: Session constructor hangs -- fixed GH #1303: HTTPSClientSession::sendRequest() fails if server has wildcard cert -- fixed GH #1304: URI doesn't know "ws:/" or "wss://" schemes -- fixed GH #1307: Upgrade bundled expat to 2.2.0 -- fixed GH #1313: XML library compilation error -- fixed GH #1316: Empty SocketReactor never sleeps -- Upgraded bundled SQLite to 3.13.0 - - -Release 1.7.3 (2016-05-02) -========================== - -- fixed GH #993: Invalid zip format when opening a docx in word -- fixed GH #1235: Poco::Net::HTTPClientSession::sendRequest() should also handle HTTP_PATCH -- fixed GH #1236: Remove Poco::Data::Row::checkEmpty() as it prevents Row from being used - with all NULL rows -- fixed GH #1239: Poco::Zip::Compress with non-seekable stream fails for CM_STORE -- fixed GH #1242: Poco::Data::RowFormatter generate exception if the first column of first - row is null -- fixed GH #1253: ListMap does not maintain insertion order if key already exists -- Upgraded bundled SQLite to 3.12.2 - - -Release 1.7.2 (2016-03-21) -========================== - -- fixed GH #1197: Upgrade bundled expat to 2.1.1 - Expat 2.1.1 fixes a CVE: https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2015-1283 -- fixed GH #1204: getdtablesize has been removed on Android 21 -- fixed GH #1203: Poco::Data::RecordSet should be reusable -- fixed GH #1198: Upgrade bundled SQLite to 3.12.1 - - -Release 1.7.1 (2016-03-14) -========================== - -- fixed GH #1187: Data/MySQL: Seeing frequent "MySQL server has gone away" errors -- fixed GH #1184: Attempting to connect via a proxy throws a DNS error "Host not found" -- fixed GH #1180: Possible deadlock when TaskManager::count() is called in onFinished -- NetSSL_OpenSSL: use TLS_*_method() instead of deprecated SSLv23_*_method() - if OpenSSL version is >= 1.1; initialize default/fallback client context to support - all TLS protocols, not just TLSv1 - - -Release 1.7.0 (2016-03-07) -========================== - -- POSSIBLE BREAKING CHANGE: removed automatic registration of Data connectors due to - issues with static initialization order. -- NetSSL_OpenSSL: added support for ECDH and DH ciphers; added support to disable - specific protocols (Poco::Net::Context::disableProtocols()); - new Poco::Net::Context constructor taking a Poco::Net::Context::Params structure that - allows specifying ECDH and DH parameters. -- Poco::Net::TCPServer: add additional try ... catch block around poll() to - gracefully deal with errors due to high system load (e.g., out of file descriptors). -- fixed GH #1171: Poco::Data::RecordSet: rowCount not reset after execute -- fixed GH #1167: CMake & POCO_UNBUNDLED: expat sources are compiled in libPocoXML -- fixed GH #1160: Poco::Net::NetException - "SSL Exception: error:1409F07F:SSL routines:ssl3_write_pending:bad write retry" -- fixed GH #1152: Wrong TaskProgressNotification description -- fixed GH #1141: Poco::StringTokenizer::TOK_TRIM changes behavior between 1.4 and 1.6 -- fixed GH #1137: Missing 'longint' type in SQLite -- fixed GH #1135: Different package on github and official web site -- fixed GH #1030: tvOS / WatchOS bitcode enabled for simulators -- fixed GH #1114: World-write permissions on files created by daemon -- fixed GH #1087: prevent line breaks in base64-encoded creds -- fixed GH #1026: Fixes for producing the poco-1.6.2 release on a Cygwin x86 platform -- fixed GH #1022: Abbreviation in setThreadName can happen even if thread name is not too long -- fixed GH #1002: ActiveDispatcher saves reference to event context after event was - performed until it gets new event -- fixed GH #973: overwrite existing files on windows when moving files -- fixed GH #969: Poco::File::renameTo() behaviour differs on windows and linux -- fixed GH #967: Missing data types in SQLite -- fixed GH #966: Possible crash when processing a corrupted Zip file -- fixed GH #958: Bug while reading X509Certificate subjectName -- fixed GH #937: Missing build_vs140.cmd -- fixed GH #933: Change in JSON::Object::set(key,value) behavior in 1.6.1 -- fixed GH #931: make strToInt() more strict in what it accepts -- fixed GH #921: `BasicUnbufferedStreamBuf` needs to be marked for import/export -- fixed GH #848: MailMessage::_encoding is not set when retrieving plain/text message -- fixed GH #767: Inconsistency in getPath & getPathAndQuery returns -- fixed GH #724: Poco 1.6.0 is not compiled with openssl 1.0.0 -- fixed GH #713: Improved support for producing Canonical XML in XMLWriter -- fixed GH #696: bug in parsing name of attachment poco c++ 1.6.0 -- fixed GH #335: Compress with nonseekable -- upgraded bundled SQLite to 3.11.0 -- added Poco::Crypto::X509Certificate::equals() to compare two certificates -- support for detecting Win8/Win10 in Poco::Environment -- Poco::Net::HTTPServerRequestImpl: fixed an issue with DELETE in persistent connections -- NetSSL: added Context::preferServerCiphers() -- NetSSL: added support for ECDH, new Context constructor -- NetSSL: add support for disabling certain protocols -- SMTPClientSession: added support for XOAUTH2 authentication -- Poco::Data::SessionPool: re-added customizeSession() method from 1.4.x releases -- improved SSLManager to automatically set-up a reasonable client Context if - none is configured -- add brew OpenSSL search paths to Darwin configs -- add HTTP/1.1 version to HTTPRequest for client WebSocket, as this is required for - most servers -- remove GCC_DIAG_OFF as this caused more issues than it solved -- respect POCO_NO_FORK_EXEC in ServerApplication (tvOS) -- tvOS and WatchOS support -- fix: need an implementation of available() for WebSocketImpl -- HTTPSessionInstantiator: respect global proxy config -- added constant for HTTP PATCH method to Poco::Net::HTTPRequest -- NumberParser::parseHex[64](): allow 0x/0X prefix - - -Release 1.6.1 (2015-08-03) -========================== - -- added project and solution files for Visual Studio 2015 -- upgraded bundled SQLite to 3.8.11.1 -- fixed GH #782: Poco::JSON::PrintHandler not working for nested arrays -- fixed GH #819: JSON Stringifier fails with preserve insert order -- fixed GH #878: UUID tryParse -- fixed GH #869: FIFOBuffer::read(T*, std::size_t) documentation inaccurate -- fixed GH #861: Var BadCastException -- fixed GH #779: BUG in 1.6.0 Zip code -- fixed GH #769: Poco::Var operator== throws exception -- fixed GH #766: Poco::JSON::PrintHandler not working for objects in array -- fixed GH #763: Unable to build static with NetSSL_OpenSSL for OS X -- fixed GH #750: BsonWriter::write missing size ? -- fixed GH #741: Timestamp anomaly in Poco::Logger on WindowsCE -- fixed GH #735: WEC2013 build fails due to missing Poco::Path methods. -- fixed GH #722: poco-1.6.0: Unicode Converter Test confuses string and char types -- fixed GH #719: StreamSocket::receiveBytes and FIFOBuffer issue in 1.6 -- fixed GH #706: POCO1.6 Sample EchoServer BUG -- fixed GH #646: Prevent possible data race in access to Timer::_periodicInerval -- DeflatingStream: do not flush underlying stream on sync() as these can cause - corrupted files in Zip archives - - -Release 1.6.0 (2014-12-22) -========================== - -- fixed GH #625: MongoDB ensureIndex double insert? -- fixed GH #622: Crypto: RSATest::testSign() should verify with public key only -- fixed GH #620: Data documentation sample code outdated -- fixed GH #618: OS X 10.10 defines PAGE_SIZE macro, conflicts with PAGE_SIZE in Thread_POSIX.cpp -- fixed GH #616: Visual Studio warning C4244 -- fixed GH #612: OpenSSLInitializer calls OPENSSL_config but not CONF_modules_free -- fixed GH #608: (Parallel)SocketAcceptor ctor/dtor call virtual functions -- fixed GH #607: Idle Reactor high CPU usage -- fixed GH #606: HTMLForm constructor read application/x-www-form-urlencoded UTF-8 request - body first parameter with BOM in name -- fixed GH #596: For OpenSSL 1.0.1, include openssl/crypto.h not openssl/fips.h -- fixed GH #592: Incorrect format string in Poco::Dynamic::Struct -- fixed GH #590: Poco::Data::SQlite doesn't support URI filenames -- fixed GH #564: URI::encode -- fixed GH #560: DateTime class calculates a wrong day -- fixed GH #549: Memory allocation is not safe between fork() and execve() -- fixed GH #500: SSLManager causes a crash -- fixed GH #490: 2 byte frame with payload length of 0 throws "Incomplete Frame Received" exception -- fixed GH #483: multiple cases for sqlite_busy -- fixed GH #482: Poco::JSON::Stringifier::stringify bad behaviour -- fixed GH #478: HTTPCredentials not according to HTTP spec -- fixed GH #471: vs2010 release builds have optimization disabled ? -- fixed GH #468: HTTPClientSession/HTTPResponse not forwarding exceptions -- fixed GH #438: Poco::File::setLastModified() doesn't work -- fixed GH #402: StreamSocket::receiveBytes(FIFOBuffer&) and sendBytes(FIFOBuffer&) are - not thread safe -- fixed GH #345: Linker warning LNK4221 in Foundation for SignalHandler.obj, String.obj - and ByteOrder.obj -- fixed GH #331: Poco::Zip does not support files with ".." in the name. -- fixed GH #318: Logger local time doesn't automatically account for DST -- fixed GH #294: Poco::Net::TCPServerParams::setMaxThreads(int count) will not accept count == 0. -- fixed GH #215: develop WinCE build broken -- fixed GH #63: Net::NameValueCollection::size() returns int -- Poco::Logger: formatting methods now support up to 10 arguments. -- added Poco::Timestamp::raw() -- Poco::DeflatingOutputStream and Poco::InflatingOutputStreams also flush underlying stream - on flush()/sync(). -- Poco::Util::Timer: prevent re-schedule of cancelled TimerTask -- enabled WinRegistryKey and WinRegistryConfiguration for WinCE -- Poco::BasicEvent improvements and preparations for future support of lambdas/std::function -- upgraded bundled sqlite to 3.8.7.2 -- Poco::Thread: added support for starting functors/lambdas -- Poco::Net::HTTPClientSession: added support for global proxy configuration -- added support for OAuth 1.0/2.0 via Poco::Net::OAuth10Credentials and - Poco::Net::OAuth20Credentials classes. -- Poco::Net::IPAddress: fixed IPv6 prefix handling issue on Windows -- added Poco::Timestamp::TIMEVAL_MIN and Poco::Timestamp::TIMEVAL_MAX -- added Poco::Clock::CLOCKVAL_MIN and Poco::Clock::CLOCKVAL_MAX -- added poco_assert_msg() and poco_assert_msg_dbg() macros -- Poco::Net::Context: fixed a memory leak if the CA file was not found while creating the - Context object (the underlying OpenSSL context would leak) -- Poco::URI: added new constructor to create URI from Path -- Various documentation and style fixes -- Removed support (project/solution files) for Visual Studio.NET 2003 and Visual Studio 2005. -- Improved CMake support - - -Release 1.5.4 (2014-10-14) -========================== - -- fixed GH #326: compile Net lib 1.5.2 without UTF8 support enabled -- fixed GH #518: NetworkInterface.cpp compile error w/ POCO_NO_WSTRING (1.5.3) -- Fixed MSVC 2010 warnings on large alignment -- make HTTPAuthenticationParams::parse() add value on end of string -- fixed GH #482: Poco::JSON::Stringifier::stringify bad behaviour -- fixed GH #508: Can't compile for arm64 architecture -- fixed GH #510: Incorrect RSAKey construction from istream -- fix SharedMemory for WinCE/WEC2013 -- Add NIOS2 double conversion detection, fixes compile errors -- added VS2013 project/solution files for Windows Embedded Compact 2013 -- added Process::isRunning() -- NetSSL: Fix typo in documentation -- NetSSL_OpenSSL: support for TLS 1.1 and 1.2 -- Zip: Added CM_AUTO, which automatically selects CM_STORE or CM_DEFLATE based - on file extension. Used to avoid double-compression of already compressed file - formats such as images. -- added %L modifier to PatternFormatter to switch to local time -- removed unnecessary explicit in some multi-arg constructors -- Allow SecureStreamSocket::attach() to be used in server connections -- added Var::isBoolean() and fixed JSON stringifier -- added poco_unexpected() macro invoking Bugcheck::unexpected() to deal - with unexpected exceptions in destructors -- fixed GH #538 prevent destructors from throwing exceptions -- improved HTTP server handling of errors while reading header -- fixed GH #545: use short for sign -- upgraded SQLite to 3.8.6 -- fixed GH #550 WebSocket fragmented message problem -- improved HTTPClientSession handling of network errors while sending the request -- updated bundled PCRE to 8.35.0 -- fixed GH #552: FIFOBuffer drain() problem -- fixed GH #402: StreamSocket::receiveBytes(FIFOBuffer&) and sendBytes(FIFOBuffer&) are - not thread safe -- HTTPCookie: fix documentation for max age -- added Timestamp::raw() and Clock::raw() -- Poco::Buffer properly handles zero-sized buffers -- GH #512: Poco:Data:ODBC:Binder.h causes a crash -- Added Crypto_Win and NetSSL_Win libraries which are re-implementations of existing - Crypto and NetSSL_OpenSSL libraries based on WinCrypt/Schannel. The new libraries - can be used as an almost drop-in replacement for the OpenSSL based libraries on - Windows and Windows Embedded Compact platforms. Only available from GitHub for now. - - -Release 1.5.3 (2014-06-30) -========================== - -- fixed GH# 316: Poco::DateTimeFormatter::append() gives wrong result for - Poco::LocalDateTime -- Poco::Data::MySQL: added SQLite thread cleanup handler -- Poco::Net::X509Certificate: improved and fixed domain name verification for - wildcard domains -- added Poco::Clock class, which uses a system-provided monotonic clock - (if available) and is thus not affected by system realtime clock changes. - Monotonic Clock is available on Windows, Linux, OS X and on POSIX platforms - supporting clock_gettime() and CLOCK_MONOTONIC. -- Poco::Timer, Poco::Stopwatch, Poco::TimedNotificationQueue and Poco::Util::Timer - have been changed to use Poco::Clock instead of Poco::Timestamp and are now - unaffected by system realtime clock changes. -- fixed GH# 350: Memory leak in Data/ODBC with BLOB -- Correctly set MySQL time_type for Poco::Data::Date. -- fixed GH #352: Removed redundant #includes and fixed spelling mistakes. -- fixed setting of MYSQL_BIND is_unsigned value. -- fixed GH #360: CMakeLists foundation: add Clock.cpp in the list of source files -- Add extern "C" around on HPUX platform. -- added runtests.sh -- fixed CPPUNIT_IGNORE parsing -- fixed Glob from start path, for platforms not alowing transverse from root (Android) -- added NTPClient (Rangel Reale) -- added PowerShell build script -- added SmartOS build support -- fix warnings in headers -- XMLWriter: removed unnecessary apostrophe escaping (&apos) -- MongoDB: use Int32 for messageLength -- fixed GH #380: SecureSocket+DialogSocket crashes with SIGSEGV when timeout occours -- Improve RSADigestEngine, using Poco::Crypto::DigestEngine to calculate hash before signing -- added Poco::PBKDF2Engine -- Fixed GH #380: SecureSocket+DialogSocket crashes with SIGSEGV when timeout occours -- added support for a 'Priority' attribute on cookies. -- GH #386: fixed bug in MailMessage without content-transfer-encoding header -- GH #384: ew hash algorithms support for RSADigestEngine -- fixed Clock overflow bug on Windows -- Poco::ByteOrder now uses intrinsics, if available -- CMake: added /bigobj option for msvc -- Fix typo to restore Net/TestSuite_x64_vs120 build -- correct path for CONFIGURE_FILE in CMakeLists.txt -- Building Poco 1.5.2 for Synology RS812+ (Intel Atom) (honor POCO_NO_INOTIFY) -- added WEC2013 support to buildwin.cmd and buildwin.ps1 -- HTMLForm: in URL encoding, percent-encode more characters -- Fixed #include conflict with other libraries -- Poco::Net::X509Certificate::verify() no longer uses DNS reverse lookups to validate host names -- cert hostname validation is case insensitive and stricter for wildcard certificates -- TCPServer: do not reduce the capacity of the default ThreadPool -- added POCO_LOG_DEBUG flag -- Zip: fixed a crash caused by an I/O error -- added runtest script for windows -- added SQlite Full Text Search support -- added Thread::trySleep() and Thread::wakeUp() -- fixed GH #410: Bug in JSON::Object.stringify() in 1.5.2 -- fixed GH #362: Defect in Var::parseString when there is no space between value and newline -- fixed GH #314: JSON parsing bug -- added GH #313: MetaColumn additions for Data::ODBC and Data::SQLite -- fixed GH #346: Make Poco::Data::Date and Poco::Data::Time compare functions const. -- fixed GH #341: Compiling poco-1.5.2 for Cygwin -- fixed GH #305: There are bugs in Buffer.h -- fixed GH #321: trivial build fixes (BB QNX build) -- fixed GH #440: MongoDB ObjectId string formatting -- added SevenZip library (Guenter Obiltschnig) -- fixed GH #442: Use correct prefix length field of Windows IP_ADAPTER_PREFIX structure -- improved GH #328: NetworkInterface on Windows XP -- fixed GH #154 Add support for MYSQL_TYPE_NEWDECIMAL to Poco::Data::MySQL -- fixed GH #290: Unicode support -- fixed GH #318: Logger local time doesn't automatically account for DST -- fixed GH #363: DateTimeParser tryParse/parse -- added HTMLForm Content-Length calculation (Rangel Reale) -- Make TemporaryFile append a slash to tempDir -- fixed GH #319 android build with cmake -- added hasDelegates() method to AbstractEvent -- fixed GH #230: Poco::Timer problem -- fixed GH #317: Poco::Zip does not support newer Zip file versions. -- fixed GH #176: Poco::JSON::Stringifier UTF encoding -- fixed GH #458: Broadcast address and subnet mask for IEEE802.11 network interface -- fixed GH #456: poco: library install dirs per RUNTIME/LIBRARY/ARCHIVE - - -Release 1.5.2 (2013-09-16) -========================== - -- added MongoDB library -- fixed GH #57: poco-1.5.1: Doesn't compile for Android -- added VoidEvent (Arturo Castro) -- fixed GH #80: NumberFormatter::append broken -- fixed GH #93: ParallelSocketAcceptor virtual functions -- optional small object optimization for IPAddress, SocketAddress, Any and Dynamic::Var -- SQLite events (insert, update, delete, commit, rollback) handlers -- merged GH #91: Improve SQLite multi-threaded use (Rangel Reale) -- merged GH #86: Invalid pointers to vector internals (Adrian Imboden) -- automatic library initialization macros -- fixed GH #110: WebSocket accept() fails when Connection header contains multiple tokens -- fixed GH #71: WebSocket and broken Timeouts (POCO_BROKEN_TIMEOUTS) -- fixed a warning in Poco/Crypto/OpenSSLInitializer.h -- fixed GH #109: Bug in Poco::Net::SMTPClientSession::loginUsingPlain -- added clang libc++ build configurations for Darwin and iPhone (Andrea Bigagli) -- fixed GH #116: Wrong timezone parsing in DateTimeParse (Matej Knopp) -- fixed GH #118: JSON::Object::stringify endless loop -- added Recursive and SortedDirectoryIterator (Marian Krivos) -- added ListMap (map-like container with preserving insertion order) -- MailMessage: attachments saving support and consistent read/write -- fixed GH #124: Possible buffer overrun in Foundation/EventLogChannel -- fixed GH #119: JSON::Object holds values in ordered map -- added JSON::PrintHandler -- renamed JSON::DefaultHandler to ParseHandler (breaking change!) -- fixed GH #127: Eliminate -Wshadow warnings -- fixed GH #79: Poco::Thread leak on Linux -- fixed GH #61: static_md build configs for Crypto and NetSSL -- fixed GH #130: prefer sysconf over sysctlbyname -- fixed GH #131: no timezone global var on OpenBSD -- fixed GH #102: Some subprojects don't have x64 solutions for VS 2010 -- added GH #75: Poco::Uri addQueryParameter method -- Poco::Environment::osDisplayName() now recognizes Windows 8/Server 2012 -- fixed GH #140: Poco::Runnable threading cleanup issue -- simplified default TCP/HTTPServer construction -- fixed GH #141: Application::run() documentation/implementation discrepancy -- changed RowFormatter to SharedPtr in Data::RecordSet interface (breaking change!) -- fixed GH #144: Poco::Dynamic emits invalid JSON -- removed naked pointers from Data interfaces -- fixed GH #82: name conflict in Data::Keywords::bind -- fixed GH #157: MySQL: cannot bind to 'long' data type on Windows/Visual C++ -- fixed GH #158: MySQL: MYSQL_BIND 'is_unsigned' member is not set -- fixed GH #160: MultipartReader ignores first part, if preamble is missing -- fixed GH #156: Possible buffer overrun in Foundation/EventLogChannel -- XML: fixed an issue with parsing a memory buffer > 2 GB -- upgraded to expat 2.1.0 -- Data/ODBC: added support for setting query timeout (via setProperty - of "queryTimeout"). Timeout is int, given in seconds. -- fixed a potential endless loop in SecureStreamSocketImpl::sendBytes() - and also removed unnecessary code. -- fixed GH #159: Crash in openssl CRYPTO_thread_id() after library libPocoCrypto.so - has been unloaded. -- fixed GH #155: MailOutputStream mangles consecutive newline sequences -- fixed GH #139: FileChannel::PROP_FLUSH is invalid (contains a tab character) -- fixed GH #173: HTTPClientSession::proxyConnect forces DNS lookup of host names -- fixed GH #194: MessageNotification constructor is inefficient. -- fixed GH #189: Poco::NumberParser::tryParse() documentation bug -- fixed GH #172: IPv6 Host field is stripped of Brackets in HTTPClientSession -- fixed GH #188: Net: SocketAddress operator < unusable for std::map key -- fixed GH #128: DOMWriter incorrectly adds SYSTEM keyword to DTD if PUBLIC is - already specified -- fixed GH #65: Poco::format() misorders sign and padding specifiers -- upgraded bundled SQLite to 3.7.17 -- replaced JSON parser with Poco::Web::JSON parser (from sandbox) -- added JSON conversion to Dynamic Struct and Array -- added VarIterator -- modified behavior of empty Var (empty == empty) -- added Alignment.h header for C++03 alignment needs -- added Data/WebNotifier (DB, WebSocket) example -- fixed GH #209: Poco::NumberFormatter double length -- fixed GH #204: Upgrade zlib to 1.2.8 -- fixed GH #198: The "application.configDir" property is not always created. -- fixed GH #185: Poco::NumberFormatter::format(double value, int precision) - ignore precision == 0 -- fixed GH #138: FreeBSD JSON tests fail -- fixed GH #99: JSON::Query an JSON::Object -- limited allowed types for JSON::Query to Object, Array, Object::Ptr, - Array::Ptr and empty -- fixed GH #175: HTMLForm does not read URL parameters on POST or PUT -- added GH #187: MySQL: allow access to the underlying connection handle -- added GH #186: MySQL: support for MYSQL_SECURE_AUTH -- fixed GH #174: MySQL: 4GB allocated when reading any largetext or largeblob field -- fixed a potential memory leak in Poco::Net::HTTPClientSession if it is misused - (e.g., sendRequest() is sent two times in a row without an intermediate call to - receiveResponse(), or by calling receiveResponse() two times in a row without - an intermediate call to sendRequest()) - GH #217 -- removed a few unnecessary protected accessor methods from Poco::Net::HTTPClientSession - that would provide inappropriate access to internal state -- merged GH #210: Don't call CloseHandle() twice on Windows; Ability to select the - threadpool that will be used to start an Activity(Patrice Tarabbia) -- fixed GH #212: JSONConfiguration was missing from the vs90 project(Patrice Tarabbia) -- fixed GH #220: add qualifiers for FPEnvironment in C99 (Lucas Clemente) -- fixed GH #222: HTTPCookie doesn't support expiry times in the past (Karl Reid) -- fixed GH #224: building 1.5.1 on Windows for x64 -- fixed GH# 233: ServerSocket::bind6(Poco::UInt16 port, bool reuseAddress, bool ipV6Only) does not work -- fixed GH# 231: Compatibility issue with Poco::Net::NetworkInterface -- fixed GH# 236: Bug in RecursiveDirectoryIterator -- added ColorConsoleChannel and WindowsColorConsoleChannel classes supporting - colorizing log messages -- fixed GH# 259: Poco::EventLogChannel fails to find 64bit Poco Foundation dll -- fixed GH# 254: UTF8::icompare unexpected behavior -- Poco::UUID::tryParse() also accepts UUIDs without hyphens. Also updated documentation - (links to specifications). -- added GH# 268: Method to get JSON object value using Poco::Nullable -- fixed GH# 267: JSON 'find' not returning empty result if object is expected but another value is found -- Added support for ARM64 architecture and iPhone 5s 64-bit builds - (POCO_TARGET_OSARCH=arm64). - - -Release 1.5.1 (2013-01-11) -========================== - -- using double-conversion library for floating-point numeric/string conversions -- added Poco::istring (case-insensitive string) and Poco::isubstr -- added SQLite sys.dual (in-memory system table) -- applied SF Patch #120: The ExpireLRUCache does not compile with a tuple as key on Visual Studio 2010 -- fixed SF Bug #599: JSON::Array and JSON::Object size() member can implicitly lose precision -- fixed SF Bug #602: iterating database table rows not correct if no data in table -- fixed SF Bug #603: count() is missing in HashMap -- fixed GH #23: JSON::Object::stringify throw BadCastException -- fixed GH #16: NetworkInterface::firstAddress() should not throw on unconfigured interfaces -- Android compile/build support (by Rangel Reale) -- TypeHandler::prepare() now takes const-reference -- fixed GH #27: Poco::URI::decode() doesn't properly handle '+' -- fixed GH #31: JSON implementation bug -- fixed SF #597: Configure script ignores cflags -- fixed SF #593: Poco 1.5.0 on FreeBSD: cannot find -ldl -- added SF #542: SocketAddress() needs port-only constructor -- fixed SF #215: Wrong return type in SocketConnector.h -- applied SF Patch #97: fix c++0x / clang++ bugs -- fixed GH32/SF596: Poco::JSON: Parsing long integer (int64) value fails. -- added Net ifconfig sample (contributed by Philip Prindeville) -- merged GH #34: add algorithm header (Roger Meier/Philip Prindeville) -- fixed GH #26: Cannot compile on gcc -- merged SF #111: FTP Client logging (Marian Krivos) -- fixed GH #30: Poco::Path::home() throws when called from Windows Service -- fixed GH #22: MySQL connection string lowercased -- added MySQL support for Date/Time -- upgraded SQLite to version 3.7.15.1 (2012-12-19) -- improved SQLite execute() return (affected rows) value and added tests -- added SQLite::Utility::isThreadSafe() function -- added SQLite::Utility::setThreadMode(int mode) function -- fixed GH #36: 'distclean' requires 3 traversals of project tree -- fixed GH #41: Buffer::resize crash -- fixed GH #42: Linux unbundled builds don't link -- fixed GH #44: Problems with win x64 build -- fixed GH #46: 1.5.1 build fails on OS X when using libc++ -- fixed GH #48: Need getArgs() accessor to Util::Application to retrieve start-up arguments -- fixed GH #49: NetworkInterface::list doesn't return MAC addresses -- fixed GH #51: Android should use isfinite, isinf, isnan and signbit from the std namespace -- fixed GH #53: JSON unicode fixes and running tests on invalid unicode JSON -- added ParallelAcceptor and ParallelReactor classes -- added EOF and error to FIFOBuffer - - -Release 1.5.0 (2012-10-14) -========================== - -- added JSON library -- added Util::JSONConfiguration -- added FIFOBuffer and FIFOBufferStream -- fixed SF# 3522906: Unregistering handlers from SocketReactor -- fixed SF# 3522084: AbstractConfiguration does not support 64-bit integers -- HTTPServer::stopAll(): close the socket instead of just shutting it down, as the latter won't wake up a select() on Windows -- added SMTPLogger -- added cmake support -- fixed SF#3538778: NetworkInterface enumeration uses deprecated API -- fixed SF#3538779: IPAddress lacks useful constructors: from prefix mask, native SOCKADDR -- fixed SF#3538780: SocketAddress needs operator < function -- fixed SF#3538775: Issues building on Fedora/Centos, etc. for AMD64 -- fixed SF#3538786: Use size_t for describing data-blocks in DigestEngine -- added IPAddress bitwise operators (&,|,^,~) -- added IPAddress BinaryReader/Writer << and >> operators -- modified IPAddress to force IPv6 to lowercase (RFC 5952) -- fixed SF#3538785: SMTPClientSession::sendMessage() should take recipient list -- added IPAddress::prefixLength() -- UTF portability improvements -- fixed SF#3556186: Linux shouldn't use in Net/SocketDefs.h -- added IPAddress RFC 4291 compatible site-local prefix support -- fixed SF#3012166: IPv6 patch -- added SF#3558085: Add formatter to MACAddress object -- fixed SF#3552774: Don't hide default target in subordinate makefile -- fixed SF#3534307: Building IPv6 for Linux by default -- fixed SF#3516844: poco missing symbols with external >=lipcre-8.13 -- added SF#3544720: AbstractConfigurator to support 64bit values -- fixed SF#3522081: WinRegistryConfiguration unable to read REG_QWORD values -- fixed SF#3563626: For Win32 set Up/Running flags on NetworkInterface -- fixed SF#3560807: Deprecate setPeerAddress() as this is now done in getifaddrs -- fixed SF#3560776: Fix byte-ordering issues with INADDR_* literals -- fixed SF#3563627: Set IP address on multicast socket from socket family -- fixed SF#3563999: Size BinaryWriter based on buffer's capacity(), not size() -- fixed SF#102 Fix building Poco on Debian GNU/kFreeBSD -- fixed SF#321 Binding DatTime or Timestamp -- fixed SF#307 Detect the SQL driver type at run time -- added VS 2012 Projects/Solutions -- enhanced and accelerated numeric parsing for integers and floats -- fixed SF#590 Segfault on FreeBSD when stack size not rounded -- added warn function and warnmsg macro in CppUnit -- fixed SF# 3558012 Compilation fails when building with -ansi or -std=c++0x -- fixed SF# 3563517 Get rid of loss-of-precision warnings on x64 MacOS -- fixed SF#3562244: Portability fix for AF_LINK -- fixed SF #3562400: DatagramSocketImpl comment is incorrect - - -Release 1.4.7p1 (2014-11-25) -============================ - -- Fixed Visual C++ 2010-2013 project files. Release builds now have optimization enabled. -- Poco::URI: added constructor to create URI from Path. -- fixed GH #618: OS X 10.10 defines PAGE_SIZE macro, conflicts with PAGE_SIZE in Thread_POSIX.cpp -- Poco::Net::HTTPClientSession: added support for global proxy configuration -- fixed GH #331: Poco::Zip does not support files with .. in the name. -- fixed a memory leak in Poco::Net::Context constructor when it fails to load the certificate - or private key files. -- upgraded bundled SQLite to 3.8.7.2 -- fixed GH #229: added missing value() function -- fixed GH #69: MySQL empty text/blob - - -Release 1.4.7 (2014-10-06) -========================== - -- fixed GH #398: PropertyFileConfiguration: input != output -- fixed GH #368: Build failure of Poco 1.4.6p2 on FreeBSD 9.2 -- fixed GH #318: Logger local time doesn't automatically account for DST -- fixed GH #317: Poco::Zip does not support newer Zip file versions. -- fixed GH #454: Fix: handle unhandled exceptions -- fixed GH #463: XML does not compile with XML_UNICODE_WCHAR_T -- fixed GH #282: Using Thread in a global can cause crash on Windows -- fixed GH #424: Poco::Timer deadlock -- fixed GH #465: Fix result enum type XML_Error -> XML_Status -- fixed GH #510: Incorrect RSAKey construction from istream -- fixed GH #332: POCO::ConsoleChannnel::initColors() assigns no color to - PRIO_TRACE and wrong color to PRIO_FATAL -- fixed GH #550: WebSocket fragmented message problem -- Poco::Data::MySQL: added SQLite thread cleanup handler -- Poco::Net::X509Certificate: improved and fixed domain name verification for - wildcard domains -- fixed a crash in Foundation testsuite with Visual C++ 2012 -- improved and fixed domain name verification for wildcard domains in - Poco::Net::X509Certificate -- updated TwitterClient sample to use new 1.1 API and OAuth -- added Poco::Clock class, which uses a system-provided monotonic clock - (if available) and is thus not affected by system realtime clock changes. - Monotonic Clock is available on Windows, Linux, OS X and on POSIX platforms - supporting clock_gettime() and CLOCK_MONOTONIC. -- Poco::Timer, Poco::Stopwatch, Poco::TimedNotificationQueue and Poco::Util::Timer - have been changed to use Poco::Clock instead of Poco::Timestamp and are now - unaffected by system realtime clock changes. -- added Poco::PBKDF2Engine class template -- Poco::Net::HTTPCookie: added support for Priority attribute (backport from develop) -- fixed makedepend.* scripts to work in paths containing '.o*' - (contributed by Per-Erik Bjorkstad, Hakan Bengtsen) -- Upgraded bundled SQLite to 3.8.6 -- Support for Windows Embedded Compact 2013 (Visual Studio 2012) -- Project and solution files for Visual Studio 2013 -- Changes for C++11 compatibility. -- fixed an issue with receiving empty web socket frames (such as ping) -- improved error handling in secure socket classes -- Poco::ByteOrder now uses intrinsics if available -- added new text encoding classes: Latin2Encoding, Windows1250Encoding, Windows1251Encoding -- Zip: Added CM_AUTO, which automatically selects CM_STORE or CM_DEFLATE based on file extension. - Used to avoid double-compression of already compressed file formats such as images. - - -Release 1.4.6p4 (2014-04-18) -============================ - -- no longer use reverse DNS lookups for cert hostname validation -- cert hostname validation is case insensitive and more strict -- HTMLForm: in URL encoding, percent-encode more special characters -- fixed thread priority issues on POSIX platforms with non-standard scheduling policy -- XMLWriter no longer escapes apostrophe character -- fixed GH #316: Poco::DateTimeFormatter::append() gives wrong result for Poco::LocalDateTime -- fixed GH #305 (memcpy in Poco::Buffer uses wrong size if type != char) -- Zip: fixed a crash caused by an I/O error (e.g., full disk) while creating a Zip archive - - -Release 1.4.6p3 (2014-04-02) -============================ - -- Fixed a potential security vulnerability in client-side X509 - certificate verification. - - -Release 1.4.6p2 (2013-09-16) -============================ - -- fixed GH #156: Possible buffer overrun in Foundation/EventLogChannel -- XML: fixed an issue with parsing a memory buffer > 2 GB -- upgraded to expat 2.1.0 -- Data/ODBC: added support for setting query timeout (via setProperty - of "queryTimeout"). Timeout is int, given in seconds. -- fixed a potential endless loop in SecureStreamSocketImpl::sendBytes() - and also removed unnecessary code. -- fixed GH #159: Crash in openssl CRYPTO_thread_id() after library libPocoCrypto.so - has been unloaded. -- fixed GH #155: MailOutputStream mangles consecutive newline sequences -- fixed GH# 139: FileChannel::PROP_FLUSH is invalid (contains a tab character) -- fixed GH# 173: HTTPClientSession::proxyConnect forces DNS lookup of host names -- fixed GH# 194: MessageNotification constructor is inefficient. -- fixed GH# 189: Poco::NumberParser::tryParse() documentation bug -- fixed GH# 172: IPv6 Host field is stripped of Brackets in HTTPClientSession -- fixed GH# 188: Net: SocketAddress operator < unusable for std::map key -- fixed GH# 128: DOMWriter incorrectly adds SYSTEM keyword to DTD if PUBLIC is - already specified -- fixed GH# 65: Poco::format() misorders sign and padding specifiers -- upgraded bundled SQLite to 3.7.17 -- upgraded bundled zlib to 1.2.8 -- fixed a potential memory leak in Poco::Net::HTTPClientSession if it is misused - (e.g., sendRequest() is sent two times in a row without an intermediate call to - receiveResponse(), or by calling receiveResponse() two times in a row without - an intermediate call to sendRequest()) - GH #217 -- removed a few unnecessary protected accessor methods from Poco::Net::HTTPClientSession - that would provide inappropriate access to internal state -- fixed GH# 223 (Poco::Net::HTTPCookie does not support expiry times in the past) -- fixed GH# 233: ServerSocket::bind6(Poco::UInt16 port, bool reuseAddress, bool ipV6Only) - does not work -- added ColorConsoleChannel and WindowsColorConsoleChannel classes supporting - colorizing log messages -- fixed GH# 259: Poco::EventLogChannel fails to find 64bit Poco Foundation dll -- fixed GH# 254: UTF8::icompare unexpected behavior -- Poco::UUID::tryParse() also accepts UUIDs without hyphens. Also updated documentation - (links to specifications). -- Added support for ARM64 architecture and iPhone 5s 64-bit builds - (POCO_TARGET_OSARCH=arm64). - - -Release 1.4.6p1 (2013-03-06) -============================ - -- fixed GH# 71: WebSocket and broken Timeouts (POCO_BROKEN_TIMEOUTS) -- fixed an ambiguity error with VC++ 2010 in Data/MySQL testsuite -- Poco::Net::NetworkInterface now provides the interface index even for IPv4 -- added DNS::reload() as a wrapper for res_init(). -- On Linux, Poco::Environment::nodeId() first always tries to obtain the - MAC address of eth0, before looking for other interfaces. -- Poco::Net::HTTPSession now always resets the buffer in connect() to clear - any leftover data from a (failed) previous session -- fixed copysign namespace issue in FPEnvironment_DUMMY.h -- fixed a warning in Poco/Crypto/OpenSSLInitializer.h -- added a build configuration for BeagleBoard/Angstrom -- fixed GH# 109: Bug in Poco::Net::SMTPClientSession::loginUsingPlain) -- fixed compile errors with clang -std=c++11 -- fixed GH# 116: Wrong timezone parsing in DateTimeParse (fix by Matej Knopp) -- updated bundled SQLite to 3.7.15.2 - - -Release 1.4.6 (2013-01-10) -========================== - -- changed FPEnvironment_DUMMY.h to include instead of -- updated bundled SQLite to 3.7.15.1 -- fixed GH# 30: Poco::Path::home() throws -- fixed SF Patch# 120 The ExpireLRUCache does not compile with a tuple as key on VS2010 -- fixed SF# 603 count() is missing in HashMap -- Crypto and NetSSL_OpenSSL project files now use OpenSSL *MD.lib library files for - static_md builds. Previously, the DLL import libs were used. -- Poco::Environment::osDisplayName() now recognizes Windows 8/Server 2012 - - -Release 1.4.5 (2012-11-19) -========================== - -- added Visual Studio 2012 project files -- buildwin.cmd now support building with msbuild for VS2010 and 2012. -- added Poco::Optional class -- fixed SF# 3558012 Compilation fails when building with -ansi or -std=c++0x -- fixed SF# 3563517 Get rid of loss-of-precision warnings on x64 MacOS -- fixed SF# 3562244: Portability fix for AF_LINK -- fixed SF# 3562400: DatagramSocketImpl comment -- fixed SF# 594: Websocket fails with small masked payloads -- fixed SF# 588: Missing POCO_ARCH and POCO_ARCH_LITTLE_ENDIAN define for WinCE on SH4 -- fixed SF# 581: Out-of-bound array access in Unicode::properties() function. -- fixed SF# 590: Segfault on FreeBSD when stack size not rounded -- fixed SF# 586: Poco::DateTimeParser and ISO8601 issues when seconds fraction has more than 6 digits -- Poco::Net::HTTPSSessionInstantiator::registerInstantiator() now optionally accepts a - Poco::Net::Context object. -- added Poco::XML::XMLWriter::depth() member function. -- added Poco::XML::XMLWriter::uniquePrefix() and Poco::XML::XMLWriter::isNamespaceMapped(). -- Poco::FileChannel now supports a new rotateOnOpen property (true/false) which can be used - to force rotation of the log file when it's opened. -- fixed a bug in Poco::XML::XMLWriter::emptyElement(): need to pop namespace context -- OS X builds now use Clang as default compiler -- Updated SQLite to 3.7.14.1 -- POCO_SERVER_MAIN macro now has a try ... catch block for Poco::Exception and writes - the displayText to stderr. -- Poco/Platform.h now defines POCO_LOCAL_STATIC_INIT_IS_THREADSAFE macro if the compiler - generates thread-safe static local initialization code. - - -Release 1.4.4 (2012-09-03) -========================== - -- ZipStream now builds correctly in unbundled build. -- added proxy digest authentication support to Net library -- integrated MySQL BLOB fixes from Franky Braem. -- use standard OpenSSL import libraries (libeay32.lib, ssleay32.lib) for Crypto and - NetSSL_OpenSSL Visual Studio project files. -- fixed a potential buffer corruption issue in Poco::Net::SecureStreamSocket if lazy - handshake is enabled and the first attempt to complete the handshake fails -- Poco::DateTimeParser::tryParse() without format specifier now correctly parses ISO8601 - date/times with fractional seconds. -- Poco::Process::launch() now has additional overloads allowing to specify an initial - directory and/or environment. -- Poco::Net::FTPClientSession: timeout was not applied to data connection, only to - control connection. -- Fixed potential IPv6 issue with socket constructors if IPv6 SocketAddress is given - (contributed by ??????? ????????? ). -- Added an additional (optional) parameter to Poco::Thread::setOSPriority() allowing to - specify a scheduling policy. Currently this is only used on POSIX platforms and allows - specifying SCHED_OTHER (default), SCHED_FIFO or SCHED_RR, as well as other - platform-specific policy values. -- Added Poco::Crypto::DigestEngine class providing a Poco::DigestEngine interface to - the digest algorithms provided by OpenSSL. -- Fixed some potential compiler warnings in Crypto library -- In some cases, when an SSL exception was unexpectedly closed, a generic Poco::IOException - was thrown. This was fixed to throw a SSLConnectionUnexpectedlyClosedException instead. -- Added Poco::ObjectPool class template. -- Poco::Net::HTTPServer has a new stopAll() method allowing stopping/aborting of all - currently active client connections. -- The HTTP server framework now actively prevents sending a message body in the - response to a HEAD request, or in case of a 204 No Content or 304 Not Modified - response status. -- fixed a DOM parser performance bug (patch by Peter Klotz) -- fixed SF# 3559325: Util Windows broken in non-Unicode -- updated iOS build configuration to use xcode-select for finding toolchain -- Poco::Net::SecureSocketImpl::shutdown() now also shuts down the underlying socket. -- fixed SF# 3552597: Crypto des-ecb error -- fixed SF# 3550553: SecureSocketImpl::connect hangs -- fixed SF# 3543047: Poco::Timer bug for long startInterval/periodic interval -- fixed SF# 3539695: Thread attributes should be destroyed using the pthread_attr_destroy() -- fixed SF# 3532311: Not able to set socket option on ServerSocket before bind - Added Poco::Net::Socket::init(int af) which can be used to explicitely - initialize the underlying socket before calling bind(), connect(), etc. -- fixed SF# 3521347: Typo in UnWindows.h undef -- fixed SF# 3519474: WinRegistryConfiguration bug - Also added tests and fixed another potential issue with an empty root path passed to the constructor. -- fixed SF# 3516827: wrong return value of WinRegistryKey::exists() -- fixed SF# 3515284: RSA publickey format(X.509 SubjectPublicKeyInfo) -- fixed SF# 3503267: VxWorks OS prio is not set in standard constructor -- fixed SF# 3500438: HTTPResponse failure when reason is empty -- fixed SF# 3495656: numberformater, numberparser error in mingw -- fixed SF# 3496493: Reference counting broken in TaskManager postNotification -- fixed SF# 3483174: LogFile flushing behavior on Windows - Flushing is now configurable for FileChannel and SimpleFileChannel - using the "flush" property (true or false). -- fixed SF# 3479561: Subsequent IPs on a NIC is not enumerated -- fixed SF# 3478665: Permission checks in Poco::File not correct for root -- fixed SF# 3475050: Threading bug in initializeNetwork() on Windows -- fixed SF# 3552680: websocket small frames bug and proposed fix -- fixed a WebSocket interop issue with Firefox -- added Poco::Net::MessageHeader::hasToken() -- Poco::AtomicCounter now uses GCC 4.3 builtin atomics on more platforms -- fixed SF# 3555938: NetSSL: socket closed twice -- socket exceptions now include OS error code -- fixed SF# 3556975: Need to fix Shared Memory for memory map -- Poco::Net::SecureSocketImpl::close() now catches exceptions thrown by its call to shutdown(). -- fixed SF# 3535990: POCO_HAVE_IPv6 without POCO_WIN32_UTF8 conflict -- fixed SF# 3559665: Poco::InflatingInputStream may not always inflate completely -- added Poco::DirectoryWatcher class -- fixed SF# 3561464: Poco::File::isDevice() can throw due to sharing violation -- Poco::Zip::Compress::addRecursive() has a second variant that allows to specify the compression method. -- Upgraded internal SQLite to 3.7.14 - - -Release 1.4.3p1 (2012-01-23) -============================ - -- fixed SF# 3476926: RegDeleteKeyEx not available on Windows XP 32-bit - - -Release 1.4.3 (2012-01-16) -========================== - -- fixed a compilation error with Data/MySQL on QNX. -- fixed Util project files for WinCE (removed sources not compileable on CE) -- removed MD2 license text from Ackowledgements document -- fixed iPhone build config for Xcode 4.2 (compiler name changed to llvm-g++) -- Poco::Util::XMLConfiguration: delimiter char (default '.') is now configurable. - This allows for working with XML documents having element names with '.' in them. -- Poco::Util::OptionProcessor: Required option arguments can now be specified as - separate command line arguments, as in "--option value" in addition to the - "--option=value" format. -- Poco::Util::HelpFormatter: improved option help formatting if indentation has - been set explicitely. -- added Mail sample to NetSSL_OpenSSL, showing use of Poco::Net::SecureSMTPClientSession. -- added additional read() overloads to Poco::Net::HTMLForm. -- fixed SF# 3440769: Poco::Net::HTTPResponse doesn't like Amazon EC2 cookies. -- added support for requiring TLSv1 to Poco::Net::Context. -- added an additional constructor to Poco::Net::HTTPBasicCredentials, allowing - the object to be created from a string containing a base64-encoded, colon-separated - username and password. -- Poco::Zip::ZipStreamBuf: fixed a crash if CM_STORE was used. -- Added setContentLength64() and getContentLength64() to Poco::Net::HTTPMessage. -- added Poco::Environment::osDisplayName(). -- fixed SF# 3463096: WinService leaves dangling handles (open() now does not reopen the - service handle if it's already open) -- fixed SF# 3426537: WinRegistryConfiguration can't read virtualized keys -- added Poco::Buffer::resize() -- fixed SF# 3441822: thread safety issue in HTTPClientSession: - always use getaddrinfo() instead of gethostbyname() on all platforms supporting it -- added version resource to POCO DLLs -- fixed SF# 3440599: Dir Path in Quotes in PATH cause PathTest::testFind to fail. -- fixed SF# 3406030: Glob::collect problem -- added Poco::Util::AbstractConfiguration::enableEvents() -- Poco::AtomicCounter now uses GCC builtins with GCC 4.1 or newer - (contributed by Alexey Milovidov) -- made Poco::Logger::formatDump() public as it may be useful for others as well - (SF# 3453446) -- Poco::Net::DialogSocket now has a proper copy constructor (SF# 3414602) -- Poco::Net::MessageHeader and Poco::Net::HTMLForm now limit the maximum number of - fields parsed from a message to prevent certain kinds of denial-of-service - attacks. The field limit can be changed with the new method setFieldLimit(). - The default limit is 100. -- Poco::NumberFormatter, Poco::NumberParser and Poco::format() now always use the - classic ("C") locale to format and parse floating-point numbers. -- added Poco::StreamCopier::copyStream64(), Poco::StreamCopier::copyStreamUnbuffered64() - and Poco::StreamCopier::copyToString64(). These functions use a 64-bit integer - to count the number of bytes copied. -- upgraded internal zlib to 1.2.5 -- upgraded internal sqlite to 3.7.9 -- XML: integrated bugfix for Expat bug# 2958794 (memory leak in poolGrow) -- Added support for HTTP Digest authentication (based on a contribution by - Anton V. Yabchinskiy (arn at bestmx dot ru)). For information on how - to use this, see the Poco::Net::HTTPCredentials, Poco::Net::HTTPDigestCredentials - and Poco::Net::HTTPAuthenticationParams classes. -- Poco::Net::HTTPStreamFactory and Poco::Net::HTTPSStreamFactory now support Basic - and Digest authentication. Username and password must be provided in the URI. -- added Poco::Net::WebSocket, supporting the WebSocket protocol as described in RFC 6455 -- NetSSL_OpenSSL: added client-side support for Server Name Indication. - Poco::Net::SecureSocketImpl::connectSSL() now calls SSL_set_tlsext_host_name() - if its available (OpenSSL 9.8.6f and later). -- added Poco::Net::HTTPClientSession::proxyConnect() (factored out from - Poco::Net::HTTPSClientSession::connect()) -- added Poco::Process::kill(const Poco::ProcessHandle&) which is preferable to - kill(pid) on Windows, as process IDs on Windows may be reused. -- fixed SF# 3471463: Compiler warnings with -Wformat -- Poco::Util::Application::run() now catches and logs exceptions thrown in initialize() -- Fixed a WinCE-specific bug in Poco::Util::ServerApplication where uninitialize() would - be called twice. -- fixed SF# 3471957: WinRegistryKey::deleteKey() unable to delete alt views -- Added additional constructor to Poco::ScopedLock and Poco::ScopedLockWithUnlock - accepting a timeout as second argument. -- Added Poco::Logger::parseLevel() -- Poco::format(): an argument that does not match the format - specifier no longer results in a BadCastException. The string [ERRFMT] is - written to the result string instead. -- PageCompiler: added createSession page attribute. - - -Release 1.4.2p1 (2011-09-24) -============================ - -- On Linux, the RTLD_DEEPBIND option is no longer passed to dlopen(). - This change was introduced in 1.4.2 to solve a specific problem one customer - was having. Unfortunately, it leads to problems with RTTI. -- It's now possible to pass flags (SHLIB_GLOBAL, SHLIB_LOCAL) to - Poco::SharedLibrary::load() (and the constructor implicitly calling load()), - controlling the mode flags (RTLD_GLOBAL, RTLD_LOCAL) passed to dlopen(). - On platforms not using dlopen(), these flags are ignored. -- fixed SF# 3400267: Path_WIN32.cpp bug - - -Release 1.4.2 (2011-08-28) -========================== - -- added Poco::DateTimeFormat::ISO8601_FRAC_FORMAT -- added new Poco::DateTimeFormatter and Poco::DateTimeParser format specifier: - %s for seconds with optional fractions of a second -- fixed a problem with ioctl() on BSD platforms (including OS X) where the - second argument to ioctl() is unsigned long instead of int, causing bad - things on a OS X 64-bit kernel. -- fixed a potential endless loop when enumerating IPv6 network addresses - (reported by Laurent Carcagno) -- new compile-time config option on Windows to set thread names in - debugger. Enable with -DPOCO_WIN32_DEBUGGER_THREAD_NAMES. Available - only in debug builds. -- Cipher can now create Base64 and HexBinary encoded output without linefeeds - (suitable for use in cookies, etc.) -- added Poco::Path::popFrontDirectory() -- improved VxWorks support -- IPv6 fixes: added proper scope id handling in IPAddress, SocketAddress - and related classes. -- Added Poco::Net::ServerSocket::bind6() which allows control over the - IPPROTO_IPV6/IPV6_V6ONLY socket option. -- Removed Poco::MD2Engine class due to licensing issues (the - license for the MD2 code from RSA only allows non-commercial - use). Note that the MD4 and MD5 code from RSA does not have - this issue. -- fixed a Net HTTP client testsuite issue where some tests might - have failed due to prematurely aborted connections by - the HTTPTestServer. -- Poco::Net::SocketAddress: when there is more than one address - returned by a DNS lookup for a name, IPv4 addresses will be - preferred to IPv6 ones. -- NetworkInterface::list() now also returns IPv4 interfaces on Windows when - built with -DPOCO_HAVE_IPv6 -- XMLWriter: fixed a bug with attribute namespaces (no namespace prefix - written if attribute namespace is the same as element namespace) -- fixed SF# 3378588: Mismatched new[]/delete (in RSAEncryptImpl and RSADecryptImpl) -- fixed SF# 3212954 (OpenSSLInitializer::uninitialize() crash) and - SF# 3196862 (Static OpenSSLInitializer instance causes Windows deadlocks) by - removing the static Poco::Crypto::OpenSSLInitializer instance. Automatic OpenSSL - initialization is now done through Poco::Crypto::Cipher, Poco::Crypto::CipherKey, - Poco::Crypto::X509Certificate, Poco::Net::Context classes; however, it is still - recommended to call Poco::Crypto::initializeCrypto() and - Poco::Crypto::uninitializeCrypto() early at application startup, and late at - shutdown respectively (or Poco::Net::initializeSSL()/Poco::Net::uninitializeSSL() - if the NetSSL library is used) to avoid multiple full OpenSSL init/uninit cycles - during application runtime. -- Poco::Logger now also support a symbolic log level "none" - (for use with setLevel()) that disables logging completely - for that Logger (equivalent to setLevel(0)). -- Added experimental Android support, using the existing gmake-based - build system. -- fixed SF# 3288584: DateTimeFormatter link error -- fixed SF# 3187117: Typo in InflatingInputStream doc -- fixed SF# 3309731: _WIN32_WCE comparison should be with 0x600 not 600 -- fixed SF# 3393026: RegularExpression.h identical enum value -- fixed SF# 3274222: AtomicCounter's postfix operators aren't atomic on Windows -- fixed SF# 3317177: Handle leak on windows -- fixed SF# 3181882: Poco::URI::getPathEtc() double-encodes query -- fixed SF# 3379935: ThreadPool Start Bug -- fixed SF# 3354451: Poco::Format::parsePrec never sets the precision to zero -- fixed SF# 3387258: _MAX_PATH used but unknown in Path_WIN32 -- fixed a problem in RSAKeyImpl where direct access to the RSA in a EVP_PKEY - would no longer work in recent OpenSSL versions. Using EVP_PKEY_get1_RSA() - fixes the issue. -- added Poco::Crypto::EncryptingInputStream, Poco::Crypto::EncryptingOutputStream, - Poco::Crypto::DecryptingInputStream and Poco::Crypto::DecryptingOutputStream. -- fixed SF# 3148126: HTTPSClientSession destructor (!) throws an IOException -- fixed SF# 3178098: Add constructor to Poco::TemporaryFile to specify directory -- fixed SF# 3175310: Absolute path when device -- fixed SF# 3301207: Guided tour example contradicts apidoc (API doc was wrong) -- Poco::Net::HTTPMessage::setContentLength() and Poco::Net::HTTPMessage::getContentLength() now - use std::streamsize instead of int. This enables 64-bit Content-Length support at least - on 64-bit platforms. -- fixed SF# 3177530: TemporaryFile::tempName() + glob bug on xp -- fixed SF# 3177372: FileChannel documentation inconsistency -- added %E format specifier to Poco::PattermFormatter (epoch time in seconds - since midnight, January 1 1970) -- On Windows, Poco::Util::ServerApplication now supports a /description command - line argument for specifying a service description (together with /registerService) -- added Poco::Util::WinService::setDescription() and - Poco::Util::WinService::getDescription() -- fixed SF# 3155477: Incorrect URI path handling -- fixed SF# 3309736: Extended Exception macros to set default exception code - new macro is named POCO_DECLARE_EXCEPTION_CODE -- added getter functions for modulus and exponents to Poco::Crypto::RSAKey. -- added Poco::Net::SocketAddress::operator == () and - Poco::Net::SocketAddress::operator != () -- fixed SF# 3182746: IPAddress.cpp IPv6 bug on big-endian -- fixed SF# 3196961: Unix daemon fails to loadConfiguration() if started from cwd -- fixed SF# 3393700: NotificationCenter may call a removed observer and crash. -- Reworked implementation of the events framework (Poco::BasicEvent and friends). - The framework is now completely multithreading save (even in the case that - an event subscriber object unsubscribes and is deleted while an event is - being dispatched). Also, the restriction that any object can only register - one delegate for each event has been removed. For most cases, dispatching - events should be faster, as dispatching an event now needs less dynamic memory - allocations. -- fixed SF# 3178109: getNodeByPath() changes: - getNodeByPath() and getNodeByPathNS() have been moved to Poco::XML::Node. - Furthermore, when invoked on a Poco::XML::Document, the behavior has changed - so that the document element is now included when traversing the path (previously, - traversal would start at the document element, now it starts at the document). - The path expression can now start with a double-slash, which results in a recursive - search for the path's first element in the DOM tree. -- fixed SF# 3382935: String data being truncated using ODBC, and - SF# 2921813: Wrong implementation of the ODBC string binding - - -Release 1.4.1p1 (2011-02-08) -============================ - -- Poco::Mutex is now a recursive mutex again on Linux - (this was caused by an unfortunate feature test for - PTHREAD_MUTEX_RECURSIVE which did not work on Linux - as PTHREAD_MUTEX_RECURSIVE is an enum value and not - a macro) -- Poco::Net::SecureSocketImpl::abort() now only shuts - down the underlying socket connection and does not free - the SSL object, due to multithreading issues. - - -Release 1.4.1 (2011-01-29) -========================== - -- fixed SF# 3150223: Poco::BinaryReader cannot read std::vector correctly -- fixed SF# 3146326: SharedMemory issue -- made Poco::Net::HTTPSession::abort() virtual -- added Poco::Net::SecureStreamSocket::abort() to immediately close - a SSL/TLS connection without performing an orderly SSL/TLS shutdown. -- fixed SF# 3148126: HTTPSClientSession destructor (!) throws an IOException. - Added try/catch block to Poco::Net::SecureSocketImpl destructor. -- added additional constructor to Poco::Net::HTTPSClientSession, taking - both a socket and a session object. -- Poco::Net::HTTPSession::abort() now also can be used with a - Poco::Net::HTTPSClientSession. -- fixed SF# 3148045: make clean and distclean issues -- changed Data library names on Unix/Linux platforms to - match the names on Windows (PocoSQLite -> PocoDataSQLite, - PocoMySQL -> PocoDataMySQL, PocoODBC -> PocoDataODBC) -- added additional options to configure script -- added additional documentation to Poco::Net::HTTPClientSession -- Poco::Net::HTTPClientSession::receiveResponse() closes the connection - if an exception is thrown while reading the response header. - This ensures that a new connection will be set up for the next request - if persistent connections are used. -- improved Poco::Net::MultipartDecoder performance by reading directly from streambuf -- improved performance of Poco::Base64Encoder, Poco::Base64Decoder, - Poco::HexBinaryEncoder and Poco::HexBinaryDecoder by working directly with the - given stream's streambuf. -- improved performance of MessageHeader::read() by reading directly from streambuf - instead of istream. -- it is now possible to specify additional MIME part header fields - for a MIME part through the Poco::Net::PartSource class. -- upgraded SQLite to release 3.7.4 -- added experimental VxWorks support for VxWorks 5.5.1/Tornado 2.2 and - newer. Please see the VxWorks Platform Notes in the reference documentation - for more information. Currently, the VxWorks is untested; full support - will be available in release 1.4.2. -- fixed SF# 3165918: Poco::DynamicAny fails to convert from string to float -- fixed SF# 3165910: Poco::Net::MessageHeader does not accept HTTP conforming header -- made Poco::Task::cancel() virtual so that tasks can implement custom - cancellation behavior. -- added optional argument to Poco::Util::WinRegistryKey constructor - to specify additional flags (in addition to KEY_READ and KEY_WRITE) - for the samDesired argument of RegOpenKeyEx() or RegCreateKeyEx(). -- improved Poco::BasicEvent::notify() performance by avoiding an unnecessary heap - allocation. -- added additional well-known port numbers to Poco::URI: rtsp, sip, sips, xmpp. -- added Poco::Net::MediaType::matchesRange() -- improved invalid socket handling: a Poco::Net::InvalidSocketException is - now thrown instead of an assertion when an operation is attempted on a closed or - otherwise uninitialized socket. - - -Release 1.4.0 (2010-12-14) -========================== - -- SSLManager: documentation fixes, code cleanup -- SSLManager: renamed PrivateKeyPassPhrase event to PrivateKeyPassphraseRequired -- added HTTPServerRequestImpl::socket() to get access to the underlying socket -- added Socket::secure() to find out whether a given socket supports SSL/TLS -- added SecureStreamSocket::havePeerCertificate() -- NetSSL: added support for turning off extended certificate validation (hostname matching) -- fixed SF# 2941228: ICMPClient::ping() issues on Mac OS X -- fixed SF# 2941231: ICMPEventArgs out of bounds array access -- added PageCompiler sample -- added missing newline at end of xmlparse.c -- Poco::Glob can now be used with an empty pattern which will match nothing (patch from Kim Graesman) -- added support for HTTP proxy authentication (Basic authentication only) -- fixed SF# 2958959: XMLWriter must encode CR, LF and TAB in attribute values as character entities. -- HTMLForm now supports PUT requests as well (see ) -- fixed SF# #2970521: FileOutputStream and file permissions. - (also fixed in File class) -- removed an unused (and wrong) default parameter from EventImpl constructor for WIN32. -- added full support for session caching to NetSSL_OpenSSL -- fixed SF# 2984454: Poco::Util::Timer::scheduleAtFixedRate() works incorrectly -- fixed a bug in Poco::Util::Timer that could lead to high CPU load if - the system clock is moved forward. -- added system.nodeId to SystemConfiguration -- added a note to Poco::Util::ServerApplication documentation regarding - creation of threads -- added Poco::Net::IPAddress::broadcast() and Poco::Net::IPAddress::wildcard() to - create broadcast (255.255.255.255) and wildcard (0.0.0.0) addresses. -- fixed SF# 2916154: Poco::Net::IPAddress::isLoopback() only works for 127.0.0.1. -- added build configuration for iPhone Simulator -- GNU Make based build system provides new variables: POCO_HOST_BINDIR, POCO_HOST_BINPATH, - POCO_HOST_LIBDIR, POCO_HOST_LIBPATH and POCO_TARGET_* equivalents. -- Application::initialize() and Application::uninitialize() will now be called from within run(). - This solves various issues with uninitialize() not being called, or being called inappropriately - from the Application destructor. - Please note that this change will break applications that use the Application class, - but only call init() and not run(). -- added /startup option to specify startup mode for Windows services (automatic or manual) -- fixed SF# 2967354: SecureSocketImpl shutdown/close problem -- fixed SF# 3006340: LinearHashTable grows even if key already exists -- fixed a particularly nasty Windows error handling issue that manifested itself on WinCE: - WSAGetLastError() would be called after a std::string was created. The string creation could result - in a heap operation which called a Windows API to allocate memory. This would reset the - GetLastError() error code. Since WSAGetLastError() is just an alias for GetLastError(), the actual - error code from the socket operation would be lost. -- upgraded SQLite to 3.7.3 -- added --header-prefix option to PageCompiler -- fixed SF# 3003875: SQLite data binding is broken -- fixed SF# 2993988: Issue with multiple calls to open()/close() on File*Stream -- fixed SF# 2990256: HTMLForm and file uploads -- fixed SF# 2969227: DateTimeParser bug -- fixed SF# 2966698: Socket connect with timeout issue -- fixed SF# 2981041: Bind NULL to a query (patch supplied) -- fixed SF# 2961419: UTF8Encoding::convert() doesn't work properly in DEBUG mode -- fixed SF# 2957068: Timeout value not picked up by proxy in HTTPSClientSession -- fixed NetSSL_OpenSSL test runner for Poco::Util::Application class changes -- Poco::AbstractEvent, Poco::AbstractCache and related classes now accept a Mutex class as additional template argument. - Poco::NullMutex can be used if no synchronization is desired. -- Added Poco::AbstractEvent::empty() to check whether an event has registered delegates. -- Poco::URI now correctly handles IPv6 addresses. -- Added Poco::Nullable class template. -- Added Poco::NullMutex, a no-op mutex to be used as template argument for template classes - taking a mutex policy argument. -- Poco::XML::XMLWriter: fixed a namespace handling issue that occured with startPrefixMapping() and endPrefixMapping() -- Poco::Net::Context now allows for loading certificates and private keys from Poco::Crypto::X509Certificate objects - and Poco::Crypto::RSAKey objects. -- Poco::Crypto::RSAKey no longer uses temporary files for stream operations. Memory buffers are used instead. -- fixed SF# 2957865: added Poco::UUID::tryParse() -- All Zip classes now use Poco::File[Input|Output]Stream instead of std::[i|o]fstream. - UTF-8 filenames will now be handled correctly on Windows. -- fixed SF# 2902029: zlib flush support (Z_SYNC_FLUSH) -- added Poco::TextBufferIterator class -- fixed SF# 2977249: Use epoll instead select under Linux - Socket::select() and Socket::poll() will use epoll under Linux if the Net library is compiled - with -DPOCO_HAVE_FD_EPOLL. This is the default for the Linux build configuration (but not for - the various build configurations targeting embedded Linux platforms). -- fixed SF# 2941664: Memory leak in DeflatingStream with zero-length streams (also fixed some other potential, - but unlikely, memory leaks) -- fixed SF# 2946457: added RejectCertificateHandler -- fixed SF# 2946621: Poco::Path bug with POCO_WIN32_UTF8 -- fixed SF# 2929805: Environment::nodeId() does not work if no eth0 device exists -- Environment::nodeId() no longer throws if no hardware ethernet address can be determined. - It returns an all-zero address instead. -- Added additional classification functions to Poco::Unicode class; made classification functions inline. -- added Ascii class for ASCII character classification. - Methods of the Ascii class are now used instead of the - standard library functions (std::isspace(), etc.) due to - possible inconsistent results or assertions when the - standard library functions are used with character codes - outside the ASCII range. -- Poco::Net::MailMessage: fixed a bug in StringPartHandler that resulted in incorrect handling of non-ASCII data if - char is signed. -- Improved Poco::Net::SMTPClientSession compatibility with various mail servers when using AUTH_LOGIN authentication. -- Added CRAM-SHA1 support to Poco::Net::SMTPClientSession -- Poco::Net::SMTPClientSession now also supports login with AUTH PLAIN. -- Added Poco::Net::SecureSMTPClientSession class, supporting STARTTLS for secure SMTP connections. -- fixed an issue with SharedMemory on POSIX systems, where a shared memory region would be deleted - despite the server flag set to true (see http://pocoproject.org/forum/viewtopic.php?f=12&t=3494). -- PageCompiler: added a new page context directive, to allow passing custom context objects to the - request handler. -- fixed StreamSocketImpl::sendBytes() for non-blocking sockets -- added Poco::Net::DialogSocket::receiveRawBytes(), which should be used instead of receiveBytes() due to internal - buffering by DialogSocket. -- DOMParser: FEATURE_WHITESPACE has been renamed to FEATURE_FILTER_WHITESPACE (which now matches the underlying URI) - and is now handled correctly (previously we did the exact reverse thing) -- added Poco::Util::AbstractConfiguration::remove() to remove a configuration property; added removeRaw() implementations - to all implementations (contributions by Daniel Hobi and Alexey Shults). -- fixed NetSSL_OpenSSL compilation error on Windows with OpenSSL 1.0 -- Added optional FIPS mode support to NetSSL_OpenSSL (contributed by Lior Okman). - If OpenSSL has been configured and built with FIPS support, then FIPS support can - be enabled by calling Poco::Crypto::OpenSSLInitializer::enableFIPSMode(true); or - by setting the fips property in the OpenSSL configuration to true (see Poco::Net::SSLManager - for details). -- fixed SF# 3031530: Ping and possible no timeout -- added Poco::Net::SocketReactor::onBusy(), called whenever at least one notification will - be dispatched. -- fixed SF# 3034863: Compiler warning in net/IPAddress.h with poco 1.3.2 -- added support for CRAM-SHA1 authentication to SMTPClientSession -- Poco::format(): arguments can now be addressed by their index, e.g. %[2]d -- Poco::Util::Timer::cancel() now accepts an optional boolean argument. - If true is passed, cancel() waits until the task queue has been purged. - Otherwise, it returns immediately and works asynchronously, as before. -- Poco::Net::HTTPServerResponse::redirect() now accepts an optional additional - argument to specify the HTTP status code for the redirection. -- fixed a warning (BinaryReader.cpp) and error (ThreadLocal.cpp) in Foundation when compiling with Visual Studio 2010 -- fixed a wrong exception in Poco::Net::POP3ClientSession -- Poco::Net::FTPClientSession and Poco::Net::SMTPClientSession now set the error code in exceptions they throw -- fixed a potential race condition with terminating a Windows service based on Poco::Util::ServerApplication -- fixed a bug in global build configuration file: explicitly setting POCO_CONFIG did not work on Solaris platforms, - as it was always overridden by the automatically determined configuration. -- Added support for MinGW cross builds on Linux. -- Changed location of statically linked build products in the gmake-based build system. - Statically linked executables are now in bin/$(OSNAME)/$(OSARCH)/static and no longer - have the _s suffix -- The POCO_VERSION macro now is in its own header file, "Poco/Version.h". It is no longer - available through "Poco/Foundation.h". -- added Poco::Net::HTTPCookie::escape() and Poco::Net::HTTPCookie::unescape(). -- fixed SF# 3021173: Thread (POSIX) returns uninitialised value for OS priority -- fixed SF# 3040870: ThreadPool has no function to get assigned name -- fixed SF# 3044303: Can't use own config file on Solaris & OSARCH_64BITS ignored -- fixed SF# 2943896: AsyncChannel::log blocks -- fixed a bug in Poco::Util::WinRegistryKey::getInt(): - The size variable passed to RegQueryValueExW() should be initialized to the size - of the output buffer. -- Added rudimentary support for compiling with Clang 2.0 (Xcode 4) on Mac OS X. -- New build configurations for Mac OS X: Darwin32 and Darwin64 for explicit - 32-bit and 64-bit builds. Note that the default Darwin build configuration - will build 64-bit on Snow Leopard and 32-bit on Leopard, but will always place - build products in Darwin/i386. The new Darwin32 and Darwin64 configurations - will use the correct directories. -- fixed SF# 3051598: Bug in URL encoding -- Poco::ThreadPool::stopAll() (and thus also the destructor) will now wait for each - pooled thread to terminate before returning. This fixes an issue with creating - and orderly shutting down a thread pool in a plugin. Previously, a pooled thread - in a thread pool created by a dynamically loaded library might still be running - when the plugin's shared library was unloaded, resulting in Bad Things happening. - This can now no longer happen. As a downside, a pooled thread that fails to - finish will block stopAll() and the destructor forever. -- NetSSL_OpenSSL: for a SecureStreamSocket, available() now returns the number of bytes that - are pending in the SSL buffer (SSL_pending()), not the actual socket buffer. -- Added Poco::Net::HTTPClientSession::secure() to check for a secure connection. -- Poco::Net::HTTPRequest::setHost() now does not include the port number in the Host header - if it's either 80 or 443. -- log messages can now optionally include source file path and line number -- Poco::PatternFormatter can format source file path and line number (%U, %u) -- logging macros (poco_information(), etc.) now use __LINE__ and __FILE__ -- new logging macros that incorporate Poco::format(): poco_information_f1(logger, format, arg) with up to 4 arguments -- added Poco::Net::HTTPSession::attachSessionData() and Poco::Net::HTTPSession::sessionData() - to attach arbitrary data to a HTTP session. -- added additional constructors to zlib stream classes that allow passing - a windowBits parameter to the underlying zlib library. -- fixed a potential error handling issue in Poco::Net::SecureSocketImpl. -- fixed SF# 3110272: RSACipherImpl bug. -- fixed SF# 3081677: ConfigurationView's getRaw not retrieving xml attributes. -- added basic support for Canonical XML and better pretty-printing support to Poco::XML::XMLWriter. -- Poco::Util::AbstractConfiguration now supports events fired when changing or - removing properties. -- XML: added support for finding DOM nodes by XPath-like - expressions. Only a very minimal subset of XPath is supported. - See Poco::XML::Element::getNodeByPath(), Poco::XML::Element::getNodeByPathNS() - and the same methods in Poco::XML::Document. -- Poco::Timer: If the callback takes longer to execute than the - timer interval, the callback function will not be called until the next - proper interval. The number of skipped invocations since the last - invocation will be recorded and can be obtained by the callback - by calling skipped(). -- Poco::BinaryReader and Poco::BinaryWriter now support reading and - writing std::vectors of the supported basic types. Also, strings - can now be written in a different encoding (a Poco::TextEncoding - can be optionally passed to the constructor). -- Poco::UUID::nil() and Poco::UUID::isNil() have been renamed to - Poco::UUID::null() and Poco::UUID::isNull(), respectively, to avoid - issues with Objective-C++ projects on Mac OS X and iOS where nil is - a system-provided macro. -- Crypto bugfixes: RSACipherImpl now pads every block of data, not just the - last (or last two). -- Improved Crypto testsuite by adding new tests -- Added new Visual Studio project configurations: debug_static_mt and release_static_mt - (linking with static runtime libraries). The existing configurations debug_static - and release_static have been renamed to debug_static_md and release_static_md, respectively. - The suffixes of the static libraries have also changed. The static_md configurations - now build libraries with suffixes md[d], while the libraries built by the static_mt - configurations have mt[d] suffixes. -- Added Visual Studio project files for 64-bit builds. -- Added Visual Studio 2010 project files. -- Removed the use of local static objects in various methods due to - their construction not being threadsafe (and thus leading to - potential race conditions) on Windows/Visual C++. -- Fixed some warning on 64-bit Windows builds. -- The name of the Data connector libraries have changed. They are now - named PocoDataMySQL, PocoDataODBC and PocoDataSQLite. -- fixed SF# 3125498: Linux NetworkInterface::list() doesn't return IPv6 IPs -- fixed SF# 3125457: IPv6 IPAddress tests are wrong -- Added initialization functions for the NetSSL_OpenSSL and Crypto libraries. - These should be called instead of relying on automatic initialization, - implemented with static initializer objects, as this won't work with - statically linked executables (where the linker won't include the - static initializer object). - The functions are Poco::Crypto::initializeCrypto(), Poco::Crypto::uninitializeCrypto(), - Poco::Net::initializeSSL() and Poco::Net::uninitializeSSL(). - Applications using Crypto and/or NetSSL should call these methods appropriately at - program startup and shutdown. - Note: In release 1.3.6, similar functions have been added to the Net library. - - -Release 1.3.6p2 (2010-01-15) -============================ - -- fixed an issue in the Windows implementation Poco::RWLock, where - tryReadLock() sometimes would return false even if no writers - were using the lock (fix contributed by Bjrn Carlsson) -- added Poco::Environment::libraryVersion(). -- fixed SF# 2919461: Context ignores parameter cypherList -- removed an unused enum from RSACipherImpl.cpp (Crypto) -- integrated a new expat patch for CVE-2009-3560. -- fixed SF# 2926458: SSL Context Problem. The Poco::Net::Context - class now makes sure that OpenSSL is properly initialized. -- updated iPhone build configuration (contributed by Martin York) -- fixed SF# 1815124 (reopened): XML Compile failed on VS7.1 with - XML_UNICODE_WCHAR_T -- fixed SF# 2932647: FTPClientSession::getWorkingDirectory() returns a bad result - - -Release 1.3.6p1 (2009-12-21) -============================ - -- added support for using external zlib, pcre, expat and sqlite3 instead of - bundled ones (-DPOCO_UNBUNDLED, configure --unbundled) -- fixed SF# 2911407: Add sh4 support -- fixed SF# 2912746: RSAKey::EXP_LARGE doesn't work -- fixed SF# 2904119: abstractstrategy uses std::set but do not includes it -- fixed SF# 2909946: localtime NULL pointer -- fixed SF# 2914986: potential expat DoS security issues (CVE-2009-3560 and CVE-2009-3720) -- fixed SF# 2916305: SSL Manager crashes -- fixed SF# 2903676: Tuple TypeHander does not handle composites. - - -Release 1.3.6 (2009-11-24) -========================== - -- added Environment::processorCount() -- added POCO_VERSION macro to Poco/Foundation.h -- fixed SF# 2807527: Poco::Timer bug for long startInterval/periodic interval -- fixed a bug similar to SF# 2807527 in Poco::Util::Timer. -- fixed SF# 2795395: Constructor doesn't treat the params "key" and "iv" -- fixed SF# 2804457: DateTime::checkLimit looks wrong -- fixed SF# 2804546: DateTimeParser requires explicit RFC1123 format -- added ReleaseArrayPolicy to Poco::SharedPtr -- upgraded to SQLite 3.6.20 -- fixed SF# 2782709: Missing semicolons in "Logger.h" convenience -- fixed SF# 2526407: DefaultStrategy.h ++it instead of it++ in a loop -- fixed SF# 2502235: Poco STLPort patch -- fixed SF# 2186643: Data::Statement::reset() not implemented in 1.3.3 -- fixed SF# 2164227: Allow File opened read only by FileInputSteam to be writable -- fixed SF# 2791934: use of char_traits::copy in BufferedStreamBuf::underflow -- fixed SF# 2807750: Support additional SQL types in SQLite -- fixed documentation bugs in Timed/PriorityNotificationQueue -- fixed SF# 2828401: Deadlock in SocketReactor/NotificationCenter (also fixes patch# 1956490) - NotificationCenter now uses a std::vector internally instead of a std::list, and the mutex is - no longer held while notifications are sent to observers. -- fixed SF# 2835206: File_WIN32 not checking aganist INVALID_HANDLE_VALUE -- fixed SF# 2841812: Posix ThreadImpl::sleepImpl throws exceptions on EINTR -- fixed SF# 2839579: simple DoS for SSL TCPServer, HTTPS server - No SSL handshake is performed during accept() - the handshake is delayed until - sendBytes(), receiveBytes() or completeHandshake() is called for the first time. - This also allows for better handshake and certificate validation when using - nonblocking connections. -- fixed SF# 2836049: Possible handle leak in FileStream - If sync() fails, close() now simply set's the stream's bad bit. - In any case, close() closes the file handle/descriptor. -- fixed SF# 2814451: NetSSL: receiveBytes crashes if socket is closed -- added a workaround for Vista service network initialization issue - (an Windows service using the Net library running under Vista will - crash in the call to WSAStartup() done in NetworkInitializer). - Workaround is to call WSAStartup() in the application's main(). - Automatic call to WSAStartup() in the Net library can now be disabled - by compiling Net with -DPOCO_NET_NO_AUTOMATIC_WSASTARTUP. Also - the new Poco::Net::initializeNetwork() and Poco::Net::uninitializeNetwork() - functions can be used to call WSAStartup() and WSACleanup(), respectively, - in a platform-independent way (on platforms other than Windows, these - functions will simply do nothing). -- added VCexpress build script support (contributed by Jolyon Wright) -- fixed SF# 2851052: Poco::DirectoryIterator copy constructor is broken -- fixed SF# 2851197: IPAddress ctor throw keyword missing -- added Poco::ProtocolException -- PageCompiler improvements: new tags, support for buffered output, etc. -- better error reporting in Data MySQL connector (patch #2881270 by Jan "HanzZ" Kaluza) -- fixed SF# 1892462: FTPClient:Choose explicitely between EPSV and PASV -- fixed SF# 2806365: Option for PageCompiler to write output to different dir -- fixed a documentation bug (wrong sample code) in Process::launch() documentation -- added --header-output-dir option to PageCompiler -- fixed SF# 2849144: Zip::Decompress notifications error -- SAXParser has a new feature: "http://www.appinf.com/features/enable-partial-reads". - See ParserEngine::setEnablePartialReads() for a description of what this does. -- fixed SF# 2876179: MySQL Signed/Unsigned value bug -- fixed SF# 2877970: possible bug in timer task -- fixed SF# 2874104: wrong parsing empty http headers -- fixed SF# 2860694: Incorrect return code from SecureStreamSocketImpl::sendBytes -- fixed SF# 2849750: Possible bug with XMLWriter? -- added MailMessage::encodeWord() to support RFC 2047 word encoded - mail header fields when sending out mail containing non-ASCII - characters. -- fixed SF# 2890975: SMTPClientSession bug with 7BIT encoding -- fixed an issue with retrieving the value of socket options on Windows 7. - Before obtaining the value of a socket, we now initialize the variable receiving the - socket option value to zero. -- fixed SF# 2836141: Documentation errors -- fixed SF# 2864232: Socket::select() does not detect closed sockets on windows -- fixed SF# 2812143: Socket::select() should check socket descriptors... -- fixed SF# 2801750: NetworkInterface forName returns wrong subnetMask -- fixed SF# 2816315: Problem with POSIX Thread::sleepImpl -- fixed SF# 2795646: IPv6 address parsing bug -- fixed #0000092: ServerApplication::waitForTerminationRequest(), SIGINT and GDB. - Poco::Util::ServerApplication::waitForTerminationRequest() no longer registers a - signal handler for SIGINT if the environment variable POCO_ENABLE_DEBUGGER - is defined. -- fixed SF# 2896070: Poco::Net::Context with non-ASCII paths -- added Unicode Surrogate support to Poco::UTF16Encoding. - See Poco::TextEncoding::queryConvert() and Poco::TextEncoding::sequenceLength() - for how this is implemented. Contributed by Philippe Cuvillier. -- fixed SF# 2897650: [branch 1.3.6] Net.SocketAddress won't compile for CYGWIN -- fixed SF# 2896161: Building on Windows fails when basedir has space in it -- fixed SF# 2864380: Memory leak when using secure sockets -- NetSSL_OpenSSL: the SSL/TLS session cache is now disabled by default and - can be enabled per Context using Poco::Net::Context::enableSessionCache(). -- fixed SF# 2899039: Wrong DST handling in LocalDateTime -- added RWLock::ScopedReadLock and RWLock::ScopedWriteLock (contributed by Marc Chevrier) -- added Thread::TID type, as well as Thread::tid() and Thread::currentTid() to obtain the native - thread handle/ID -- added Zip file comment support -- On Windows, Poco::SharedLibrary::load() now uses LoadLibraryEx instead of LoadLibrary - and uses the LOAD_WITH_ALTERED_SEARCH_PATH if an absolute path is specified. This will - add the directory containing the library to the search path for DLLs that the - loaded library depends upon. -- Mac OS X build settings now match those used by default Xcode projects, making linking the - POCO libs to Xcode projects easier -- Replaced use of std::valarray in Poco::Net::ICMPEventArgs with std::vector due to issues with - std::valarray together with STDCXX debug mode on OS X - - -Release 1.3.5 (2009-05-11) -========================== - -- fixed SF# 2779410: Poco::Data::ODBC::HandleException impovement -- fixed wrong exception text for Poco::UnhandledException -- Fixed a problem with SSL shutdown that causes clients (web browsers) - to hang when the server attempts to perform a clean SSL shutdown. We now call - SSL_shutdown() once, even if the shutdown is not complete after the first call. -- added Poco::Crypto::X509Certificate::save() -- fixed a bug in Poco::Zip::Decompress that results in wrong paths for extracted files -- fixed a bug in Poco::Zip::ZipManipulator where the Zip file was opened in text format - on Windows. -- added Poco::Crypto::X509Certificate::issuedBy() to verify certificate chain. -- fixed 0000089: Thread::sleep() on Linux is extremely inaccurate -- added methods to extract the contents of specific fields from the - subject and issuer distinguished names of a certificate. - - -Release 1.3.4 (2009-04-21) -========================== - -- fixed SF# 2611804: PropertyFileConfiguration continuation lines -- fixed SF# 2529788: ServerApplication::beDaemon() broken -- fixed SF# 2445467: Bug in Thread_WIN32.cpp -- Improved performance of HTTP Server by removing some - string copy operations -- fixed SF# 2310735: HTTPServer: Keep-Alive only works with send() -- fixed appinf.com IP address in Net testsuite -- fixed RFC-00188: NumberFormatter and float/double numbers -- added --pidfile option to ServerApplication on Unix -- fixed SF# 2499504: Bug in Win32_Thread when using from dll (fixed also for POSIX threads) -- fixed SF# 2465794: HTTPServerRequestImpl memory leak -- fixed SF# 2583934: Zip: No Unix permissions set -- the NetSSL_OpenSSL library has been heavily refactored -- added NumberFormatter::append*() and DateTimeFormatter::append() functions -- use NumberFormatter::append() and DateTimeFormatter::append() instead of format() where - it makes sense to gain some performance -- added system.dateTime and system.pid to Poco::Util::SystemConfiguration -- added %F format specifier (fractional seconds/microseconds) to DateTimeFormatter, - DateTimeParser and PatternFormatter. -- fixed SF# 2630476: Thread_POSIX::setStackSize() failure with g++ 4.3 -- fixed SF# 2679279: Handling of -- option broken -- added compile options to reduce memory footprint of statically linked applications - by excluding various classes from automatically being linked. - See the POCO_NO_* macros in Poco/Config.h. -- fixed SF# 2644940: on Windows the COMPUTER-NAME and the HOSTNAME can be different -- added DNS::hostName() function -- added build configuration for iPhone (using Apple's SDK) -- basic support for AIX 5.x/xlC 8 -- fixed a bug resulting in a badly formatted exception message with IOException - thrown due to a socket-related error -- fixed SF# 2644718: NetworkInterface name conflict in MinGW -- added a missing #include to CryptoTransform.h -- fixed SF# 2635377: HTTPServer::HTTPServer should take AutoPtr -- replaced plain pointers with smart pointers in some interfaces -- upgraded to sqlite 3.6.13 -- improved Data::SQLite error reporting -- Poco::Glob now works with UTF-8 encoded strings and supports case-insensitive comparison. - This also fixes SF# 1944831: Glob::glob on windows should be case insensitve -- added Twitter client sample to Net library -- Fixed SF# 2513643: Seg fault in Poco::UTF8::toLower on 64-bit Linux -- Poco::Data::SessionPool: the janitor can be disabled by specifying a zero idle time. -- added Poco::Data::SessionPool::customizeSession() -- added support for different SQLite transaction modes (DEFERRED, IMMEDIATE, EXCLUSIVE) -- fixed a few wrong #if POCO_HAVE_IPv6 in the Net library -- added support for creating an initialized, but unconnected StreamSocket. -- added File::isDevice() -- added family() member function to SocketAddress, -- Data::SQLite: added support for automatic retries if the database is locked -- XMLConfiguration is now writable -- fixed an IPv6 implementation for Windows bug in HostEntry -- Timer class improvement: interval between callback is no longer influenced by the - time needed to execute the callback. -- added PriorityNotificationQueue and TimedNotificationQueue classes to Foundation. - These are variants of the NotificationQueue class that support priority and - timestamp-tagged notifications. -- added Poco::Util::Timer class. This implements a timer that can schedule different - tasks at different times, using only one thread. -- the signatures of Poco::NotificationQueue and Poco::NotificationCenter member functions - have been changed to accept a Poco::Notification::Ptr instead of Poco::Notification* - to improve exception safety. This change should be transparent and fully backwards - compatible. The signature of the methods returning a Poco::Notification* have not been - changed for backwards compatibility. It is recommended, that any Notification* obtained - should be immediately assigned to a Notification::Ptr. -- SQLite::SessionImpl::isTransaction() now uses sqlite3_get_autocommit() to find out - about the transaction state. -- refactored Crypto library to make it independent from NetSSL_OpenSSL. -- added support for RSA-MD5 digital signatures to Crypto library. -- removed SSLInitializer from NetSSL library (now moved to Crypto library) -- added build configs for static libraries to Crypto library -- OpenSSL now depends on Crypto library (which makes more sense than - vice versa, as it was before). Poco::Net::X509Certificate is now - a subclass of Poco::Crypto::X509Certificate (adding the verify() - member function) and the Poco::Net::SSLInitializer class was - moved to Poco::Crypto::OpenSSLInitializer. -- added build configs for static libraries to Zip -- added batch mode to CppUnit::WinTestRunner. - WinTestRunnerApp supports a batch mode, which runs the - test using the standard text-based TestRunner from CppUnit. - To enable batch mode, start the application with the "/b" - or "/B" command line argument. Optionally, a path to a file - where the test output will be written to may be given: - "/b:" or "/B:". - When run in batch mode, the exit code of the application - will denote test success (0) or failure (1). -- testsuites now also work for static builds on Windows -- The IPv6 support for Windows now basically works (Net library compiled with POCO_HAVE_IPv6) -- fixed a potential error when shutting down openssl in a statically linked application -- added static build configs to Data library -- added Poco::AtomicCounter class, which uses OS-specific APIs for atomic (thread-safe) - manipulation of counter values. -- Poco::RefCountedObject and Poco::SharedPtr now use Poco::AtomicCounter for - reference counting -- fixed SF# 2765569: LoadConfiguration failing from current directory - - -Release 1.3.3p1 (2008-10-09) -============================ - -- Fixed SF# 2153031: 1.3.3 Crypto won't compile on 64-bit Linux -- Fixed a warning in MySQL connector -- Updated README -- The global Makefile in the Zip archive is no longer broken - - -Release 1.3.3 (2008-10-07) -========================== - -- Threads now have optional user-settable stack size (if the OS supports that feature) -- Events now support simplified delegate syntax based on delegate function template. - See Poco::AbstractEvent documentation for new syntax. -- Cache supports new access expire strategy. -- Upgraded to SQLite 3.6.2 -- Upgraded to PCRE 7.8 -- added HttpOnly support to Poco::Net::HTTPCookie -- NetworkInterface now has displayName() member (useful only on Windows) -- Poco::Util::WinRegistryKey now has a read-only mode -- Poco::Util::WinRegistryKey::deleteKey() can now recursively delete registry keys -- Poco::File::created() now returns 0 if the creation date/time is not known, as - it's the case on most Unix platforms (including Linux). - On FreeBSD and Mac OS X, it returns the real creation time. -- Time interval based log file rotation (Poco::FileChannel) now works - correctly. Since there's no reliable and portable way to find out the creation - date of a file (Windows has the tunneling "feature", most Unixes don't provide - the creation date), the creation/rotation date of the log file is written into - the log file as the first line. -- added Environment::nodeId() for obtaining the Ethernet address of the system - (this is now also used by UUIDGenerator - the corresponding code from UUIDGenerator - was moved into Environment) -- added a release policy argument to SharedPtr template -- Socket::select() will no longer throw an InvalidArgumentException - on Windows when called with no sockets at all. If all three socket - sets are empty, Socket::select() will return 0 immediately. -- SocketReactor::run() now catches exceptions and reports them via - the ErrorHandler. -- SocketReactor has a new IdleNotification, which will be posted when - the SocketReactor has no sockets to handle. -- added referenceCount() method to Poco::SharedPtr. -- POCO now builds with GCC 4.3 (but there are some stupid warnings: - "suggest parentheses around && within ||". -- Solution and project files for Visual Studio 2008 are included -- The Zip library is now officially part of the standard POCO release. -- The Crypto library (based on OpenSSL) has been added. The original code - was kindly contributed by Ferdinand Beyer. -- A Data Connector to MySQL, contributed by Sergey Kholodilov, is now part - of the POCO release. -- fixed SF# 1859738: AsyncChannel stall -- fixed SF# 1815124: XML Compile failed on VS7.1 with XML_UNICODE_WCHAR_T -- fixed SF# 1867340: Net and NetSSL additional dependency not set - ws2_32.lib -- fixed SF# 1871946: no exception thrown on error -- fixed SF# 1881113: LinearHashTable does not conform to stl iterators -- fixed SF# 1899808: HTMLForm.load() should call clear() first -- fixed SF# 2030074: Cookie problem with .NET server -- fixed SF# 2009707: small bug in Net/ICMPPacketImpl.cpp -- fixed SF# 1988579: Intel Warning: invalid multibyte character sequence -- fixed SF# 2007486: Please clarify license for Data/samples/* -- fixed SF# 1985180: Poco::Net::DNS multithreading issue -- fixed SF# 1968106: DigestOutputStream losing data -- fixed SF# 1980478: FileChannel loses messages with "archive"="timestamp" -- fixed SF# 1906481: mingw build WC_NO_BEST_FIT_CHARS is not defined -- fixed SF# 1916763: Bug in Activity? -- fixed SF# 1956300: HTTPServerConnection hanging -- fixed SF# 1963214: Typo in documentation for NumberParser::parseFloat -- fixed SF# 1981865: Cygwin Makefile lacks ThreadTarget.cpp -- fixed SF# 1981130: pointless comparison of unsigned integer with zero -- fixed SF# 1943728: POCO_APP_MAIN namespace issue -- fixed SF# 1981139: initial value of reference to non-const must be an lvalue -- fixed SF# 1995073: setupRegistry is broken if POCO_WIN32_UTF8 enabled -- fixed SF# 1981125: std::swap_ranges overloading resolution failed -- fixed SF# 2019857: Memory leak in Data::ODBC Extractor -- fixed SF# 1916761: Bug in Stopwatch? -- fixed SF# 1951443: NetworkInterface::list BSD/QNX no netmask and broadcast addr -- fixed SF# 1935310: Unhandled characters in Windows1252Encoding -- fixed SF# 1948361: a little bug for win32 -- fixed SF# 1896482: tryReadLock intermittent error -- workaround for SF# 1959059: Poco::SignalHandler deadlock - the SignalHandler can now be disabled globally by adding a - #define POCO_NO_SIGNAL_HANDLER to Poco/Config.h -- fixed SF# 2012050: Configuration key created on read access -- fixed SF# 1895483: PCRE - possible buffer overflow -- fixed SF# 2062835: Logfile _creationDate is wrong -- fixed SF# 2118943: out_of_bound access in Poco::Data::BLOB:rawContent -- fixed SF# 2121732: Prevent InvalidArgumentException in SocketReactor -- fixed SF# 1891132: Poco::Data::StatementImpl::executeWithLimit is not correct -- fixed SF# 1951604: POCO refuses to compile with g++ 4.3.0 -- fixed SF# 1954327: CYGWIN's pthread does not define PTHREAD_STACK_MIN -- fixed SF# 2124636: Discrepancy between FileWIN32(U)::handleLastError -- fixed SF# 1558300: MinGW/MSYS Builds -- fixed SF# 2123266: Memory leak under QNX6 with dinkum library -- fixed SF# 2140411: ScopedUnlock documentation bug -- fixed SF# 2036460: UUID regression tests are failing on Linux with g++ 4.3.1 -- fixed SF# 2150438: Tuple TypeHandler position increment size is wrong - - -Release 1.3.2 (2008-02-04) -========================== - -Foundation, XML, Net, Util: -- added POCO_NO_SHAREDMEMORY to Config.h -- POCO_NO_WSTRING now really disables all wide string related calls -- added template specialization for string hashfunction (performance) -- XML parser performance improvements (SAX parser is now up to 40 % faster -- added parseMemoryNP() to XMLReader and friends -- URIStreamOpener improvement: redirect logic is now in URIStreamOpener. - this enables support for redirects from http to https. -- added support for temporary redirects and useproxy return code -- added getBlocking() to Socket -- added File::isHidden() -- better WIN64 support (AMD64 and IA64 platforms are recognized) -- added support for timed lock operations to [Fast]Mutex -- SharedLibrary: dlopen() is called with RTLD_GLOBAL instead of RTLD_LOCAL - (see http://gcc.gnu.org/faq.html#dso) -- Poco::Timer threads can now run with a specified priority -- added testcase for SF# 1774351 -- fixed SF# 1784772: Message::swap omits _tid mem -- fixed SF# 1790894: IPAddress(addr,family) doesn't fail on invalid address -- fixed SF# 1804395: Constructor argument name wrong -- fixed SF# 1806807: XMLWriter::characters should ignore empty strings -- fixed SF# 1806994: property application.runAsService set too late -- fixed SF# 1828908: HTMLForm does not encode '+' -- fixed SF# 1831871: Windows configuration file line endings not correct. -- fixed SF# 1845545: TCP server hangs on shutdown -- fixed SF# 1846734: Option::validator() does not behave according to doc -- fixed SF# 1856567: Assertion in DateTimeParser::tryParse() -- fixed SF# 1864832: HTTP server sendFile() uses incorrect date -- HTTPServerResponseImpl now always sets the Date header automatically - in the constructor. -- fixed SF# 1787667: DateTimeFormatter and time related classes - (also SF# 1800031: The wrong behavior of time related classes) -- fixed SF# 1829700: TaskManager::_taskList contains tasks that never started -- fixed SF# 1834127: Anonymous enums in Tuple.h result in invalid C++ -- fixed SF# 1834130: RunnableAdapter::operator= not returning a value -- fixed SF# 1873924: Add exception code to NetException -- fixed SF# 1873929: SMTPClientSession support for name in sender field -- logging performance improvements (PatternFormatter) -- fixed SF# 1883871: TypeList operator < fails for tuples with duplicate values -- CYGWIN build works again (most things work but Foundation testsuite still fails) -- new build configuration for Digi Embedded Linux (ARM9, uclibc) -- new build configuration for PowerPC Linux - -Data: -- fixed SF# 1724388: ODBC Diagnostics -- fixed SF# 1804797: ODBC Statement multiple execution fails -- fixed SF# 1803435: SessionPool onJanitorTimer called too often? -- fixed SF# 1851997: Undefined Behavior in ODBC::Preparation -- updated SQlite to 3.5.5 - - -Release 1.3.1 (2007-08-08) -========================== - -Foundation, XML, Net, Util: -- DynamicAny fixes for char conversions -- fixed SF# 1733362: Strange timeout handling in SocketImpl::poll and Socket::select -- fixed SF patch# 1728912: crash in POCO on Solaris -- fixed SF# 1732138: Bug in WinRegistryConfiguration::getString -- fixed SF# 1730790: Reference counting breaks NetworkInterface::list() -- fixed SF# 1720733: Poco::SignalHandler bug -- fixed SF# 1718724: Poco::StreamCopier::copyStream loops forever -- fixed SF# 1718437: HashMap bug -- changed LinearHashTable iterator implementation. less templates -> good thing. -- fixed SF# 1733964: DynamicAny compile error -- UUIDGenerator: fixed infinite loop with non ethernet interfaces -- updated expat to 2.0.1 -- fixed SF# 1730566: HTTP server throws exception -- Glob supports symbolic links (additional flag to control behavior) -- fixed a problem with non blocking connect in NetSSL_OpenSSL - (see http://www.appinf.com/poco/wiki/tiki-view_forum_thread.php?comments_parentId=441&topics_threshold=0&topics_offset=29&topics_sort_mode=commentDate_desc&topics_find=&forumId=6) -- fixed a problem with SSL renegotiation in NetSSL_OpenSSL (thanks to Sanjay Chouksey for the fix) -- fixed SF# 1714753: NetSSL_OpenSSL: HTTPS connections fail with wildcard certs -- HTTPClientSession: set Host header only if it's not already set (proposed by EHL) -- NetworkInterface (Windows): Loopback interface now has correct netmask; - interfaces that do not have an IP address assigned are no longer reported. -- Fixes for VC++ W4 warnings from EHL -- SharedMemory: first constructor has an additional "server" parameter - Setting to true does not unlink the shared memory region when the SharedMemory object is destroyed. (Alessandro Oliveira Ungaro) -- fixed SF# 1768231: MemoryPool constructor - -Data: -- fixed SF# 1739989: Data::RecordSet::operator = () (in 1.3 branch) -- fixed SF# 1747525: SQLite, Transactions and Session Pooling (in 1.3 branch) -- upgraded to SQLite 3.4.1 - - -Release 1.3.0 (2007-05-07) -========================== - -- added HashMap, HashSet classes -- the HashFunction class template has been changed in an incompatible - way. The member function formerly named hash() is now the function - call operator. If you have defined your own HashFunction classes, - you have to update your code. Sorry for the inconvenience. -- added Poco::Tuple -- added AbstractCache::getAllKeys(), improved performance of the get operation -- fixed AbstractCache::size() to do cache replacement before returning the size -- added additional match() method to RegularExpression and documented the fact that the simple - match() method internally sets RE_ANCHORED and RE_NOTEMPTY. -- added ExpirationDecorator template. Decorates data types so that they can be used with UniqueExpireCaches -- added operator ! to AutoPtr and SharedPtr -- Buffer uses std::size_t instead of int -- Exception::what() now returns exception name instead of message -- added poco_ndc_dbg() macro (same as poco_ndc(), but only enabled in debug builds) -- added Environment::get(name, defaultValue); -- Foundation.h now includes Config.h at the very beginning. -- added replace() and replaceInPlace() to Poco/String.h -- added AutoPtr::assign() and SharedPtr::assign() -- added operator () to AbstractEvent -- gcc Makefiles now strip release builds -- Void now has a == and != operator -- Base64Encoder and HexBinaryEncoder now support an unlimited line length - (no newlines written), by specifying a line length of 0 -- NumberParser now has stricter syntax requirements: garbage following a number leads to a SyntaxException - (Thanks to phireis@gmail.com for the suggestion) -- fixed SF# 1676830: Don't use -rpath in libraries -- fixed SF# 1670279: AbstractConfiguration::unckeckedExpand crash -- fixed a warning in Hashtable -- HTTPClientSession now uses a keepAliveTimeout for better persistent connection handling -- added DateTime::makeUTC() and DateTime::makeLocal() -- added another constructor to LocalDateTime -- POCO_WIN32_UTF8 is ignored on non-Windows platforms -- fixed a timeout bug (with NetSSL) in HTTPSession -- AsyncChannel is automatically opened with first log() -- minor fix to NotificationQueue sample (reported by Laszlo Keresztfalvi) -- added File::canExecute() and File::setExecutable() -- added SharedMemory class to Foundation -- added FileStream, FileInputStream, FileOutputStream to Foundation -- added NodeAppender class to XML for faster DOM tree creation -- HTTPServerRequest and HTTPServerResponse are now abstract base classes, - actual functionality has moved into HTTPServerRequestImpl and - HTTPServerResponseImpl. This allows us to plug other HTTP servers - into POCO. -- added DynamicAny class to Foundation -- replaced std::fstream with Poco::FileStream across POCO. -- added Poco::Checksum class to Foundation. -- fixed SF# 1700811: conflict in threadpool -- bugfix: File::moveTo() does not work if the target is a directory -- File::copyTo() and File::moveTo() now copy/move directories recursively -- refactored NetworkInterface (now using pimpl idiom); - added broadcast address and netmask support -- fixed SF# 1688982: POP3ClientSession fails when retrieving mails with attachment -- fixed SF# 1655104: Enhance Poco::TextEncoding functionality -- added Poco::Condition class, implementing a POSIX-style condition variable -- fixed a bug in File::create() for Windows -- added poco_static_assert (imported from boost) -- added Thread::join(timeout) and Thread::tryJoin() -- ClassLoader support for named manifests (see ClassLibrary.h - POCO_EXPORT_NAMED_MANIFEST) -- POCO_WIN32_UTF8: UNICODE #define is no longer required (and no longer - automatically defined in POCO_WIN32_UTF8 is defined) -- PCRE: upgraded to PCRE version 7.1 -- fixed SF# 1682162: Suggestion on thread priority -- fixed SF# 1613460: MSVC/STLPort warnings -- fixed SF# 1709358: Format double percent std::String bug -- added WindowsConsoleChannel class to Foundation -- added AutoPtr::unsafeCast<>() and SharedPtr::unsafeCast<>() -- fixed SF# 1708552: Failed to build on arm and powerpc -- fixed SF$ 1708529: Failed to build using GCC 4.3: missing #includes -- fixed SF# 1710053: LogStream proposal -- fixed a bug involving empty root directories in Windows DirectoryIterator implementation - (see http://www.appinf.com/poco/wiki/tiki-view_forum_thread.php?comments_parentId=343&forumId=6) -- robustness improvements to ActiveMethod - removed the opportunity for memory leaks in - case something goes while invoking the method -- made C library usage more C++-like - use C++ headers (e.g. ) instead of - C ones (). Also, use C library functions in std namespace. -- added Unicode and UTF8String for improved Unicode support. - The Unicode class can be used to obtain the Unicode properties of a character. - The UTF8 class provides case insensitive comparison and case conversion - for UTF-8 encoded strings. -- added UnWindows.h header file, replaced all #include with #include "Poco/UnWindows.h". - See the Poco/UnWindows.h header file for a rationale and explanations. -- fixed SF# 1713820: StreamSocketImpl::sendBytes sends too many bytes -- File::copyTo(): on Windows, the copy now always has the read-only flag reset, to be consistent - with other platforms. -- With Microsoft Visual C++, the necessary POCO libraries are now implicitly linked when - the corresponding header files are included (#pragma comment(lib, "PocoXYZ.lib") is used). - To disable this, compile POCO with the preprocessor symbol POCO_NO_AUTOMATIC_LIBS #define'd - (see Poco/Foundation.h and Poco/Config.h). -- The Visual Studio project files for the POCO libraries now include configurations - for building static libraries. - - -Release 1.2.9 (2007-02-26) -========================== - -- fixed a formatting problem in Util::HelpFormatter -- HTTPClientSession::sendRequest() now attempts to send the complete request in one network packet. -- improved network performance of ChunkedOutputStream: chunk size and chunk data - are sent in one network packet if possible -- fixed SF# 1655035: Wrong expires field calculation in HTTPCookie - (thanks to Sergey N. Yatskevich for this and other fixes) -- fixed SF# 1655049: Fix discrepancy of a code to the description -- fixed SF# 1655170: Poco::Timezone::standardName() problem on WIN32 -- fixed SF# 1629095: POCO_WIN32_UTF8 problem - There is a new function Path::transcode() that can be used to convert a path (or any other string) - from UTF-8 to the current Windows code page. This string can the be passed as a filename - to an fstream or fopen(). This function only does the conversion on Windows, - and only, if POCO_WIN32_UTF8 is defined. Otherwise, it simply returns the unmodified argument. -- fixed SF# 1659607: Probably a bug in Poco::Net::DialogSocket -- HTTPServer network performance improvement: responses that fit into a single network packet - sent with HTTPServerResponse::sendFile() or the new HTTPServerResponse::sendBuffer() are - sent in only one packet. -- added HTTPServerResponse::sendBuffer() -- HTTPServer now sends a Bad Request response if it fails to parse the HTTP request header. -- HTTPServer now sends an Internal Server Error response if the request handler throws an - exception prior to sending a response.- enabled TCP_NODELAY per default on TCPServer/HTTPServer -- fixed a bug in HTTP persistent connection handling - (server does not send Connection: close when it reaches connection maximum) -- HTMLForm - POST submission of URL encoded form no longer uses chunked transfer encoding - (thus improving interoperability with certain web servers) -- integrated Environment.cpp from Main (missing get(var, default)) -- added missing AutoPtr include to Util/Application - (and using Poco::AutoPtr is no longer necessary for POCO_APP_MAIN macro) -- fixed SF# 1635420: Per Regents of the University of Calfornia letter, - remove advertising from BSD licensed parts -- fixed SF# 1633133: MultipartWriter writes superluous CR-LF at beginning - - -Release 1.2.8 (2007-01-04) -========================== - -- fixed SF# 1613906: Util/Application.h and GCC 3.3 -- fixed a byte order issue (failed test) in IPv6 address formatting -- fixed SF# 1626640: Poco::Net::SocketReactor bug -- fixed client side chunked transfer encoding handling -- fixed client side persistent connection handling -- fixed SF# 1623536: HTTP Server Chunked Transfer Encoding Bug -- improved HTTP server exception text -- fixed SF# 1616294: KeepAlive HTTPServerSession patch -- fixed SF# 1616296: Trivial Poco::TaskCustomNotification patch -- fixed SF# 1619282: PurgeStrategy bug fix -- fixed SF# 1620855: Format problem - there is a new format specifier %z for std::size_t, as well as a new - flag ? for %d, %i, %o, %x meaning any signed or unsigned integer - - -Release 1.2.7 (2006-12-07) -========================== - -- Poco::File: fixed root directory handling -- fixed UUIDGenerator documentation -- clarified Application::setUnixOptions() documentation -- fixes for issue [SOAPLite Transport 0000023]: SOAP Transport Listener should be able to use existing HTTPServer instance -- fixing mantis issues 13, 14, 15, 16, 17, 18, 19, 21 -- fixed SF# 1597022: Signed/unsigned warning in StringTokenizer::operator[] -- fixed SF# 1598601: Message::op= leaks -- fixed SF# 1605960: PatternFormatter crashes on custom property -- fixed SF# 1605950: Memory leak in Logger sample code -- fixed SF# 1591635: Copy Paste Error in sample code -- fixed SF# 1591512: SMTPClientSession response stream -- fixed SF #1592776: LayeredConfiguration: getRaw should enumerate in reverse order -- SF Patch # 1599848 ] VS 2005 Util build fails -- Logger::dump() now uses std::size_t instead of int for buffer size -- LayeredConfiguration now supports a priority value for each configuration. - Also, it's possible to specify for each configuration added whether it - should be writeable. -- ServerApplication: cd to root directory only if running as a daemon -- added Message::swap() -- improvements to build system: - global Makefile has correct dependencies for samples - on Windows, samples build after libraries are ready - configure supports --no-wstring and --no-fpenvironment flags - build system supports POCO_FLAGS environment variable for compiler flags -- RemoteGen: fixed error handling for write protected files (SystemException) - fixing integral constant overflow messages with large cache expiration, m_ support for type serializers, - case-insensitive comparison added - - -Release 1.2.6 (2006-11-19) -========================== - -- added additional match() method to RegularExpression and documented the fact that the simple - match() method internally sets RE_ANCHORED and RE_NOTEMPTY. -- added ExpirationDecorator template. Decorates data types so that they can be used with UniqueExpireCaches -- added operator ! to AutoPtr and SharedPtr -- Buffer uses std::size_t instead of int -- added poco_ndc_dbg() macro (same as poco_ndc(), but only enabled in debug builds) -- Foundation.h now includes Config.h at the very beginning. -- added AutoPtr::assign() and SharedPtr::assign() -- added operator () to AbstractEvent -- gcc Makefiles now strip release builds -- documentation improvements - - -Release 1.2.5 (2006-10-23) -========================== - -- Improved LoggingConfigurator: channel creation and configuration is now a two-step process. - This means that the previous problems with PropertyFileConfiguration and IniFileConfiguration when referencing other channels are solved. -- improved options handling: better handling of (non) ambiguities. - If both an option named "help" and one named "helper" is specified, this no longer causes ambiguity errors. -- added check for duplicate option definition -- ThreadPool bugfix: fixed a crash that occurs on Linux multiprocessor machines - (caused by an thread unsafe string assignment corrupting the heap...) - (SF# 1575315) -- improved ThreadPool performance -- XML now compiles with -DXML_UNICODE_WCHAR_T (SF# 1575174) -- fixed SF# 1572757: HTML forms can have more than one key/value pair with the same name -- got rid of the dynamic casts in Events, Events/Cache: simpler/faster Delegate < operator, - prevents some rare dynamic casts error from occuring when using StrategyCollection with Caches -- improvements to Logger and LoggingConfigurator: - * added Logger::unsafeGet() - * added Logger::setProperty(loggerName, propertyName, value) - * LoggingConfigurator now correctly (re)configures existing Loggers - (prior to this change, if a Logger named "a.b.c" existed before - the LoggingConfigurator started its work, and the LoggingConfigurator - configured a Logger named "a.b", then "a.b.c" would not inherit - the new configuration). -- improvements to SplitterChannel and EventLogChannel configuration -- improved LoggingRegistry exception messages -- MessageHeader::read() is more liberal with malformed message headers. - This fixes problems with certain network cameras sending malformed HTTP headers. - - -Release 1.2.4 (2006-10-02) -========================== - -- some code beautifying and improvements to comments -- DOMParser now automatically sets FEATURE_NAMESPACE_PREFIXES -- fixed SF #1567051: DOMBuilder/DOMParser/NamespaceStrategy bug -- fixed SF #1567364: POCO_APP_MAIN -- added Document::getElementById() (two-argument) and getElementByIdNS() -- added another test for DOMParser -- added AutoPtr::isNull() (to be consistent with SharedPtr) -- this release again compiles on PA-RISC HP-UX systems with aCC -- added CMAKE support files contributed by Andrew J. P. Maclean - - -Release 1.2.3 (2006-09-14) -========================== - -- configure script now checks if (auto)selected configuration is supported -- fixed SF #1552904: NamedEvent bug? -- fixed SF #1552787: POCO not handling EINTR -- fixed SF #1552846: Random::~Random uses scalar delete -- fixed SF #1552987: TLSSlot should explicitly default-construct _value -- IPAddress no longer accepts an empty address string -- split up Observer.h into AbstractObserver.h and Observer.h -- added NObserver class template which supports an AutoPtr - argument for the notification callback -- changed EchoServer sample to use NObserver -- some Windows-specific files were missing in the tarballs - - -Release 1.2.2 (2006-09-01) -========================== - -- fixed SF # 1549973: NotificationCenter::hasObservers() returns wrong result -- fixed a memory leak in EchoServer sample -- fixed SocketReactor TimeoutNotification bug (SF #1549365, SocketNotifier::addObserver() incorrect behavior) -- fixed SF# 1549513: MultipartReader does not work with Unix-style linefeeds -- MailMessage and HTMLForm: processing of multipart messages will no longer fail if a PartHandler does not read all data from the part stream. -- added additional test case (Unix-style line ends) to MultipartReaderTest - - -Release 1.2.1 (2006-08-29) -========================== - -- fixed Config.h header (no more #undefs) - -Release 1.2.0 (2006-08-29) -========================== - -- DateTime fixes: Julian Day is no longer stored internally. - Times (hours, minutes, seconds, ...) are now always taken from an utcValue (if available) and not from the Julian day. - The Julian day is only used for calculating year, month and day (except when the Julian day is the only thing we have) - This helps us get rid of rounding errors that the Julian Day arithmetic introduced.- on Windows, UUIDGenerator no longer uses Netbios, but GetAdaptersInfo instead -- The main Makefile now has correct dependencies -- updated poco-doc.pl with latest version by Caleb Epstein -- fixed SF #1542722: InflatingInputStream: buffer error -- improved Windows UTF-8 support -- added Logger::names() -- added configure script and make install target -- XMLWriter bugfix: pretty-print bug with characters() and rawCharacters() -- improvements to build system: support builds outside of source tree -- added header doc conversion tool contributed by Caleb Epstein -- fixed SF #1542618 (build/config/Linux patch) -- bugfix: BinaryReader/BinaryWriter BOM is now 16 bits, as documented -- fixed SF #1542247 (Compiler warning from OptionCallback) -- fixed SF #1542253 (ServerApplication::handleOption doesn't call Application::handleOption) -- added Application::stopOptionsProcessing() -- updated samples -- Util::Application command line handling now supports: - * argument validation (Option::validator(); see Validator, IntValidator, RegExpValidator) - * binding of argument values to config properties (Option::binding()) - * callbacks for arguments (Option::callback()) - * checking of required parameters -- changed header file locations: - Foundation headers are now in Poco (#include "Poco/Foundation.h") - XML headers are now in Poco/XML, Poco/SAX and Poco/DOM (#include "Poco/XML/XML.h") - Util headers are now in Poco/Util (#include "Poco/Util/Util.h") - etc. - Unfortunately, this change will break existing code. However, fixing the code is - a matter of a few global search/replace operations and can be done quickly. - On the plus side, POCO is now a much better citizen when used with other - libraries. -- changed namespaces: - Foundation is now Poco - XML is now Poco::XML - Util is now Poco::Util - Net is now Poco::Net -- removed namespace macros -- fixed some warnings reported by gcc -Wall -Wextra -- fixed AutoPtr and LayeredConfiguration documentation -- improved StreamSocket::receiveBytes() doc -- added Pipe and PipeStream classes -- added support for I/O redirection (pipes) to Process::launch() -- added LogStream class (ostream interface to Logger) -- improved Makefiles (no more double-building if clean all is specified) -- added CppUnit and DateTime testsuite contributions by Andrew Marlow -- improved Cygwin and minimal MinGW support -- FileChannel: gzip compression if archived files now runs in a background thread (SF #1537481) -- POCO now compiles with large (64-bit) file support on Linux (SF #1536634) -- added format() function, which provides typesafe sprintf-like functionality (SF #1327621) -- added File::isLink() -- bugfix: dangling symbolic links in a directory no longer cause recursive remove to fail with file not found error -- added Void class (useful as argument to ActiveMethod) -- ActiveResult now supports exceptions -- bugfix: Timezone::utcOffset() and Timezone::dst() returned wrong values on Unix platforms (SF #1535428) -- added ActiveDispatcher class -- added ActiveStarter class, which is a policy used by ActiveMethod for starting methods -- ActiveRunnable moved to its own header file -- ThreadPool: added startWithPriority(), which allows for running threads with a different priority -- added error handling to dir sample -- added additional test case to HTTPServer test suite- HTMLForm: should now work with request methods other than POST and GET (all non-POST requests are treated the same as GET) -- clarified HTMLForm documentation -- HTMLForm bugfix: uploaded files no longer end up in value; PartHandler is called instead -- NameValueCollection: added get(name, defaultValue) -- added HTTPFormServer sample -- added Foundation::HashTable and SimpleHashTable -- added Net::HTTPSessionFactory -- improvements to AutoPtr and SharedPtr -- improvements to namespaces handling in XMLWriter -- Foundation Cache: fixed add implementation to match the docu: a 2nd add will now simply overwrite existing entries -- added DateTime::isValid() -- added Exception::rethrow() (virtual, must be overridden by all subclasses) -- Timer can now use a user-supplied ThreadPool -- added rethrow() to exception classes -- Net: made some constructors explicit -- Net: added SocketAddress constructor to HTTPClientSession -- Net: added HTTPSession::networkException() to check for exceptions swallowed by stream classes -- Net: added single string argument constructor to SocketAddress. -- Net: improved HTTPClientSession error handling (no more "Invalid HTTP version string" exceptions when the server prematurely closes the connection due to too much load) -- Net: improved HTTPSession error handling. Exceptions while sending and receiving data are stored for later retrieval and no longer get lost since streambufs swallow them. -- Net: added HTTPLoadTest sample -- fixed a bug when opening logfiles on Unix platforms causing an existing logfile to be truncated -- bugfix: log file purge intervals given in months did not work, due to a stupid typo -- added RawSocket and ICMP classes -- UUID: fixed a doc formatting bug -- NetworkInterface::list() now includes loopback interface on Windows (SF #1460309) -- made Exception::message() and Exception::nested() inline -- added Net::UnsupportedRedirectException -- HTTPStreamFactory throws an UnsupportedRedirectException if it encounters a redirect to https -- HTTP: fixed bad 100 Continue handling in client and server code -- added CONTRIBUTORS file - - -Release 1.1.2 (2006-07-07) -========================== - -- Changed license to Boost license -- DBlite and NetSSL have been removed from the Boost-licensed release. - Please contact Applied Informatics (info@appinf.com) if you're interested in them. - - -Release 1.1.1 (2006-04-03) -========================== - -- NetSSL_OpenSSL now supports separate certificate verification - settings for client and server. -- fixed SF #1460309 (enumerating network interfaces failed on 64bit Linux) -- TCPServer no longer crashes if accept() fails - - -Release 1.1.0 (2006-03-23) -========================== - -- events no longer require awkward += new syntax -- source code and documentation cleanups -- basic support for new compilers and platforms - - -Release 1.1b2 (2006-03-04) -========================== - -- made NetSSL threadsafe (added locking callbacks for OpenSSL) -- improved OpenSSL initialization (random generator seeding) -- various changes to improve compatibility with various platforms - - -Release 1.1b1 (2006-03-03) -========================== - -- New Events package in Foundation. The package supports C#-style event handling -- New Cache package in Foundation: a templates-based caching framework -- added Any class to Foundation -- added DBLite library -- fixed a memory leak with layered configurations in the application -- made POCO_DLL the default (unless POCO_STATIC is #defined) - It is no longer necessary to specify POCO_DLL in projects that use Poco - (SourceForge Patch #1408231 and Feature Request #1407575). -- added Buffer template class to Foundation -- added the UnicodeConverter utility class. This is mainly used for Windows Unicode support and probably of little use for anything else. -- added Path::resolve() -- added Windows Unicode support. This calls the Unicode variant of the Windows API functions. - For this to work, all strings must be UTF-8 encoded and POCO_WIN32_UTF8 must be defined in all compilation units. -- added StreamCopier::copyToString() -- added URIStreamOpener::unregisterStreamFactory() and new variants of URIStreamOpener::open() that also work with filesystem paths. - This fixes SourceForge Bug #1409064 and Feature Request #1409062. -- added NodeIterator::currentNodeNP() to XML library -- added some sanity checks to UTF8Encoding::convert() -- added NetSSL - SSL support for Net library, based on OpenSSL -- console output of processes launched with Process::launch() is now visible - - -Release 1.0.0 (2006-01-19) -========================== - -- removed unnecessary console output from ProcessTest -- documentation fixes - - -Release 1.0b2 (2006-01-16) -========================== - -- added ProcessHandle class -- Process::launch() now returns a ProcessHandle instead of a process ID. - This fixes a potential problem on Windows with Process::wait() when - the process terminates before wait() is called. -- added SplitterChannel::close() -- added Logger::destroy() -- added POP3ClientSession::deleteMessage() -- added test for Process::launch() -- documentation fixes - - -Release 1.0b1 (2006-01-09) -========================== - -- improved recognition of Windows paths in Path::parseGuess() -- added setCurrentLineNumber()/getCurrentLineNumber() to CountingStreamBuf -- improvememts to StreamTokenizer and Token; fixed documentation -- added a workaround for some strange istream behaviour with VS 2005 and FTPClientSessionTest -- improved exception/error reporting in cppunit -- added POP3ClientSession -- added Process::launch() and Process::wait() -- added Mail sample -- added MailStream and SMTPClientSession classes -- renamed some methods in DialogSocket to make them more general -- NullPartHandler has moved out of HTMLForm.cpp into a separate file -- Base64Encoder now always writes \r\n line ends -- MessageHeader::quote has an optional addition arg controlling the treatment of whitespace -- bugfix: MultipartReader had a problem with empty lines (\r\n sequences) in a part -- added MailMessage and MailRecipient classes -- added text encoding support for Windows-1252 codepage - - -Release 1.0a1 (2006-01-03) [internal] -===================================== - -- mediaType is used consistently to refer to a MIME media type (some occurences of contentType and mimeType have been replaced) -- moved MediaType::quote() to MessageHeader and made it public -- added MultipartWriter::stream() -- Renamed AttachmentSource to PartSource and AttachmentHandler to PartHandler -- SIGPIPE is always blocked in main thread on Unix systems -- added EchoServer sample -- fixed a bug in SocketImpl::setBlocking() - did exactly the opposite (value to ioctl was wrong) -- fixed a memory leak in NotificationQueue sample -- added comparison operators to Socket so that Sockets can be used as keys in maps -- added Socket::setBlocking() -- added StreamSocket::connectNB() (non-blocking connect) -- added Observer::accepts() -- added SocketReactor, SocketConnector and SocketAcceptor classes to support event-based socket programming -- NamespacePrefixesStrategy now uses expat's XML_SetReturnNSTriplet(). - The previously used separate namespace handling code has been removed. - This improves performance if NamespacePrefixesStrategy is used (both the n - amespaces and namespace-prefixes SAX2 features are used) -- upgraded expat to 2.0 pre-release (2005-12-27) snapshot -- added TeeInputStream and TeeOutputStream classes -- added download sample for URIStreamOpener -- renamed registerOpener() to registerFactory() in HTTPStreamFactory and FTPStreamFactory -- added LineEndingConverter streams -- added FTPClientSession -- code and documentation clean-up -- added DialogSocket class -- reorganized HTTP test suites -- added FTPClientSession and FTPStreamFactory -- added DialogSocket class - - -Release 0.96.1 (2005-12-28) -=========================== - -- fixed a memory leak caused by a bug in Microsoft's stream implementation (see the comment in Foundation/StreamUtil.h for an explanation) -- added samples for Net library -- added uptime() and startTime() to Util::Application -- added DateTimeFormatter::format() for Timespan -- added ErrorHandler class and better exception handling for threads -- added poco_debugger() and poco_debugger_msg() macros -- added project and solution files for Visual Studio 2005 (due to some bugs/leaks in Microsofts standard library - see - http://lab.msdn.microsoft.com/productfeedback/viewfeedback.aspx?feedbackid=e08bd793-3fef-40ff-adda-ed313e0eafcc - we do not recommend using this for production purposes) -- fixed two problems with out-of-range string iterator in Path (the testsuite triggered an assertion in VC++ 8.0) -- fixed mac line endings in a few files -- added a workaround to the class loader that fixes strange behavior with VC++ 8.0. There seems to be a problem with typeid() not returning a valid typeinfo under certain circumstances. -- added buffer allocator argument to buffered stream buffer templates -- added buffer pools to HTTP to reduce memory fragmentation and to improve performance -- added Net to Windows build.cmd script -- added swap() to various classes that already support assignment -- added a null pointer check in DOMWriter::writeNode() -- fixed documentation in BinaryWriter.h and BinaryReader.h -- added explicit support for network byte order to BinaryReader and BinaryWriter -- added basic support for FreeBSD (needs more testing) -- BinaryReader: renamed readRawData() to readRaw() to be consistent with BinaryWriter::writeRaw() -- added support for uppercase output to HexBinaryEncoder. -- added MediaType class -- added QuotedPrintableEncoder and QuotedPrintableDecoder classes -- renamed ObjectFactory to Instantiator. This should prevent the confusion caused by DynamicFactory and ObjectFactory. Sorry for the inconvenience if you are already using this. -- AttachmentSource::filename() now returns const string& -- added StringAttachmentSource -- replaced old-style C casts with C++ casts in NetworkInterface.cpp -- MutexImpl (WIN32): replaced InitializeCriticalSection with InitializeCriticalSectionAndSpinCount, which should increase performance on multiprocessor or multicore systems when many locks are used. -- fixed a problem with STLport 5.0 when compiling StreamTokenizer -- HTTPStreamOpener now also works with no-path URIs (like http://www.appinf.com) -- fixed wrong delete usage (plain delete instead of delete [] was used in a few cases) -- fixed a handle leak in WinTestRunner - - -Release 0.95.4 (2005-11-07) -=========================== - -- fixed #1348006 and #1348005 - - -Release 0.95.3 (2005-10-28) [internal] -====================================== - -- updated build scripts (patch #1339015) -- added support for AMD64 platforms (patch #1339015) -- MultipartWriter creates its own boundary if an empty string is passed in as boundary -- made MultipartWriter::createBoundary() public -- fixed wrong documentation for DateTimeFormat::HTTP_FORMAT -- added support for HTTP Basic authentication -- added support for HTTP Cookies -- added support for HTML forms - - -Release 0.95.2 (2005-10-22) [internal] -====================================== - -- fixed a potential problems with streams when close in destructor fails (added try..catch block around close in destructors) -- added HTTPServer & friends -- added hasIdleThreads() method to NotificationQueue -- added TCPServer and friend -- added support for HTTP proxies to HTTPClientSession and HTTPStreamOpener -- fixed documentation bugs (Mutex.h, ClassLoader.h) - - -Relesae 0.95.1 (2005-10-15) [internal] -====================================== - -- Tasks can now throw custom notifications (contributed by Alex Fabijanic) -- renamed URIFileStreamFactory to FileStreamFactory -- added a few methods to URI (setPathEtc(), getPathEtc(), getPathAndQuery()) -- added new exception classes -- fixed some documentation -- added basic checks when reading a MessageHeader from a stream -- added HTTP classes (testsuite still incomplete) -- added MessageHeader, NameValueCollection, MultipartReader and MultipartWriter classes -- added Timespan::useconds() -- added ClassLoader::isLibraryLoaded() -- Socket classes use Timespan::useconds() to fill struct timeval -- added DatagramSocket, MulticastSocket and NetworkInterface classes -- added socket classes and related basic stuff -- added additonal constructor/assign to Timespan- added BasicBufferedBidirectionalStreamBuf -- fixed a potential MT issue in Base64Decoder -- code beautifying in [Un]BufferedStreamBuf -- more improvements to ClassLoader -- code cleanup and naming convention fixes (changed all *Imp classes to *Impl for consistency) - - -Release 0.94.1 (2005-09-30) [internal] -====================================== - -- added MetaSingleton (based on a contribution by Alex Fabijanic) -- added ClassLoader::create() -- added ClassLoader::instance() -- code clean-ups in FileChannel and related classes -- added SimpleFileChannel -- RotateAtTimeStrategy: - ::getNextRollover() rewritten (buggy) -- DateTime - microseconds assert corrected - asserts in computeGregorian() (except for year - see comment in computeGregorian()) - milliseconds calculation modified in computeGregorian() - microseconds assigned in computeGregorian() - normalize() and checkLimit() private functions to correct cases of overflow for milli/microseconds -- LocalDateTime: added timestamp() method -- FileChannel: - added "times" property (used to determine whether to use UTC or local time with RotateAtTimeStrategy) - ::setProperty() modified (whenever "times" property is set, methods setRotation and setArchive are - reinvoked to reflect the change) -- FileChannel: added support for archived file compression and archived file purging -- FileChannel tests modified -- FileChannel: put LogFile, RotateStrategy and ArchiveStrategy into their own files -- Message: added thread id field -- PatternFormatter: added %I specifier for thread id -- ThreadPool: PooledThread can be assigned a name -- TaskManager: task name is reflected in thread name -- fixed LocalDateTime::operator - (const Timespan&) [#0000004] -- upon startup all loggers' channels are set to a console channel -- improved search for application configuration files (see loadConfiguration()). -- added Glob class (fixes #1249700) -- upgraded to zlib 1.2.3 (fixes #1261712) -- added Logger::dump() -- fixed a wrong condition in Logger::log(const Message&) -- Path::find() now also works with relative paths in addition to plain file names -- added Path(const Path&, const Path&) constructor -- added SharedPtr template -- added Path::tryParse() -- SAXParser::parse()/EntityResolverImpl now works for both URIs and local filesystem paths (fixes #1254812) - - -Release 0.93.1 (2005-08-01) -=========================== - -This release contains various new features, improvements and bugfixes: -- bugfix: UUIDGenerator throws an exception if no connected ethernet adapter can - be found (and thus no MAC address can be obtained) -- added UUIDGenerator::createOne() method -- added error handling to UUID sample application -- added relational (==, !=, <, <=, >, >=) and arithmetic operators (+, -, +=, -=) to DateTime -- added LocalDateTime class -- added support for LocalDateTime to DateTimeParser and DateTimeFormatter -- added enqueueUrgentNotification() to NotificationQueue -- added support for timezone specifiers (%z, %Z) to PatternFormatter -- added [] operator and count() to StringTokenizer -- added elapsed() and isElapsed() to Timestamp -- added tzd() to Timezone -- added WinRegistryKey and WinService classes (Windows only) -- added index operator and count() to StringTokenizer -- added day/time-based log rotation (thanks to Alex Fabijanic), minor improvements to DateTimeParser -- support for Mac OS X 10.4/gcc 4.0.0 -- added NamedMutex and NamedEvent -- added Process::kill() -- added NoPermissionException -- added Task and TaskManager classes -- added ServerApplication class -- bugfix: EventLogChannel - _logFile was not properly initialized in one constructor -- bugfix: File::createDirectories did not work for hierarchies deeper than three -- added Util::FilesystemConfiguration -- documented logging policy: log() must open channel if it hasn't been opened yet -- FileChannel::log() opens channel if necessary -- the application reference passed to initialize() and reinitialize() is no longer const -- improved application logging initialization -- fixed a problem with configuration view and property placeholders -- fixed Util build configuration for Visual Studio -- improved application samples -- fixed documentation for Semaphore class - - -Release 0.92.1 (2005-05-09) -=========================== - -This release introduces the Util library that provides support for -configuration file parsing (different file formats), command line -argument processing, logging configuration and a framework for -command line/server applications. -There have also been various changes to the Foundation library: -- a new RefCountedObject class that acts as a base class for - various classes that use reference counting -- some missing members have been added to the AutoPtr template -- various improvements and bugfixes to the Logging framework, as well as - new LoggingFactory and LoggingRegistry classses, and a NullChannel class -- the SignalHandler class (Unix platforms only) -- ObjectFactory and DynamicFactory template classes -- the Path::find method for searching a file in a list of directories -- various new Exception classes - - -Release 0.91.4 (2005-04-11) -=========================== - -This is mainly a maintenance release that adds support for QNX Neutrino -and OpenVMS. There are also minor bugfixes and improvements. - -The Unix build system has been modified to work on QNX Neutrino. -The OpenVMS build system has been fixed and works now. -Some missing #include's have been added for QNX Neutrino. -Foundation/String.h: icompare now supports comparison with const char*; -the classic C version of isspace() has been used in a few places instead of the -C++ version, this has been fixed. -Foundation/Exception.h: IllegalStateException added. - - -Release 0.91.3 (2005-03-19) -=========================== - -This is a maintenance release that adds support for Solaris/Sun Forte C++. -No new features have been added. - -An implementation of FPEnvironment for Solaris has been included. -All stream classes have been modified to work around an initialization -problem that surfaced with Sun's C++ compiler when using STLport. -Source-code compatibility with the previous release is not affected. Various -minor changes, mostly adding missing #include's for Solaris. - - -Release 0.91.2 (2005-02-27) -=========================== - -Minor improvements to the Unix build system. No actual changes in the -libraries. - - -Release 0.91.1 (2005-02-21) -=========================== - -This is the first public release of the C++ Portable Components. -The release does not contain all features planned for the later 1.0 release -(the NET library is missing, for example), but is already quite usable. -Please refer to the README file for more information and instructions for -building the libraries. diff --git a/base/poco/CONTRIBUTORS b/base/poco/CONTRIBUTORS deleted file mode 100644 index 7a74a9cc00c..00000000000 --- a/base/poco/CONTRIBUTORS +++ /dev/null @@ -1,52 +0,0 @@ -Guenter Obiltschnig -Alex Fabijanic -Peter Schojer -Ferdinand Beyer -Krzysztof Burghardt -Claus Dabringer -Caleb Epstein -Eran Hammer-Lahav -Chris Johnson -Sergey Kholodilov -Ryan Kraay -Larry Lewis -Andrew J. P. Maclean -Andrew Marlow -Paschal Mushubi -Jiang Shan -David Shawley -Sergey Skorokhodov -Tom Tan -Sergey N. Yatskevich -Marc Chevrier -Philippe Cuvillier -Marian Krivos -Franky Braem -Philip Prindeville -Anton Yabchinskiy -Rangel Reale -Fabrizio Duhem -Patrick White -Mike Naquin -Roger Meier -Mathaus Mendel -Arturo Castro -Adrian Imboden -Matej Knopp -Patrice Tarabbia -Lucas Clemente -Karl Reid -Pascal Bach -Cristian Thiago Moecke -Sergei Nikulov -Aaron Kaluszka -Iyed Bennour -Scott Davis -Kristin Cowalcijk -Yuval Kashtan -Christopher Baker -Scott Davis -Jeff Adams -Martin Osborne -Björn Schramke -Francis Andre diff --git a/base/poco/Crypto/include/Poco/Crypto/ECKey.h b/base/poco/Crypto/include/Poco/Crypto/ECKey.h index 14f2ac0a189..15d5401cbcd 100644 --- a/base/poco/Crypto/include/Poco/Crypto/ECKey.h +++ b/base/poco/Crypto/include/Poco/Crypto/ECKey.h @@ -23,7 +23,6 @@ #include "Poco/Crypto/KeyPair.h" #include "Poco/Crypto/ECKeyImpl.h" - namespace Poco { namespace Crypto { @@ -52,7 +51,7 @@ public: /// Extracts the EC private key from the given certificate. ECKey(const std::string& eccGroup); - /// Creates the ECKey. Creates a new public/private keypair using the given parameters. + /// Creates the ECKey. Creates a new public/private key pair using the given parameters. /// Can be used to sign data and verify signatures. ECKey(const std::string& publicKeyFile, const std::string& privateKeyFile, const std::string& privateKeyPassphrase = ""); diff --git a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h index fbcdad5b19c..2b0062a3e13 100644 --- a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h +++ b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h @@ -126,7 +126,7 @@ public: /// key is not exported. int type() const; - /// Retuns the EVPPKey type NID. + /// Returns the EVPPKey type NID. bool isSupported(int type) const; /// Returns true if OpenSSL type is supported diff --git a/base/poco/Crypto/include/Poco/Crypto/OpenSSLInitializer.h b/base/poco/Crypto/include/Poco/Crypto/OpenSSLInitializer.h index 42c97ae465d..ce822a69710 100644 --- a/base/poco/Crypto/include/Poco/Crypto/OpenSSLInitializer.h +++ b/base/poco/Crypto/include/Poco/Crypto/OpenSSLInitializer.h @@ -42,7 +42,7 @@ namespace Crypto { class Crypto_API OpenSSLInitializer - /// Initalizes the OpenSSL library. + /// Initializes the OpenSSL library. /// /// The class ensures the earliest initialization and the /// latest shutdown of the OpenSSL library. diff --git a/base/poco/Crypto/src/CryptoStream.cpp b/base/poco/Crypto/src/CryptoStream.cpp index 6b29b7f0775..9cb693990a3 100644 --- a/base/poco/Crypto/src/CryptoStream.cpp +++ b/base/poco/Crypto/src/CryptoStream.cpp @@ -111,7 +111,7 @@ int CryptoStreamBuf::readFromDevice(char* buffer, std::streamsize length) { int m = (static_cast(length) - count)/2 - static_cast(_pTransform->blockSize()); - // Make sure we can read at least one more block. Explicitely check + // Make sure we can read at least one more block. Explicitly check // for m < 0 since blockSize() returns an unsigned int and the // comparison might give false results for m < 0. if (m <= 0) diff --git a/base/poco/Crypto/src/X509Certificate.cpp b/base/poco/Crypto/src/X509Certificate.cpp index a32e465bb0a..542066665df 100644 --- a/base/poco/Crypto/src/X509Certificate.cpp +++ b/base/poco/Crypto/src/X509Certificate.cpp @@ -142,7 +142,7 @@ void X509Certificate::load(const std::string& path) _pCert = PEM_read_bio_X509(pBIO, 0, 0, 0); BIO_free(pBIO); - if (!_pCert) throw Poco::ReadFileException("Faild to load certificate from", path); + if (!_pCert) throw Poco::ReadFileException("Failed to load certificate from", path); init(); } diff --git a/base/poco/Data/ODBC/include/Poco/Data/ODBC/Binder.h b/base/poco/Data/ODBC/include/Poco/Data/ODBC/Binder.h index fd3241fbf8c..5203f65679f 100644 --- a/base/poco/Data/ODBC/include/Poco/Data/ODBC/Binder.h +++ b/base/poco/Data/ODBC/include/Poco/Data/ODBC/Binder.h @@ -557,7 +557,7 @@ private: if (size == _maxFieldSize) { getMinValueSize(val, size); - // accomodate for terminating zero + // accommodate for terminating zero if (size != _maxFieldSize) ++size; } @@ -623,7 +623,7 @@ private: if (size == _maxFieldSize) { getMinValueSize(val, size); - // accomodate for terminating zero + // accommodate for terminating zero if (size != _maxFieldSize) size += sizeof(UTF16Char); } @@ -942,9 +942,9 @@ private: /// This function runs for query and stored procedure parameters (in and /// out-bound). Some drivers, however, do not care about knowing this /// information to start with. For that reason, after all the attempts - /// to discover the required values are unsuccesfully exhausted, the values + /// to discover the required values are unsuccessfully exhausted, the values /// are both set to zero and no exception is thrown. - /// However, if the colSize is succesfully retrieved and it is greater than + /// However, if the colSize is successfully retrieved and it is greater than /// session-wide maximum allowed field size, LengthExceededException is thrown. void setParamSetSize(std::size_t length); diff --git a/base/poco/Data/ODBC/include/Poco/Data/ODBC/ODBC.h b/base/poco/Data/ODBC/include/Poco/Data/ODBC/ODBC.h index 647e4b8f060..37e4c1cb72a 100644 --- a/base/poco/Data/ODBC/include/Poco/Data/ODBC/ODBC.h +++ b/base/poco/Data/ODBC/include/Poco/Data/ODBC/ODBC.h @@ -31,7 +31,7 @@ // from a DLL simpler. All files within this DLL are compiled with the ODBC_EXPORTS // symbol defined on the command line. this symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// ODBC_API functions as being imported from a DLL, wheras this DLL sees symbols +// ODBC_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if defined(_WIN32) && defined(POCO_DLL) diff --git a/base/poco/Data/ODBC/include/Poco/Data/ODBC/SessionImpl.h b/base/poco/Data/ODBC/include/Poco/Data/ODBC/SessionImpl.h index 72937bfac28..a33b26cd895 100644 --- a/base/poco/Data/ODBC/include/Poco/Data/ODBC/SessionImpl.h +++ b/base/poco/Data/ODBC/include/Poco/Data/ODBC/SessionImpl.h @@ -57,7 +57,7 @@ public: bool autoBind = true, bool autoExtract = true); /// Creates the SessionImpl. Opens a connection to the database. - /// Throws NotConnectedException if connection was not succesful. + /// Throws NotConnectedException if connection was not successful. //@ deprecated SessionImpl(const std::string& connect, diff --git a/base/poco/Data/ODBC/include/Poco/Data/ODBC/TypeInfo.h b/base/poco/Data/ODBC/include/Poco/Data/ODBC/TypeInfo.h index f8b14a8014d..60d46fb4bc5 100644 --- a/base/poco/Data/ODBC/include/Poco/Data/ODBC/TypeInfo.h +++ b/base/poco/Data/ODBC/include/Poco/Data/ODBC/TypeInfo.h @@ -39,7 +39,7 @@ class ODBC_API TypeInfo /// /// This class provides mapping between C and SQL datatypes as well /// as datatypes supported by the underlying database. In order for database - /// types to be available, a valid conection handle must be supplied at either + /// types to be available, a valid connection handle must be supplied at either /// object construction time, or at a later point in time, through call to /// fillTypeInfo member function. /// diff --git a/base/poco/Data/ODBC/src/ODBCStatementImpl.cpp b/base/poco/Data/ODBC/src/ODBCStatementImpl.cpp index b209fe772a9..9dd8c4e7a8f 100644 --- a/base/poco/Data/ODBC/src/ODBCStatementImpl.cpp +++ b/base/poco/Data/ODBC/src/ODBCStatementImpl.cpp @@ -382,7 +382,7 @@ std::string ODBCStatementImpl::nativeSQL() delete [] pNative; throw ConnectionException(_rConnection, "SQLNativeSql()"); } - ++retlen;//accomodate for terminating '\0' + ++retlen;//accommodate for terminating '\0' }while (retlen > length); std::string sql(pNative); diff --git a/base/poco/Data/ODBC/src/Preparator.cpp b/base/poco/Data/ODBC/src/Preparator.cpp index 1b00eca2a27..55427692d0a 100644 --- a/base/poco/Data/ODBC/src/Preparator.cpp +++ b/base/poco/Data/ODBC/src/Preparator.cpp @@ -156,7 +156,7 @@ std::size_t Preparator::maxDataSize(std::size_t pos) const ODBCMetaColumn mc(_rStmt, pos); sz = mc.length(); - // accomodate for terminating zero (non-bulk only!) + // accommodate for terminating zero (non-bulk only!) MetaColumn::ColumnDataType type = mc.type(); if (!isBulk() && ((ODBCMetaColumn::FDT_WSTRING == type) || (ODBCMetaColumn::FDT_STRING == type))) ++sz; } diff --git a/base/poco/Data/include/Poco/Data/AbstractBinder.h b/base/poco/Data/include/Poco/Data/AbstractBinder.h index d88efba0a91..7f88a879030 100644 --- a/base/poco/Data/include/Poco/Data/AbstractBinder.h +++ b/base/poco/Data/include/Poco/Data/AbstractBinder.h @@ -170,7 +170,7 @@ public: /// Binds a long. virtual void bind(std::size_t pos, const unsigned long& val, Direction dir = PD_IN) = 0; - /// Binds an unsiged long. + /// Binds an unsigned long. virtual void bind(std::size_t pos, const std::vector& val, Direction dir = PD_IN); /// Binds a long vector. diff --git a/base/poco/Data/include/Poco/Data/AbstractExtraction.h b/base/poco/Data/include/Poco/Data/AbstractExtraction.h index 8960d74441b..ecab3ee72e8 100644 --- a/base/poco/Data/include/Poco/Data/AbstractExtraction.h +++ b/base/poco/Data/include/Poco/Data/AbstractExtraction.h @@ -138,7 +138,7 @@ public: bool isValueNull(const std::string& str, bool deflt); /// Overload for const reference to std::string. /// - /// Returns true when folowing conditions are met: + /// Returns true when following conditions are met: /// /// - string is empty /// - getEmptyStringIsNull() returns true @@ -146,7 +146,7 @@ public: bool isValueNull(const Poco::UTF16String& str, bool deflt); /// Overload for const reference to UTF16String. /// - /// Returns true when folowing conditions are met: + /// Returns true when following conditions are met: /// /// - string is empty /// - getEmptyStringIsNull() returns true diff --git a/base/poco/Data/include/Poco/Data/Binding.h b/base/poco/Data/include/Poco/Data/Binding.h index 464c3143e1a..2da6b214741 100644 --- a/base/poco/Data/include/Poco/Data/Binding.h +++ b/base/poco/Data/include/Poco/Data/Binding.h @@ -1472,7 +1472,7 @@ inline AbstractBindingVec& io(AbstractBindingVec& bv) template inline AbstractBinding::Ptr bind(T t, const std::string& name) /// Convenience function for a more compact Binding creation. - /// This funtion differs from use() in its value copy semantics. + /// This function differs from use() in its value copy semantics. { return new CopyBinding(t, name, AbstractBinding::PD_IN); } @@ -1481,7 +1481,7 @@ inline AbstractBinding::Ptr bind(T t, const std::string& name) template inline AbstractBinding::Ptr bind(T t) /// Convenience function for a more compact Binding creation. - /// This funtion differs from use() in its value copy semantics. + /// This function differs from use() in its value copy semantics. { return Poco::Data::Keywords::bind(t, ""); } diff --git a/base/poco/Data/include/Poco/Data/Bulk.h b/base/poco/Data/include/Poco/Data/Bulk.h index 7b30ae63350..324dc287488 100644 --- a/base/poco/Data/include/Poco/Data/Bulk.h +++ b/base/poco/Data/include/Poco/Data/Bulk.h @@ -39,10 +39,10 @@ public: /// Destroys the bulk. const Limit& limit() const; - /// Returns the limit asociated with this bulk object. + /// Returns the limit associated with this bulk object. Poco::UInt32 size() const; - /// Returns the value of the limit asociated with + /// Returns the value of the limit associated with /// this bulk object. private: diff --git a/base/poco/Data/include/Poco/Data/BulkExtraction.h b/base/poco/Data/include/Poco/Data/BulkExtraction.h index 75c25f69b7d..f47f425a818 100644 --- a/base/poco/Data/include/Poco/Data/BulkExtraction.h +++ b/base/poco/Data/include/Poco/Data/BulkExtraction.h @@ -138,7 +138,7 @@ class InternalBulkExtraction: public BulkExtraction /// Container Data Type specialization extension for extraction of values from a query result set. /// /// This class is intended for PocoData internal use - it is used by StatementImpl - /// to automaticaly create internal BulkExtraction in cases when statement returns data and no external storage + /// to automatically create internal BulkExtraction in cases when statement returns data and no external storage /// was supplied. It is later used by RecordSet to retrieve the fetched data after statement execution. /// It takes ownership of the Column pointer supplied as constructor argument. Column object, in turn /// owns the data container pointer. diff --git a/base/poco/Data/include/Poco/Data/Data.h b/base/poco/Data/include/Poco/Data/Data.h index 8a9e532e952..96a2b997d13 100644 --- a/base/poco/Data/include/Poco/Data/Data.h +++ b/base/poco/Data/include/Poco/Data/Data.h @@ -28,7 +28,7 @@ // from a DLL simpler. All files within this DLL are compiled with the Data_EXPORTS // symbol defined on the command line. this symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// Data_API functions as being imported from a DLL, wheras this DLL sees symbols +// Data_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if defined(_WIN32) && defined(POCO_DLL) diff --git a/base/poco/Data/include/Poco/Data/Extraction.h b/base/poco/Data/include/Poco/Data/Extraction.h index 7b98700e905..66fcae60b44 100644 --- a/base/poco/Data/include/Poco/Data/Extraction.h +++ b/base/poco/Data/include/Poco/Data/Extraction.h @@ -496,7 +496,7 @@ class InternalExtraction: public Extraction /// Container Data Type specialization extension for extraction of values from a query result set. /// /// This class is intended for PocoData internal use - it is used by StatementImpl - /// to automaticaly create internal Extraction in cases when statement returns data and no external storage + /// to automatically create internal Extraction in cases when statement returns data and no external storage /// was supplied. It is later used by RecordSet to retrieve the fetched data after statement execution. /// It takes ownership of the Column pointer supplied as constructor argument. Column object, in turn /// owns the data container pointer. diff --git a/base/poco/Data/include/Poco/Data/RecordSet.h b/base/poco/Data/include/Poco/Data/RecordSet.h index 91046c36b9f..e2b32e0c508 100644 --- a/base/poco/Data/include/Poco/Data/RecordSet.h +++ b/base/poco/Data/include/Poco/Data/RecordSet.h @@ -63,7 +63,7 @@ class Data_API RecordSet: private Statement /// /// The third (optional) argument passed to the Recordset constructor is a RowFormatter /// implementation. The formatter is used in conjunction with << operator for recordset - /// data formating. + /// data formatting. /// /// The number of rows in the RecordSet can be limited by specifying /// a limit for the Statement. diff --git a/base/poco/Data/include/Poco/Data/Row.h b/base/poco/Data/include/Poco/Data/Row.h index 6144967803f..ec59edc0536 100644 --- a/base/poco/Data/include/Poco/Data/Row.h +++ b/base/poco/Data/include/Poco/Data/Row.h @@ -177,13 +177,13 @@ public: /// Converts the column names to string. void formatNames() const; - /// Fomats the column names. + /// Formats the column names. const std::string& valuesToString() const; - /// Converts the row values to string and returns the formated string. + /// Converts the row values to string and returns the formatted string. void formatValues() const; - /// Fomats the row values. + /// Formats the row values. bool operator == (const Row& other) const; /// Equality operator. diff --git a/base/poco/Data/include/Poco/Data/RowFormatter.h b/base/poco/Data/include/Poco/Data/RowFormatter.h index a2a0c8614eb..ec8ec0b52ef 100644 --- a/base/poco/Data/include/Poco/Data/RowFormatter.h +++ b/base/poco/Data/include/Poco/Data/RowFormatter.h @@ -49,7 +49,7 @@ class Data_API RowFormatter /// Statement always has the ownership of the row formatter and shares /// it with rows through RecordSet. /// - /// To accomodate for various formatting needs, a formatter can operate in two modes: + /// To accommodate for various formatting needs, a formatter can operate in two modes: /// /// - progressive: formatted individual row strings are gemerated and returned from each /// call to formatValues; @@ -63,7 +63,7 @@ class Data_API RowFormatter /// void formatValues(const ValueVec&) member calls should be used in this case /// /// When formatter is used in conjunction with Row/RecordSet, the formatting members corresponding - /// to the formater mode are expected to be implemented. If a call is propagated to this parent + /// to the formatter mode are expected to be implemented. If a call is propagated to this parent /// class, the functions do nothing or silently return empty string respectively. /// { @@ -132,7 +132,7 @@ public: /// to empty strings and row count to INVALID_ROW_COUNT. Mode getMode() const; - /// Returns the formater mode. + /// Returns the formatter mo void setMode(Mode mode); /// Sets the fromatter mode. diff --git a/base/poco/Data/include/Poco/Data/SQLChannel.h b/base/poco/Data/include/Poco/Data/SQLChannel.h index e76d67626a3..b5ad819786f 100644 --- a/base/poco/Data/include/Poco/Data/SQLChannel.h +++ b/base/poco/Data/include/Poco/Data/SQLChannel.h @@ -50,12 +50,12 @@ class Data_API SQLChannel: public Poco::Channel /// /// The table name is configurable through "table" property. /// Other than DateTime filed name used for optiona time-based archiving purposes, currently the - /// field names are not mandated. However, it is recomended to use names as specified above. + /// field names are not mandated. However, it is recommended to use names as specified above. /// /// To provide as non-intrusive operation as possbile, the log entries are cached and /// inserted into the target database asynchronously by default . The blocking, however, will occur /// before the next entry insertion with default timeout of 1 second. The default settings can be - /// overriden (see async, timeout and throw properties for details). + /// overridden (see async, timeout and throw properties for details). /// If throw property is false, insertion timeouts are ignored, otherwise a TimeoutException is thrown. /// To force insertion of every entry, set timeout to 0. This setting, however, introduces /// a risk of long blocking periods in case of remote server communication delays. @@ -102,8 +102,8 @@ public: /// Table must exist in the target database. To disable archiving, /// set this property to empty string. /// - /// * async: Indicates asynchronous execution. When excuting asynchronously, - /// messages are sent to the target using asyncronous execution. + /// * async: Indicates asynchronous execution. When executing asynchronously, + /// messages are sent to the target using asynchronous execution. /// However, prior to the next message being processed and sent to /// the target, the previous operation must have been either completed /// or timed out (see timeout and throw properties for details on @@ -123,7 +123,7 @@ public: std::size_t wait(); /// Waits for the completion of the previous operation and returns - /// the result. If chanel is in synchronous mode, returns 0 immediately. + /// the result. If channel is in synchronous mode, returns 0 immediately. static void registerChannel(); /// Registers the channel with the global LoggingFactory. @@ -156,7 +156,7 @@ private: void logAsync(const Message& msg); /// Waits for previous operation completion and /// calls logSync(). If the previous operation times out, - /// and _throw is true, TimeoutException is thrown, oterwise + /// and _throw is true, TimeoutException is thrown, otherwise /// the timeout is ignored and log entry is lost. void logSync(const Message& msg); diff --git a/base/poco/Data/include/Poco/Data/Session.h b/base/poco/Data/include/Poco/Data/Session.h index bf96efefa94..a3fa9439914 100644 --- a/base/poco/Data/include/Poco/Data/Session.h +++ b/base/poco/Data/include/Poco/Data/Session.h @@ -76,7 +76,7 @@ class Data_API Session /// The above example assigns the variable i to the ":data" placeholder in the SQL query. The query is parsed and compiled exactly /// once, but executed 100 times. At the end the values 0 to 99 will be present in the Table "DUMMY". /// - /// A faster implementaton of the above code will simply create a vector of int + /// A faster implementation of the above code will simply create a vector of int /// and use the vector as parameter to the use clause (you could also use set or multiset instead): /// /// std::vector data; @@ -200,7 +200,7 @@ public: /// reconnect a disconnected session. /// If the connection is not established, /// a ConnectionFailedException is thrown. - /// Zero timout means indefinite + /// Zero timeout means indefinite void close(); /// Closes the session. diff --git a/base/poco/Data/include/Poco/Data/SessionImpl.h b/base/poco/Data/include/Poco/Data/SessionImpl.h index 223c2bf978b..924e4b3e279 100644 --- a/base/poco/Data/include/Poco/Data/SessionImpl.h +++ b/base/poco/Data/include/Poco/Data/SessionImpl.h @@ -65,7 +65,7 @@ public: /// a disconnected session. /// If the connection is not established within requested timeout /// (specified in seconds), a ConnectionFailedException is thrown. - /// Zero timout means indefinite + /// Zero timeout means indefinite virtual void close() = 0; /// Closes the connection. diff --git a/base/poco/Data/include/Poco/Data/Statement.h b/base/poco/Data/include/Poco/Data/Statement.h index 5373e092cc8..f84a7445c89 100644 --- a/base/poco/Data/include/Poco/Data/Statement.h +++ b/base/poco/Data/include/Poco/Data/Statement.h @@ -66,7 +66,7 @@ class Data_API Statement /// /// See individual functions documentation for more details. /// - /// Statement owns the RowFormatter, which can be provided externaly through setFormatter() + /// Statement owns the RowFormatter, which can be provided externally through setFormatter() /// member function. /// If no formatter is externally supplied to the statement, the SimpleRowFormatter is lazy /// created and used. diff --git a/base/poco/Data/include/Poco/Data/StatementCreator.h b/base/poco/Data/include/Poco/Data/StatementCreator.h index ca278041731..bd0b8134683 100644 --- a/base/poco/Data/include/Poco/Data/StatementCreator.h +++ b/base/poco/Data/include/Poco/Data/StatementCreator.h @@ -33,7 +33,7 @@ class Data_API StatementCreator { public: StatementCreator(); - /// Creates an unitialized StatementCreator. + /// Creates an uninitialized StatementCreator. StatementCreator(Poco::AutoPtr ptrImpl); /// Creates a StatementCreator. diff --git a/base/poco/Data/include/Poco/Data/StatementImpl.h b/base/poco/Data/include/Poco/Data/StatementImpl.h index d9371a74037..daf99778380 100644 --- a/base/poco/Data/include/Poco/Data/StatementImpl.h +++ b/base/poco/Data/include/Poco/Data/StatementImpl.h @@ -129,7 +129,7 @@ public: /// affected for all other statements (insert, update, delete). /// If reset is true (default), the underlying bound storage is /// reset and reused. In case of containers, this means they are - /// cleared and resized to accomodate the number of rows returned by + /// cleared and resized to accommodate the number of rows returned by /// this execution step. When reset is false, data is appended to the /// bound containers during multiple execute calls. @@ -182,7 +182,7 @@ protected: virtual std::size_t next() = 0; /// Retrieves the next row or set of rows from the resultset and - /// returns the number of rows retreved. + /// returns the number of rows retrieved. /// /// Will throw, if the resultset is empty. /// Expects the statement to be compiled and bound. @@ -279,7 +279,7 @@ protected: /// usually the case when stored procedures are called). In such cases /// no storage is needed because output parameters serve as storage. /// At the Data framework level, this function always returns false. - /// When connector-specific behavior is desired, it should be overriden + /// When connector-specific behavior is desired, it should be overridden /// by the statement implementation. std::size_t currentDataSet() const; diff --git a/base/poco/Data/include/Poco/Data/Transaction.h b/base/poco/Data/include/Poco/Data/Transaction.h index a8fdc4699b1..ceb913d4bb1 100644 --- a/base/poco/Data/include/Poco/Data/Transaction.h +++ b/base/poco/Data/include/Poco/Data/Transaction.h @@ -54,7 +54,7 @@ public: /// reference as an argument. /// /// When transaction is created using this constructor, it is executed and - /// commited automatically. If no error occurs, rollback is disabled and does + /// committed automatically. If no error occurs, rollback is disabled and does /// not occur at destruction time. If an error occurs resulting in exception being /// thrown, the transaction is rolled back and exception propagated to calling code. /// @@ -82,7 +82,7 @@ public: ~Transaction(); /// Destroys the Transaction. - /// Rolls back the current database transaction if it has not been commited + /// Rolls back the current database transaction if it has not been committed /// (by calling commit()), or rolled back (by calling rollback()). /// /// If an exception is thrown during rollback, the exception is logged @@ -109,7 +109,7 @@ public: void execute(const std::vector& sql); /// Executes all the SQL statements supplied in the vector and, after the last - /// one is sucesfully executed, commits the transaction. + /// one is successfully executed, commits the transaction. /// If an error occurs during execution, transaction is rolled back. /// Passing true value for commit disables rollback during destruction /// of this Transaction object. diff --git a/base/poco/Foundation/include/Poco/AbstractCache.h b/base/poco/Foundation/include/Poco/AbstractCache.h index 37cfc25b1ef..4866e103101 100644 --- a/base/poco/Foundation/include/Poco/AbstractCache.h +++ b/base/poco/Foundation/include/Poco/AbstractCache.h @@ -55,7 +55,8 @@ public: initialize(); } - AbstractCache(const TStrategy& strat): _strategy(strat) + AbstractCache(const TStrategy & strategy) + : _strategy(strategy) { initialize(); } @@ -85,7 +86,7 @@ public: /// If for the key already an entry exists, it will be overwritten. /// The difference to add is that no remove or add events are thrown in this case, /// just a simply silent update is performed - /// If the key doesnot exist the behavior is equal to add, ie. an add event is thrown + /// If the key does not exist the behavior is equal to add, ie. an add event is thrown { typename TMutex::ScopedLock lock(_mutex); doUpdate(key, val); @@ -105,7 +106,7 @@ public: /// If for the key already an entry exists, it will be overwritten. /// The difference to add is that no remove or add events are thrown in this case, /// just an Update is thrown - /// If the key doesnot exist the behavior is equal to add, ie. an add event is thrown + /// If the key does not exist the behavior is equal to add, ie. an add event is thrown { typename TMutex::ScopedLock lock(_mutex); doUpdate(key, val); diff --git a/base/poco/Foundation/include/Poco/AbstractEvent.h b/base/poco/Foundation/include/Poco/AbstractEvent.h index 917092a9a0e..1fceb6e58e4 100644 --- a/base/poco/Foundation/include/Poco/AbstractEvent.h +++ b/base/poco/Foundation/include/Poco/AbstractEvent.h @@ -158,9 +158,9 @@ public: { } - AbstractEvent(const TStrategy& strat): + AbstractEvent(const TStrategy& strategy): _executeAsync(this, &AbstractEvent::executeAsyncImpl), - _strategy(strat), + _strategy(strategy), _enabled(true) { } @@ -251,7 +251,7 @@ public: ActiveResult notifyAsync(const void* pSender, const TArgs& args) /// Sends a notification to all registered delegates. The order is /// determined by the TStrategy. This method is not blocking and will - /// immediately return. The delegates are invoked in a seperate thread. + /// immediately return. The delegates are invoked in a separate thread. /// Call activeResult.wait() to wait until the notification has ended. /// While executing, other objects can change the delegate list. These changes don't /// influence the current active notifications but are activated with @@ -364,9 +364,9 @@ public: { } - AbstractEvent(const TStrategy& strat): + AbstractEvent(const TStrategy& strategy): _executeAsync(this, &AbstractEvent::executeAsyncImpl), - _strategy(strat), + _strategy(strategy), _enabled(true) { } @@ -453,7 +453,7 @@ public: ActiveResult notifyAsync(const void* pSender) /// Sends a notification to all registered delegates. The order is /// determined by the TStrategy. This method is not blocking and will - /// immediately return. The delegates are invoked in a seperate thread. + /// immediately return. The delegates are invoked in a separate thread. /// Call activeResult.wait() to wait until the notification has ended. /// While executing, other objects can change the delegate list. These changes don't /// influence the current active notifications but are activated with diff --git a/base/poco/Foundation/include/Poco/AbstractStrategy.h b/base/poco/Foundation/include/Poco/AbstractStrategy.h index 8d0ac25a224..0bd8b548f69 100644 --- a/base/poco/Foundation/include/Poco/AbstractStrategy.h +++ b/base/poco/Foundation/include/Poco/AbstractStrategy.h @@ -67,7 +67,7 @@ public: virtual void onReplace(const void* pSender, std::set& elemsToRemove) = 0; /// Used by the Strategy to indicate which elements should be removed from /// the cache. Note that onReplace does not change the current list of keys. - /// The cache object is reponsible to remove the elements. + /// The cache object is responsible to remove the elements. }; diff --git a/base/poco/Foundation/include/Poco/AutoReleasePool.h b/base/poco/Foundation/include/Poco/AutoReleasePool.h index 0b13956e8ea..2319bb4abe9 100644 --- a/base/poco/Foundation/include/Poco/AutoReleasePool.h +++ b/base/poco/Foundation/include/Poco/AutoReleasePool.h @@ -29,7 +29,7 @@ template class AutoReleasePool /// An AutoReleasePool implements simple garbage collection for /// reference-counted objects. - /// It temporarily takes ownwership of reference-counted objects that + /// It temporarily takes ownership of reference-counted objects that /// nobody else wants to take ownership of and releases them /// at a later, appropriate point in time. /// diff --git a/base/poco/Foundation/include/Poco/BinaryWriter.h b/base/poco/Foundation/include/Poco/BinaryWriter.h index 7941117c049..216f01d156b 100644 --- a/base/poco/Foundation/include/Poco/BinaryWriter.h +++ b/base/poco/Foundation/include/Poco/BinaryWriter.h @@ -43,7 +43,7 @@ class Foundation_API BinaryWriter /// data type sizes (e.g., 32-bit and 64-bit architectures), as the sizes /// of some of the basic types may be different. For example, writing a /// long integer on a 64-bit system and reading it on a 32-bit system - /// may yield an incorrent result. Use fixed-size types (Int32, Int64, etc.) + /// may yield an incorrect result. Use fixed-size types (Int32, Int64, etc.) /// in such a case. { public: diff --git a/base/poco/Foundation/include/Poco/Clock.h b/base/poco/Foundation/include/Poco/Clock.h index 43bb22f6406..b685827413d 100644 --- a/base/poco/Foundation/include/Poco/Clock.h +++ b/base/poco/Foundation/include/Poco/Clock.h @@ -28,7 +28,7 @@ class Foundation_API Clock /// A Clock stores a monotonic* clock value /// with (theoretical) microseconds resolution. /// Clocks can be compared with each other - /// and simple arithmetics are supported. + /// and simple arithmetic are supported. /// /// [*] Note that Clock values are only monotonic if /// the operating system provides a monotonic clock. diff --git a/base/poco/Foundation/include/Poco/Config.h b/base/poco/Foundation/include/Poco/Config.h index 7dbe018237f..73f8d4cfe4c 100644 --- a/base/poco/Foundation/include/Poco/Config.h +++ b/base/poco/Foundation/include/Poco/Config.h @@ -94,7 +94,7 @@ // Small object size in bytes. When assigned to Any or Var, -// objects larger than this value will be alocated on the heap, +// objects larger than this value will be allocated on the heap, // while those smaller will be placement new-ed into an // internal buffer. #if !defined(POCO_SMALL_OBJECT_SIZE) && !defined(POCO_NO_SOO) diff --git a/base/poco/Foundation/include/Poco/DateTime.h b/base/poco/Foundation/include/Poco/DateTime.h index ab69894719b..3fd3513abb7 100644 --- a/base/poco/Foundation/include/Poco/DateTime.h +++ b/base/poco/Foundation/include/Poco/DateTime.h @@ -162,7 +162,7 @@ public: /// There will be no week 0 in 2007. int day() const; - /// Returns the day witin the month (1 to 31). + /// Returns the day within the month (1 to 31). int dayOfWeek() const; /// Returns the weekday (0 to 6, where diff --git a/base/poco/Foundation/include/Poco/Dynamic/VarHolder.h b/base/poco/Foundation/include/Poco/Dynamic/VarHolder.h index e0af6a6d593..944bb177532 100644 --- a/base/poco/Foundation/include/Poco/Dynamic/VarHolder.h +++ b/base/poco/Foundation/include/Poco/Dynamic/VarHolder.h @@ -135,47 +135,47 @@ public: /// (typeid) for the stored content. virtual void convert(Int8& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(Int16& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(Int32& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(Int64& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(UInt8& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(UInt16& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(UInt32& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(UInt64& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(DateTime& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(LocalDateTime& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(Timestamp& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. #ifndef POCO_LONG_IS_64_BIT @@ -189,92 +189,92 @@ public: #else virtual void convert(long long& val) const; - /// Throws BadCastException. Must be overriden in a type - /// specialization in order to suport the conversion. + /// Throws BadCastException. Must be overridden in a type + /// specialization in order to support the conversion. virtual void convert(unsigned long long & val) const; - /// Throws BadCastException. Must be overriden in a type - /// specialization in order to suport the conversion. + /// Throws BadCastException. Must be overridden in a type + /// specialization in order to support the conversion. #endif virtual void convert(bool& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(float& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(double& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(char& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(std::string& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual void convert(Poco::UTF16String& val) const; - /// Throws BadCastException. Must be overriden in a type + /// Throws BadCastException. Must be overridden in a type /// specialization in order to support the conversion. virtual bool isArray() const; /// Returns true. virtual bool isVector() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isList() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isDeque() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isStruct() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isInteger() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isSigned() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isNumeric() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isBoolean() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isString() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isDate() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isTime() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual bool isDateTime() const; - /// Returns false. Must be properly overriden in a type + /// Returns false. Must be properly overridden in a type /// specialization in order to support the diagnostic. virtual std::size_t size() const; - /// Returns 1 iff Var is not empty or this function overriden. + /// Returns 1 iff Var is not empty or this function overridden. protected: VarHolder(); diff --git a/base/poco/Foundation/include/Poco/FIFOBufferStream.h b/base/poco/Foundation/include/Poco/FIFOBufferStream.h index 6c1e1eb7872..2cf2e21832f 100644 --- a/base/poco/Foundation/include/Poco/FIFOBufferStream.h +++ b/base/poco/Foundation/include/Poco/FIFOBufferStream.h @@ -30,7 +30,7 @@ namespace Poco { class Foundation_API FIFOBufferStreamBuf: public BufferedBidirectionalStreamBuf /// This is the streambuf class used for reading from and writing to a FIFOBuffer. - /// FIFOBuffer is enabled for emtpy/non-empty/full state transitions notifications. + /// FIFOBuffer is enabled for empty/non-empty/full state transitions notifications. { public: diff --git a/base/poco/Foundation/include/Poco/FileChannel.h b/base/poco/Foundation/include/Poco/FileChannel.h index 3068a215248..8065c939e4a 100644 --- a/base/poco/Foundation/include/Poco/FileChannel.h +++ b/base/poco/Foundation/include/Poco/FileChannel.h @@ -53,7 +53,7 @@ class Foundation_API FileChannel: public Channel /// /// The rotation strategy can be specified with the /// "rotation" property, which can take one of the - /// follwing values: + /// following values: /// /// * never: no log rotation /// * [day,][hh]:mm: the file is rotated on specified day/time diff --git a/base/poco/Foundation/include/Poco/FileStream.h b/base/poco/Foundation/include/Poco/FileStream.h index cea56c9476f..368bb5f63ef 100644 --- a/base/poco/Foundation/include/Poco/FileStream.h +++ b/base/poco/Foundation/include/Poco/FileStream.h @@ -122,7 +122,7 @@ class Foundation_API FileOutputStream: public FileIOS, public std::ostream { public: FileOutputStream(); - /// Creats an unopened FileOutputStream. + /// Creates an unopened FileOutputStream. FileOutputStream(const std::string& path, std::ios::openmode mode = std::ios::out | std::ios::trunc); /// Creates the FileOutputStream for the file given by path, using @@ -159,7 +159,7 @@ class Foundation_API FileStream: public FileIOS, public std::iostream { public: FileStream(); - /// Creats an unopened FileStream. + /// Creates an unopened FileStream. FileStream(const std::string& path, std::ios::openmode mode = std::ios::out | std::ios::in); /// Creates the FileStream for the file given by path, using diff --git a/base/poco/Foundation/include/Poco/Foundation.h b/base/poco/Foundation/include/Poco/Foundation.h index 750c7775e7c..98f3d254c89 100644 --- a/base/poco/Foundation/include/Poco/Foundation.h +++ b/base/poco/Foundation/include/Poco/Foundation.h @@ -41,7 +41,7 @@ // from a DLL simpler. All files within this DLL are compiled with the Foundation_EXPORTS // symbol defined on the command line. this symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// Foundation_API functions as being imported from a DLL, wheras this DLL sees symbols +// Foundation_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if (defined(_WIN32) || defined(_WIN32_WCE)) && defined(POCO_DLL) diff --git a/base/poco/Foundation/include/Poco/HMACEngine.h b/base/poco/Foundation/include/Poco/HMACEngine.h index 0593e7d2c17..c02e45f441d 100644 --- a/base/poco/Foundation/include/Poco/HMACEngine.h +++ b/base/poco/Foundation/include/Poco/HMACEngine.h @@ -28,7 +28,7 @@ namespace Poco { template class HMACEngine: public DigestEngine - /// This class implementes the HMAC message + /// This class implements the HMAC message /// authentication code algorithm, as specified /// in RFC 2104. The underlying DigestEngine /// (MD5Engine, SHA1Engine, etc.) must be given as diff --git a/base/poco/Foundation/include/Poco/LocalDateTime.h b/base/poco/Foundation/include/Poco/LocalDateTime.h index 08c90b6582b..270d3ea357f 100644 --- a/base/poco/Foundation/include/Poco/LocalDateTime.h +++ b/base/poco/Foundation/include/Poco/LocalDateTime.h @@ -37,7 +37,7 @@ class Foundation_API LocalDateTime /// i.e. UTC = local time - time zone differential. /// /// Although LocalDateTime supports relational and arithmetic - /// operators, all date/time comparisons and date/time arithmetics + /// operators, all date/time comparisons and date/time arithmetic /// should be done in UTC, using the DateTime or Timestamp /// class for better performance. The relational operators /// normalize the dates/times involved to UTC before carrying out @@ -173,7 +173,7 @@ public: /// There will be no week 0 in 2007. int day() const; - /// Returns the day witin the month (1 to 31). + /// Returns the day within the month (1 to 31). int dayOfWeek() const; /// Returns the weekday (0 to 6, where diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index 8dbf8327c88..2a817d41ed0 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -419,22 +419,22 @@ public: /// Returns true if the log level is at least PRIO_TEST. static std::string format(const std::string& fmt, const std::string& arg); - /// Replaces all occurences of $0 in fmt with the string given in arg and + /// Replaces all occurrences of $0 in fmt with the string given in arg and /// returns the result. To include a dollar sign in the result string, /// specify two dollar signs ($$) in the format string. static std::string format(const std::string& fmt, const std::string& arg0, const std::string& arg1); - /// Replaces all occurences of $ in fmt with the string given in arg and + /// Replaces all occurrences of $ in fmt with the string given in arg and /// returns the result. To include a dollar sign in the result string, /// specify two dollar signs ($$) in the format string. static std::string format(const std::string& fmt, const std::string& arg0, const std::string& arg1, const std::string& arg2); - /// Replaces all occurences of $ in fmt with the string given in arg and + /// Replaces all occurrences of $ in fmt with the string given in arg and /// returns the result. To include a dollar sign in the result string, /// specify two dollar signs ($$) in the format string. static std::string format(const std::string& fmt, const std::string& arg0, const std::string& arg1, const std::string& arg2, const std::string& arg3); - /// Replaces all occurences of $ in fmt with the string given in arg and + /// Replaces all occurrences of $ in fmt with the string given in arg and /// returns the result. To include a dollar sign in the result string, /// specify two dollar signs ($$) in the format string. @@ -480,7 +480,7 @@ public: static Logger* has(const std::string& name); /// Returns a pointer to the Logger with the given name if it - /// exists, or a null pointer otherwse. + /// exists, or a null pointer otherwise. static void destroy(const std::string& name); /// Destroys the logger with the specified name. Does nothing diff --git a/base/poco/Foundation/include/Poco/MD4Engine.h b/base/poco/Foundation/include/Poco/MD4Engine.h index 644575e06c9..c243e7851ac 100644 --- a/base/poco/Foundation/include/Poco/MD4Engine.h +++ b/base/poco/Foundation/include/Poco/MD4Engine.h @@ -49,7 +49,7 @@ namespace Poco { class Foundation_API MD4Engine: public DigestEngine - /// This class implementes the MD4 message digest algorithm, + /// This class implements the MD4 message digest algorithm, /// described in RFC 1320. { public: diff --git a/base/poco/Foundation/include/Poco/MD5Engine.h b/base/poco/Foundation/include/Poco/MD5Engine.h index 7835ff61298..ec3e80df759 100644 --- a/base/poco/Foundation/include/Poco/MD5Engine.h +++ b/base/poco/Foundation/include/Poco/MD5Engine.h @@ -49,7 +49,7 @@ namespace Poco { class Foundation_API MD5Engine: public DigestEngine - /// This class implementes the MD5 message digest algorithm, + /// This class implements the MD5 message digest algorithm, /// described in RFC 1321. { public: diff --git a/base/poco/Foundation/include/Poco/Nullable.h b/base/poco/Foundation/include/Poco/Nullable.h index d113256cf2a..0e21933a165 100644 --- a/base/poco/Foundation/include/Poco/Nullable.h +++ b/base/poco/Foundation/include/Poco/Nullable.h @@ -182,7 +182,7 @@ public: bool operator < (const Nullable& other) const /// Compares two Nullable objects. Return true if this object's - /// value is smaler than the other object's value. + /// value is smaller than the other object's value. /// Null value is smaller than a non-null value. { if (_isNull && other._isNull) return false; diff --git a/base/poco/Foundation/include/Poco/PBKDF2Engine.h b/base/poco/Foundation/include/Poco/PBKDF2Engine.h index 357d482afe4..2f30a6f08e0 100644 --- a/base/poco/Foundation/include/Poco/PBKDF2Engine.h +++ b/base/poco/Foundation/include/Poco/PBKDF2Engine.h @@ -29,7 +29,7 @@ namespace Poco { template class PBKDF2Engine: public DigestEngine - /// This class implementes the Password-Based Key Derivation Function 2, + /// This class implements the Password-Based Key Derivation Function 2, /// as specified in RFC 2898. The underlying DigestEngine (HMACEngine, etc.), /// which must accept the passphrase as constructor argument (std::string), /// must be given as template argument. diff --git a/base/poco/Foundation/include/Poco/Path.h b/base/poco/Foundation/include/Poco/Path.h index c50c1a1a862..6c251e8574c 100644 --- a/base/poco/Foundation/include/Poco/Path.h +++ b/base/poco/Foundation/include/Poco/Path.h @@ -168,7 +168,7 @@ public: /// Appends the given path. Path& resolve(const Path& path); - /// Resolves the given path agains the current one. + /// Resolves the given path against the current one. /// /// If the given path is absolute, it replaces the current one. /// Otherwise, the relative path is appended to the current path. diff --git a/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h b/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h index c0ce2285eee..79b957d129d 100644 --- a/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h +++ b/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h @@ -226,7 +226,7 @@ private: // -// friend comparsion operators +// friend comparison operators // template inline bool operator ==(const RecursiveDirectoryIterator& a, const RecursiveDirectoryIterator& b) diff --git a/base/poco/Foundation/include/Poco/RegularExpression.h b/base/poco/Foundation/include/Poco/RegularExpression.h index 638b99e11b1..74f2c7f95cc 100644 --- a/base/poco/Foundation/include/Poco/RegularExpression.h +++ b/base/poco/Foundation/include/Poco/RegularExpression.h @@ -69,7 +69,7 @@ public: RE_NEWLINE_CRLF = 0x00300000, /// assume newline is CRLF ("\r\n") [ctor] RE_NEWLINE_ANY = 0x00400000, /// assume newline is any valid Unicode newline character [ctor] RE_NEWLINE_ANYCRLF = 0x00500000, /// assume newline is any of CR, LF, CRLF [ctor] - RE_GLOBAL = 0x10000000, /// replace all occurences (/g) [subst] + RE_GLOBAL = 0x10000000, /// replace all occurrences (/g) [subst] RE_NO_VARS = 0x20000000 /// treat dollar in replacement string as ordinary character [subst] }; diff --git a/base/poco/Foundation/include/Poco/SHA1Engine.h b/base/poco/Foundation/include/Poco/SHA1Engine.h index 81c8a4d9ea2..9aa2b45e381 100644 --- a/base/poco/Foundation/include/Poco/SHA1Engine.h +++ b/base/poco/Foundation/include/Poco/SHA1Engine.h @@ -32,7 +32,7 @@ namespace Poco { class Foundation_API SHA1Engine: public DigestEngine - /// This class implementes the SHA-1 message digest algorithm. + /// This class implements the SHA-1 message digest algorithm. /// (FIPS 180-1, see http://www.itl.nist.gov/fipspubs/fip180-1.htm) { public: diff --git a/base/poco/Foundation/include/Poco/SharedMemory_WIN32.h b/base/poco/Foundation/include/Poco/SharedMemory_WIN32.h index bd48b6932a8..c891305491c 100644 --- a/base/poco/Foundation/include/Poco/SharedMemory_WIN32.h +++ b/base/poco/Foundation/include/Poco/SharedMemory_WIN32.h @@ -50,7 +50,7 @@ public: /// will generally ignore the hint. char* begin() const; - /// Returns the beginn address of the SharedMemory segment. Will be null for illegal segments. + /// Returns the begin address of the SharedMemory segment. Will be null for illegal segments. char* end() const; /// Points past the last byte of the end address of the SharedMemory segment. Will be null for illegal segments. diff --git a/base/poco/Foundation/include/Poco/SimpleHashTable.h b/base/poco/Foundation/include/Poco/SimpleHashTable.h index c149cd6b7e8..eb1a0161d19 100644 --- a/base/poco/Foundation/include/Poco/SimpleHashTable.h +++ b/base/poco/Foundation/include/Poco/SimpleHashTable.h @@ -36,7 +36,7 @@ template > class SimpleHashTable /// A SimpleHashTable stores a key value pair that can be looked up via a hashed key. /// - /// In comparision to a HashTable, this class handles collisions by sequentially searching the next + /// In comparison to a HashTable, this class handles collisions by sequentially searching the next /// free location. This also means that the maximum size of this table is limited, i.e. if the hash table /// is full, it will throw an exception and that this class does not support remove operations. /// On the plus side it is faster than the HashTable. diff --git a/base/poco/Foundation/include/Poco/StringTokenizer.h b/base/poco/Foundation/include/Poco/StringTokenizer.h index e4bfd469f23..a7ddc91d7fb 100644 --- a/base/poco/Foundation/include/Poco/StringTokenizer.h +++ b/base/poco/Foundation/include/Poco/StringTokenizer.h @@ -67,7 +67,7 @@ public: /// Returns true if token exists, false otherwise. std::string::size_type find(const std::string& token, std::string::size_type pos = 0) const; - /// Returns the index of the first occurence of the token + /// Returns the index of the first occurrence of the token /// starting at position pos. /// Throws a NotFoundException if the token is not found. diff --git a/base/poco/Foundation/include/Poco/TextEncoding.h b/base/poco/Foundation/include/Poco/TextEncoding.h index 40d64b7c933..a948b37f646 100644 --- a/base/poco/Foundation/include/Poco/TextEncoding.h +++ b/base/poco/Foundation/include/Poco/TextEncoding.h @@ -69,7 +69,7 @@ public: /// Returns true if the given name is one of the names of this encoding. /// For example, the "ISO-8859-1" encoding is also known as "Latin-1". /// - /// Encoding name comparisions are case insensitive. + /// Encoding name comparisons are case insensitive. virtual const CharacterMap& characterMap() const = 0; /// Returns the CharacterMap for the encoding. @@ -108,7 +108,7 @@ public: /// The default implementation returns (int) bytes[0]. virtual int sequenceLength(const unsigned char* bytes, int length) const; - /// The sequenceLength function is used to get the lenth of the sequence pointed + /// The sequenceLength function is used to get the length of the sequence pointed /// by bytes. The length parameter should be greater or equal to the length of /// the sequence. /// diff --git a/base/poco/Foundation/include/Poco/Thread.h b/base/poco/Foundation/include/Poco/Thread.h index bfd6f031fd2..0c3591239f1 100644 --- a/base/poco/Foundation/include/Poco/Thread.h +++ b/base/poco/Foundation/include/Poco/Thread.h @@ -108,7 +108,7 @@ public: void setOSPriority(int prio, int policy = POLICY_DEFAULT); /// Sets the thread's priority, using an operating system specific /// priority value. Use getMinOSPriority() and getMaxOSPriority() to - /// obtain mininum and maximum priority values. Additionally, + /// obtain minimum and maximum priority values. Additionally, /// a scheduling policy can be specified. The policy is currently /// only used on POSIX platforms where the values SCHED_OTHER (default), /// SCHED_FIFO and SCHED_RR are supported. diff --git a/base/poco/Foundation/include/Poco/Timestamp.h b/base/poco/Foundation/include/Poco/Timestamp.h index 3fd5d9738e8..c0d69fc4856 100644 --- a/base/poco/Foundation/include/Poco/Timestamp.h +++ b/base/poco/Foundation/include/Poco/Timestamp.h @@ -31,7 +31,7 @@ class Foundation_API Timestamp /// A Timestamp stores a monotonic* time value /// with (theoretical) microseconds resolution. /// Timestamps can be compared with each other - /// and simple arithmetics are supported. + /// and simple arithmetic are supported. /// /// [*] Note that Timestamp values are only monotonic as /// long as the systems's clock is monotonic as well diff --git a/base/poco/Foundation/include/Poco/TypeList.h b/base/poco/Foundation/include/Poco/TypeList.h index 352696f8e01..1492223e606 100644 --- a/base/poco/Foundation/include/Poco/TypeList.h +++ b/base/poco/Foundation/include/Poco/TypeList.h @@ -320,7 +320,7 @@ struct TypeAppender, T> template struct TypeOneEraser; - /// TypeOneEraser erases the first occurence of the type T in Head. + /// TypeOneEraser erases the first occurrence of the type T in Head. /// Usage: /// /// typedef TypeListType::HeadType Type3; @@ -352,7 +352,7 @@ struct TypeOneEraser, T> template struct TypeAllEraser; - /// TypeAllEraser erases all the occurences of the type T in Head. + /// TypeAllEraser erases all the occurrence of the type T in Head. /// Usage: /// /// typedef TypeListType::HeadType Type4; @@ -384,7 +384,7 @@ struct TypeAllEraser, T> template struct TypeDuplicateEraser; - /// TypeDuplicateEraser erases all but the first occurence of the type T in Head. + /// TypeDuplicateEraser erases all but the first occurrence of the type T in Head. /// Usage: /// /// typedef TypeListType::HeadType Type4; @@ -413,7 +413,7 @@ public: template struct TypeOneReplacer; - /// TypeOneReplacer replaces the first occurence + /// TypeOneReplacer replaces the first occurrence /// of the type T in Head with type R. /// Usage: /// @@ -446,7 +446,7 @@ struct TypeOneReplacer, T, R> template struct TypeAllReplacer; - /// TypeAllReplacer replaces all the occurences + /// TypeAllReplacer replaces all the occurrence /// of the type T in Head with type R. /// Usage: /// diff --git a/base/poco/Foundation/include/Poco/UniqueAccessExpireLRUCache.h b/base/poco/Foundation/include/Poco/UniqueAccessExpireLRUCache.h index 6e7a366670f..85650eb0472 100644 --- a/base/poco/Foundation/include/Poco/UniqueAccessExpireLRUCache.h +++ b/base/poco/Foundation/include/Poco/UniqueAccessExpireLRUCache.h @@ -35,7 +35,7 @@ template < > class UniqueAccessExpireLRUCache: public AbstractCache, TMutex, TEventMutex> /// A UniqueAccessExpireLRUCache combines LRU caching and time based per entry expire caching. - /// One can define for each cache entry a seperate timepoint + /// One can define for each cache entry a separate timepoint /// but also limit the size of the cache (per default: 1024). /// Each TValue object must thus offer the following method: /// diff --git a/base/poco/Foundation/include/Poco/UniqueExpireLRUCache.h b/base/poco/Foundation/include/Poco/UniqueExpireLRUCache.h index c59bf0f7a41..5180040469b 100644 --- a/base/poco/Foundation/include/Poco/UniqueExpireLRUCache.h +++ b/base/poco/Foundation/include/Poco/UniqueExpireLRUCache.h @@ -35,7 +35,7 @@ template < > class UniqueExpireLRUCache: public AbstractCache, TMutex, TEventMutex> /// A UniqueExpireLRUCache combines LRU caching and time based per entry expire caching. - /// One can define for each cache entry a seperate timepoint + /// One can define for each cache entry a separate timepoint /// but also limit the size of the cache (per default: 1024). /// Each TValue object must thus offer the following method: /// diff --git a/base/poco/Foundation/src/DirectoryWatcher.cpp b/base/poco/Foundation/src/DirectoryWatcher.cpp index f54262ee23e..68aef99d29f 100644 --- a/base/poco/Foundation/src/DirectoryWatcher.cpp +++ b/base/poco/Foundation/src/DirectoryWatcher.cpp @@ -317,38 +317,38 @@ public: { while (n > 0) { - struct inotify_event* pEvent = reinterpret_cast(buffer.begin() + i); + struct inotify_event* event = reinterpret_cast(buffer.begin() + i); - if (pEvent->len > 0) + if (event->len > 0) { if (!owner().eventsSuspended()) { Poco::Path p(owner().directory().path()); p.makeDirectory(); - p.setFileName(pEvent->name); + p.setFileName(event->name); Poco::File f(p.toString()); - if ((pEvent->mask & IN_CREATE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED)) + if ((event->mask & IN_CREATE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_ADDED)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_ADDED); owner().itemAdded(&owner(), ev); } - if ((pEvent->mask & IN_DELETE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_REMOVED)) + if ((event->mask & IN_DELETE) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_REMOVED)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_REMOVED); owner().itemRemoved(&owner(), ev); } - if ((pEvent->mask & IN_MODIFY) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED)) + if ((event->mask & IN_MODIFY) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MODIFIED)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MODIFIED); owner().itemModified(&owner(), ev); } - if ((pEvent->mask & IN_MOVED_FROM) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_FROM)) + if ((event->mask & IN_MOVED_FROM) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_FROM)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MOVED_FROM); owner().itemMovedFrom(&owner(), ev); } - if ((pEvent->mask & IN_MOVED_TO) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_TO)) + if ((event->mask & IN_MOVED_TO) && (owner().eventMask() & DirectoryWatcher::DW_ITEM_MOVED_TO)) { DirectoryWatcher::DirectoryEvent ev(f, DirectoryWatcher::DW_ITEM_MOVED_TO); owner().itemMovedTo(&owner(), ev); @@ -356,8 +356,8 @@ public: } } - i += sizeof(inotify_event) + pEvent->len; - n -= sizeof(inotify_event) + pEvent->len; + i += sizeof(inotify_event) + event->len; + n -= sizeof(inotify_event) + event->len; } } } diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 445ba52ae1a..3d5de585b4f 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -470,7 +470,7 @@ public: namespace { - static AutoLoggerShutdown als; + static AutoLoggerShutdown auto_logger_shutdown; } diff --git a/base/poco/Foundation/src/Random.cpp b/base/poco/Foundation/src/Random.cpp index 43dd31eb166..10bf730374e 100644 --- a/base/poco/Foundation/src/Random.cpp +++ b/base/poco/Foundation/src/Random.cpp @@ -169,7 +169,7 @@ Random::~Random() /* * Compute x = (7^5 * x) mod (2^31 - 1) - * wihout overflowing 31 bits: + * without overflowing 31 bits: * (2^31 - 1) = 127773 * (7^5) + 2836 * From "Random number generators: good ones are hard to find", * Park and Miller, Communications of the ACM, vol. 31, no. 10, @@ -261,7 +261,7 @@ void Random::seed() * * Note: The Sparc platform requires that arg_state begin on an int * word boundary; otherwise a bus error will occur. Even so, lint will - * complain about mis-alignment, but you should disregard these messages. + * complain about misalignment, but you should disregard these messages. */ void Random::initState(UInt32 s, char* argState, Int32 n) { diff --git a/base/poco/Foundation/src/bignum-dtoa.cc b/base/poco/Foundation/src/bignum-dtoa.cc index f1ad7a5ae8d..5a44adfccf3 100644 --- a/base/poco/Foundation/src/bignum-dtoa.cc +++ b/base/poco/Foundation/src/bignum-dtoa.cc @@ -276,7 +276,7 @@ static void GenerateShortestDigits(Bignum* numerator, Bignum* denominator, // Let v = numerator / denominator < 10. // Then we generate 'count' digits of d = x.xxxxx... (without the decimal point) -// from left to right. Once 'count' digits have been produced we decide wether +// from left to right. Once 'count' digits have been produced we decide whether // to round up or down. Remainders of exactly .5 round upwards. Numbers such // as 9.999999 propagate a carry all the way, and change the // exponent (decimal_point), when rounding upwards. @@ -548,7 +548,7 @@ static void InitialScaledStartValuesNegativeExponentNegativePower( // // Let ep == estimated_power, then the returned values will satisfy: // v / 10^ep = numerator / denominator. -// v's boundarys m- and m+: +// v's boundaries m- and m+: // m- / 10^ep == v / 10^ep - delta_minus / denominator // m+ / 10^ep == v / 10^ep + delta_plus / denominator // Or in other words: diff --git a/base/poco/Foundation/src/deflate.c b/base/poco/Foundation/src/deflate.c index 1ec761448de..0a80edd92d8 100644 --- a/base/poco/Foundation/src/deflate.c +++ b/base/poco/Foundation/src/deflate.c @@ -1096,7 +1096,7 @@ int ZEXPORT deflateEnd (strm) /* ========================================================================= * Copy the source state to the destination state. - * To simplify the source, this is not supported for 16-bit MSDOS (which + * To simplify the source, this is not supported for 16-bit MS-DOS (which * doesn't have enough memory anyway to duplicate compression states). */ int ZEXPORT deflateCopy (dest, source) @@ -1136,7 +1136,7 @@ int ZEXPORT deflateCopy (dest, source) deflateEnd (dest); return Z_MEM_ERROR; } - /* following zmemcpy do not work for 16-bit MSDOS */ + /* following zmemcpy do not work for 16-bit MS-DOS */ zmemcpy(ds->window, ss->window, ds->w_size * 2 * sizeof(Byte)); zmemcpy((voidpf)ds->prev, (voidpf)ss->prev, ds->w_size * sizeof(Pos)); zmemcpy((voidpf)ds->head, (voidpf)ss->head, ds->hash_size * sizeof(Pos)); diff --git a/base/poco/Foundation/src/deflate.h b/base/poco/Foundation/src/deflate.h index 23ecdd312bc..477af65aab9 100644 --- a/base/poco/Foundation/src/deflate.h +++ b/base/poco/Foundation/src/deflate.h @@ -122,7 +122,7 @@ typedef struct internal_state { * bytes. With this organization, matches are limited to a distance of * wSize-MAX_MATCH bytes, but this ensures that IO is always * performed with a length multiple of the block size. Also, it limits - * the window size to 64K, which is quite useful on MSDOS. + * the window size to 64K, which is quite useful on MS-DOS. * To do: use the user input buffer as sliding window. */ diff --git a/base/poco/Foundation/src/double-conversion.h b/base/poco/Foundation/src/double-conversion.h index 1c3387d4f84..564076186b6 100644 --- a/base/poco/Foundation/src/double-conversion.h +++ b/base/poco/Foundation/src/double-conversion.h @@ -96,7 +96,7 @@ class DoubleToStringConverter { // Example with max_leading_padding_zeroes_in_precision_mode = 6. // ToPrecision(0.0000012345, 2) -> "0.0000012" // ToPrecision(0.00000012345, 2) -> "1.2e-7" - // Similarily the converter may add up to + // Similarly the converter may add up to // max_trailing_padding_zeroes_in_precision_mode in precision mode to avoid // returning an exponential representation. A zero added by the // EMIT_TRAILING_ZERO_AFTER_POINT flag is counted for this limit. @@ -138,7 +138,7 @@ class DoubleToStringConverter { // Example with decimal_in_shortest_low = -6, // decimal_in_shortest_high = 21, // EMIT_POSITIVE_EXPONENT_SIGN activated, and - // EMIT_TRAILING_DECIMAL_POINT deactived: + // EMIT_TRAILING_DECIMAL_POINT deactivated: // ToShortest(0.000001) -> "0.000001" // ToShortest(0.0000001) -> "1e-7" // ToShortest(111111111111111111111.0) -> "111111111111111110000" @@ -242,7 +242,7 @@ class DoubleToStringConverter { // Example with max_leading_padding_zeroes_in_precision_mode = 6. // ToPrecision(0.0000012345, 2) -> "0.0000012" // ToPrecision(0.00000012345, 2) -> "1.2e-7" - // Similarily the converter may add up to + // Similarly the converter may add up to // max_trailing_padding_zeroes_in_precision_mode in precision mode to avoid // returning an exponential representation. A zero added by the // EMIT_TRAILING_ZERO_AFTER_POINT flag is counted for this limit. diff --git a/base/poco/Foundation/src/fixed-dtoa.cc b/base/poco/Foundation/src/fixed-dtoa.cc index aef65fdc211..390e823d95e 100644 --- a/base/poco/Foundation/src/fixed-dtoa.cc +++ b/base/poco/Foundation/src/fixed-dtoa.cc @@ -394,7 +394,7 @@ bool FastFixedDtoa(double v, TrimZeros(buffer, length, decimal_point); buffer[*length] = '\0'; if ((*length) == 0) { - // The string is empty and the decimal_point thus has no importance. Mimick + // The string is empty and the decimal_point thus has no importance. Mimic // Gay's dtoa and and set it to -fractional_count. *decimal_point = -fractional_count; } diff --git a/base/poco/Foundation/src/gzguts.h b/base/poco/Foundation/src/gzguts.h index 75fa55ffc83..7085622a9d7 100644 --- a/base/poco/Foundation/src/gzguts.h +++ b/base/poco/Foundation/src/gzguts.h @@ -67,18 +67,7 @@ # endif #endif -#if defined(MSDOS) && defined(__BORLANDC__) && (BORLANDC > 0x410) -# ifndef HAVE_VSNPRINTF -# define HAVE_VSNPRINTF -# endif -#endif - #ifndef HAVE_VSNPRINTF -# ifdef MSDOS -/* vsnprintf may exist on some MS-DOS compilers (DJGPP?), - but for now we just assume it doesn't. */ -# define NO_vsnprintf -# endif # ifdef __TURBOC__ # define NO_vsnprintf # endif diff --git a/base/poco/Foundation/src/inftrees.h b/base/poco/Foundation/src/inftrees.h index baa53a0b1a1..f53665311c1 100644 --- a/base/poco/Foundation/src/inftrees.h +++ b/base/poco/Foundation/src/inftrees.h @@ -38,7 +38,7 @@ typedef struct { /* Maximum size of the dynamic table. The maximum number of code structures is 1444, which is the sum of 852 for literal/length codes and 592 for distance codes. These values were found by exhaustive searches using the program - examples/enough.c found in the zlib distribtution. The arguments to that + examples/enough.c found in the zlib distribution. The arguments to that program are the number of symbols, the initial root table size, and the maximum bit length of a code. "enough 286 9 15" for literal/length codes returns returns 852, and "enough 30 6 15" for distance codes returns 592. diff --git a/base/poco/Foundation/src/pcre_compile.c b/base/poco/Foundation/src/pcre_compile.c index b4b78b418f3..3a6fafe8d56 100644 --- a/base/poco/Foundation/src/pcre_compile.c +++ b/base/poco/Foundation/src/pcre_compile.c @@ -5839,7 +5839,7 @@ for (;; ptr++) else #endif /* SUPPORT_UTF */ - /* Handle the case of a single charater - either with no UTF support, or + /* Handle the case of a single character - either with no UTF support, or with UTF disabled, or for a single character UTF character. */ { c = code[-1]; diff --git a/base/poco/Foundation/src/pcre_exec.c b/base/poco/Foundation/src/pcre_exec.c index c60aad957fd..b07ad52cb81 100644 --- a/base/poco/Foundation/src/pcre_exec.c +++ b/base/poco/Foundation/src/pcre_exec.c @@ -689,7 +689,7 @@ the alternative names that are used. */ #define foc number #define save_mark data -/* These statements are here to stop the compiler complaining about unitialized +/* These statements are here to stop the compiler complaining about uninitialized variables. */ #ifdef SUPPORT_UCP @@ -1041,7 +1041,7 @@ for (;;) the result of a recursive call to match() whatever happened so it was possible to reduce stack usage by turning this into a tail recursion, except in the case of a possibly empty group. However, now that there is - the possiblity of (*THEN) occurring in the final alternative, this + the possibility of (*THEN) occurring in the final alternative, this optimization is no longer always possible. We can optimize if we know there are no (*THEN)s in the pattern; at present @@ -7016,7 +7016,7 @@ for(;;) /* If we have just passed a CR and we are now at a LF, and the pattern does not contain any explicit matches for \r or \n, and the newline option is CRLF or ANY or ANYCRLF, advance the match position by one more character. In - normal matching start_match will aways be greater than the first position at + normal matching start_match will always be greater than the first position at this stage, but a failed *SKIP can cause a return at the same point, which is why the first test exists. */ diff --git a/base/poco/Foundation/src/pcre_internal.h b/base/poco/Foundation/src/pcre_internal.h index 273d4db8a98..60f50450dae 100644 --- a/base/poco/Foundation/src/pcre_internal.h +++ b/base/poco/Foundation/src/pcre_internal.h @@ -2188,7 +2188,7 @@ regex. The lengths are used when searching for specific things, and also in the debugging printing of a compiled regex. We use a macro so that it can be defined close to the definitions of the opcodes themselves. -As things have been extended, some of these are no longer fixed lenths, but are +As things have been extended, some of these are no longer fixed lengths, but are minima instead. For example, the length of a single-character repeat may vary in UTF-8 mode. The code that uses this table must know about such things. */ diff --git a/base/poco/Foundation/src/pcre_jit_compile.c b/base/poco/Foundation/src/pcre_jit_compile.c index 0f1ea580ade..2a92feabc98 100644 --- a/base/poco/Foundation/src/pcre_jit_compile.c +++ b/base/poco/Foundation/src/pcre_jit_compile.c @@ -101,7 +101,7 @@ vertical (sub-expression) (See struct backtrack_common for more details). The condition checkers are boolean (true/false) checkers. Machine code is generated for the checker itself and for the actions depending on the result of the checker. The 'true' case is called as the matching path (expected path), and the other is called as -the 'backtrack' path. Branch instructions are expesive for all CPUs, so we avoid taken +the 'backtrack' path. Branch instructions are expensive for all CPUs, so we avoid taken branches on the matching path. Greedy star operator (*) : @@ -112,7 +112,7 @@ branches on the matching path. Backtrack path: match is required. The following example shows how the code generated for a capturing bracket -with two alternatives. Let A, B, C, D are arbirary regular expressions, and +with two alternatives. Let A, B, C, D are arbitrary regular expressions, and we have the following regular expression: A(B|C)D @@ -242,7 +242,7 @@ typedef struct assert_backtrack { typedef struct bracket_backtrack { backtrack_common common; - /* Where to coninue if an alternative is successfully matched. */ + /* Where to continue if an alternative is successfully matched. */ struct sljit_label *alternative_matchingpath; /* For rmin and rmax iterators. */ struct sljit_label *recursive_matchingpath; @@ -6607,7 +6607,7 @@ switch(type) if (common->mode == JIT_PARTIAL_HARD_COMPILE) { jump[0] = CMP(SLJIT_LESS, STR_PTR, 0, STR_END, 0); - /* Since we successfully read a char above, partial matching must occure. */ + /* Since we successfully read a char above, partial matching must occur. */ check_partial(common, TRUE); JUMPHERE(jump[0]); } @@ -7358,7 +7358,7 @@ OP1(SLJIT_MOV, TMP1, 0, ARGUMENTS, 0); OP1(SLJIT_MOV_S32, SLJIT_MEM1(STACK_TOP), CALLOUT_ARG_OFFSET(callout_number), SLJIT_IMM, cc[1]); OP1(SLJIT_MOV_S32, SLJIT_MEM1(STACK_TOP), CALLOUT_ARG_OFFSET(capture_last), TMP2, 0); -/* These pointer sized fields temporarly stores internal variables. */ +/* These pointer sized fields temporarily stores internal variables. */ OP1(SLJIT_MOV, TMP2, 0, SLJIT_MEM1(SLJIT_SP), OVECTOR(0)); OP1(SLJIT_MOV, SLJIT_MEM1(STACK_TOP), CALLOUT_ARG_OFFSET(offset_vector), STR_PTR, 0); OP1(SLJIT_MOV, SLJIT_MEM1(STACK_TOP), CALLOUT_ARG_OFFSET(subject), TMP2, 0); diff --git a/base/poco/Foundation/src/strtod.cc b/base/poco/Foundation/src/strtod.cc index 68d02171911..e74b981f17c 100644 --- a/base/poco/Foundation/src/strtod.cc +++ b/base/poco/Foundation/src/strtod.cc @@ -498,7 +498,7 @@ float Strtof(Vector buffer, int exponent) { // low-precision (3 digits): // when read from input: 123 // when rounded from high precision: 124. - // To do this we simply look at the neigbors of the correct result and see + // To do this we simply look at the neighbors of the correct result and see // if they would round to the same float. If the guess is not correct we have // to look at four values (since two different doubles could be the correct // double). diff --git a/base/poco/Foundation/src/trees.c b/base/poco/Foundation/src/trees.c index 50cf4b4571c..b9d998f1b32 100644 --- a/base/poco/Foundation/src/trees.c +++ b/base/poco/Foundation/src/trees.c @@ -312,7 +312,7 @@ local void tr_static_init() } /* =========================================================================== - * Genererate the file trees.h describing the static trees. + * Generate the file trees.h describing the static trees. */ #ifdef GEN_TREES_H # ifndef ZLIB_DEBUG diff --git a/base/poco/Foundation/src/zutil.c b/base/poco/Foundation/src/zutil.c index 083c9f58c7b..14a7bebb1b3 100644 --- a/base/poco/Foundation/src/zutil.c +++ b/base/poco/Foundation/src/zutil.c @@ -212,7 +212,7 @@ typedef struct ptr_table_s { local ptr_table table[MAX_PTR]; /* This table is used to remember the original form of pointers * to large buffers (64K). Such pointers are normalized with a zero offset. - * Since MSDOS is not a preemptive multitasking OS, this table is not + * Since MS-DOS is not a preemptive multitasking OS, this table is not * protected from concurrent access. This hack doesn't work anyway on * a protected system like OS/2. Use Microsoft C instead. */ diff --git a/base/poco/Foundation/src/zutil.h b/base/poco/Foundation/src/zutil.h index dd28aa3377b..3b6c55c8c92 100644 --- a/base/poco/Foundation/src/zutil.h +++ b/base/poco/Foundation/src/zutil.h @@ -80,23 +80,6 @@ extern z_const char * const z_errmsg[10]; /* indexed by 2-zlib_error */ /* target dependencies */ -#if defined(MSDOS) || (defined(WINDOWS) && !defined(WIN32)) -# define OS_CODE 0x00 -# ifndef Z_SOLO -# if defined(__TURBOC__) || defined(__BORLANDC__) -# if (__STDC__ == 1) && (defined(__LARGE__) || defined(__COMPACT__)) - /* Allow compilation with ANSI keywords only enabled */ - void _Cdecl farfree( void *block ); - void *_Cdecl farmalloc( unsigned long nbytes ); -# else -# include -# endif -# else /* MSC or DJGPP */ -# include -# endif -# endif -#endif - #ifdef AMIGA # define OS_CODE 0x01 #endif @@ -161,12 +144,6 @@ extern z_const char * const z_errmsg[10]; /* indexed by 2-zlib_error */ # endif #endif -#if defined(__BORLANDC__) && !defined(MSDOS) - #pragma warn -8004 - #pragma warn -8008 - #pragma warn -8066 -#endif - /* provide prototypes for these when building zlib without LFS */ #if !defined(_WIN32) && \ (!defined(_LARGEFILE64_SOURCE) || _LFS64_LARGEFILE-0 == 0) @@ -200,7 +177,7 @@ extern z_const char * const z_errmsg[10]; /* indexed by 2-zlib_error */ # define HAVE_MEMCPY #endif #ifdef HAVE_MEMCPY -# ifdef SMALL_MEDIUM /* MSDOS small or medium model */ +# ifdef SMALL_MEDIUM /* MS-DOS small or medium model */ # define zmemcpy _fmemcpy # define zmemcmp _fmemcmp # define zmemzero(dest, len) _fmemset(dest, 0, len) diff --git a/base/poco/JSON/include/Poco/JSON/Object.h b/base/poco/JSON/include/Poco/JSON/Object.h index 8812561a53a..f2e8062d343 100644 --- a/base/poco/JSON/include/Poco/JSON/Object.h +++ b/base/poco/JSON/include/Poco/JSON/Object.h @@ -90,7 +90,7 @@ public: /// Move constructor Object &operator =(Object &&other); - // Move asignment operator + // Move assignment operator #endif // POCO_ENABLE_CPP11 diff --git a/base/poco/Net/include/Poco/Net/ICMPPacketImpl.h b/base/poco/Net/include/Poco/Net/ICMPPacketImpl.h index da6bf464841..65a55c3fe58 100644 --- a/base/poco/Net/include/Poco/Net/ICMPPacketImpl.h +++ b/base/poco/Net/include/Poco/Net/ICMPPacketImpl.h @@ -43,7 +43,7 @@ public: virtual int packetSize() const = 0; /// Returns the total size of packet (ICMP header + data) in number of octets. - /// Must be overriden. + /// Must be overridden. virtual int maxPacketSize() const; /// Returns the maximum permitted size of packet in number of octets. @@ -62,14 +62,14 @@ public: /// Otherwise, it extracts the time value from the supplied buffer. /// /// Supplied buffer includes IP header, ICMP header and data. - /// Must be overriden. + /// Must be overridden. virtual bool validReplyID(unsigned char* buffer, int length) const = 0; /// Returns true if the extracted id is recognized /// (i.e. equals the process id). /// /// Supplied buffer includes IP header, ICMP header and data. - /// Must be overriden. + /// Must be overridden. virtual std::string errorDescription(Poco::UInt8* buffer, int length) = 0; /// Returns error description string. @@ -77,11 +77,11 @@ public: /// empty string is returned indicating the absence of error. /// /// Supplied buffer includes IP header, ICMP header and data. - /// Must be overriden. + /// Must be overridden. virtual std::string typeDescription(int typeId) = 0; /// Returns the description of the packet type. - /// Must be overriden. + /// Must be overridden. static const Poco::UInt16 MAX_PACKET_SIZE; static const Poco::UInt16 MAX_SEQ_VALUE; @@ -95,7 +95,7 @@ protected: virtual void initPacket() = 0; /// (Re)assembles the packet. - /// Must be overriden. + /// Must be overridden. Poco::UInt16 checksum(Poco::UInt16 *addr, Poco::Int32 len); /// Calculates the checksum for supplied buffer. diff --git a/base/poco/Net/include/Poco/Net/Net.h b/base/poco/Net/include/Poco/Net/Net.h index 4ec02035154..57bb9db4ab0 100644 --- a/base/poco/Net/include/Poco/Net/Net.h +++ b/base/poco/Net/include/Poco/Net/Net.h @@ -28,7 +28,7 @@ // from a DLL simpler. All files within this DLL are compiled with the Net_EXPORTS // symbol defined on the command line. this symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// Net_API functions as being imported from a DLL, wheras this DLL sees symbols +// Net_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if defined(_WIN32) && defined(POCO_DLL) diff --git a/base/poco/Net/include/Poco/Net/PartSource.h b/base/poco/Net/include/Poco/Net/PartSource.h index 06978d74cdf..443da4c64f4 100644 --- a/base/poco/Net/include/Poco/Net/PartSource.h +++ b/base/poco/Net/include/Poco/Net/PartSource.h @@ -41,7 +41,7 @@ public: virtual const std::string& filename() const; /// Returns the filename for the part or attachment. /// - /// May be overridded by subclasses. The default + /// May be overridden by subclasses. The default /// implementation returns an empty string. const std::string& mediaType() const; diff --git a/base/poco/Net/include/Poco/Net/SocketReactor.h b/base/poco/Net/include/Poco/Net/SocketReactor.h index 06c40bbd6ab..a503d2e6fb1 100644 --- a/base/poco/Net/include/Poco/Net/SocketReactor.h +++ b/base/poco/Net/include/Poco/Net/SocketReactor.h @@ -98,7 +98,7 @@ class Net_API SocketReactor: public Poco::Runnable /// Finally, when the SocketReactor is about to shut down (as a result /// of stop() being called), it dispatches a ShutdownNotification /// to all event handlers. This is done in the onShutdown() method - /// which can be overridded by subclasses to perform custom + /// which can be overridden by subclasses to perform custom /// shutdown processing. /// /// The SocketReactor is implemented so that it can diff --git a/base/poco/Net/src/IPAddressImpl.cpp b/base/poco/Net/src/IPAddressImpl.cpp index dcc63e6862f..f867acf8906 100644 --- a/base/poco/Net/src/IPAddressImpl.cpp +++ b/base/poco/Net/src/IPAddressImpl.cpp @@ -35,7 +35,7 @@ namespace { template unsigned maskBits(T val, unsigned size) /// Returns the length of the mask (number of bits set in val). - /// The val should be either all zeros or two contiguos areas of 1s and 0s. + /// The val should be either all zeros or two contiguous areas of 1s and 0s. /// The algorithm ignores invalid non-contiguous series of 1s and treats val /// as if all bits between MSb and last non-zero bit are set to 1. { diff --git a/base/poco/Net/src/MessageHeader.cpp b/base/poco/Net/src/MessageHeader.cpp index f1537daf908..0dbfa3e5cbd 100644 --- a/base/poco/Net/src/MessageHeader.cpp +++ b/base/poco/Net/src/MessageHeader.cpp @@ -286,7 +286,7 @@ void MessageHeader::decodeRFC2047(const std::string& ins, std::string& outs, con continue; } - // FIXME: check that we have enought chars- + // FIXME: check that we have enough chars- if (c == '=') { // The next two chars are hex representation of the complete byte. @@ -322,13 +322,13 @@ void MessageHeader::decodeRFC2047(const std::string& ins, std::string& outs, con } catch (...) { - // FIXME: Unsuported encoding... + // FIXME: Unsupported encoding... outs = tempout; } } else { - // Not conversion necesary. + // Not conversion necessary. outs = tempout; } } @@ -339,7 +339,7 @@ std::string MessageHeader::decodeWord(const std::string& text, const std::string std::string outs, tmp = text; do { std::string tmp2; - // find the begining of the next rfc2047 chunk + // find the beginning of the next rfc2047 chunk size_t pos = tmp.find("=?"); if (pos == std::string::npos) { // No more found, return @@ -347,7 +347,7 @@ std::string MessageHeader::decodeWord(const std::string& text, const std::string break; } - // check if there are standar text before the rfc2047 chunk, and if so, copy it. + // check if there are standard text before the rfc2047 chunk, and if so, copy it. if (pos > 0) { outs += tmp.substr(0, pos); } diff --git a/base/poco/Net/src/NetworkInterface.cpp b/base/poco/Net/src/NetworkInterface.cpp index ed520c32b60..7bd00c0ef2d 100644 --- a/base/poco/Net/src/NetworkInterface.cpp +++ b/base/poco/Net/src/NetworkInterface.cpp @@ -1204,7 +1204,7 @@ NetworkInterface::Map NetworkInterface::map(bool ipOnly, bool upOnly) bool hasBroadcast = (pAddress->IfType == IF_TYPE_ETHERNET_CSMACD) || (pAddress->IfType == IF_TYPE_SOFTWARE_LOOPBACK) || (pAddress->IfType == IF_TYPE_IEEE80211); if (hasBroadcast) { - // On Windows, a valid broadcast address will be all 1's (== address | ~subnetMask); additionaly, on pre-Vista versions of + // On Windows, a valid broadcast address will be all 1's (== address | ~subnetMask); additionally, on pre-Vista versions of // OS, master address structure does not contain member for prefix length; we go an extra mile here in order to make sure // we reflect the actual values held by system and protect against misconfiguration (e.g. bad DHCP config entry) ULONG prefixLength = 0; diff --git a/base/poco/Net/src/SMTPClientSession.cpp b/base/poco/Net/src/SMTPClientSession.cpp index 20e330a8d91..4ee72a46e7d 100644 --- a/base/poco/Net/src/SMTPClientSession.cpp +++ b/base/poco/Net/src/SMTPClientSession.cpp @@ -289,7 +289,7 @@ void SMTPClientSession::login(const std::string& hostname, LoginMethod loginMeth } else if (loginMethod != AUTH_NONE) { - throw SMTPException("The autentication method is not supported"); + throw SMTPException("The authentication method is not supported"); } } diff --git a/base/poco/Net/src/Socket.cpp b/base/poco/Net/src/Socket.cpp index 72a8bd9e2fa..b95634f3d0d 100644 --- a/base/poco/Net/src/Socket.cpp +++ b/base/poco/Net/src/Socket.cpp @@ -272,16 +272,16 @@ int Socket::select(SocketList& readList, SocketList& writeList, SocketList& exce SocketList readyWriteList; SocketList readyExceptList; - SocketList::iterator begE = exceptList.begin(); - SocketList::iterator endE = exceptList.end(); + SocketList::iterator beg = exceptList.begin(); + SocketList::iterator end = exceptList.end(); for (int idx = 0; idx < nfd; ++idx) { SocketList::iterator slIt = std::find_if(begR, endR, Socket::FDCompare(pPollArr[idx].fd)); if (POLLIN & pPollArr[idx].revents && slIt != endR) readyReadList.push_back(*slIt); slIt = std::find_if(begW, endW, Socket::FDCompare(pPollArr[idx].fd)); if (POLLOUT & pPollArr[idx].revents && slIt != endW) readyWriteList.push_back(*slIt); - slIt = std::find_if(begE, endE, Socket::FDCompare(pPollArr[idx].fd)); - if (POLLERR & pPollArr[idx].revents && slIt != endE) readyExceptList.push_back(*slIt); + slIt = std::find_if(beg, end, Socket::FDCompare(pPollArr[idx].fd)); + if (POLLERR & pPollArr[idx].revents && slIt != end) readyExceptList.push_back(*slIt); } std::swap(readList, readyReadList); std::swap(writeList, readyWriteList); diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/NetSSL.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/NetSSL.h index ae793fde934..7265a008e26 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/NetSSL.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/NetSSL.h @@ -29,7 +29,7 @@ // from a DLL simpler. All files within this DLL are compiled with the NetSSL_EXPORTS // symbol defined on the command line. this symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// NetSSL_API functions as being imported from a DLL, wheras this DLL sees symbols +// NetSSL_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if (defined(_WIN32) || defined(__CYGWIN__)) && defined(POCO_DLL) diff --git a/base/poco/NetSSL_OpenSSL/src/SSLException.cpp b/base/poco/NetSSL_OpenSSL/src/SSLException.cpp index 2b64d2a4d63..e098bec58da 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLException.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLException.cpp @@ -22,7 +22,7 @@ namespace Net { POCO_IMPLEMENT_EXCEPTION(SSLException, NetException, "SSL Exception") POCO_IMPLEMENT_EXCEPTION(SSLContextException, SSLException, "SSL context exception") -POCO_IMPLEMENT_EXCEPTION(InvalidCertificateException, SSLException, "Invalid certficate") +POCO_IMPLEMENT_EXCEPTION(InvalidCertificateException, SSLException, "Invalid certificate") POCO_IMPLEMENT_EXCEPTION(CertificateValidationException, SSLException, "Certificate validation error") POCO_IMPLEMENT_EXCEPTION(SSLConnectionUnexpectedlyClosedException, SSLException, "SSL connection unexpectedly closed") diff --git a/base/poco/Redis/include/Poco/Redis/Redis.h b/base/poco/Redis/include/Poco/Redis/Redis.h index bddc260b2d4..b39db2b9b67 100644 --- a/base/poco/Redis/include/Poco/Redis/Redis.h +++ b/base/poco/Redis/include/Poco/Redis/Redis.h @@ -28,7 +28,7 @@ // from a DLL simpler. All files within this DLL are compiled with the Redis_EXPORTS // symbol defined on the command line. This symbol should not be defined on any project // that uses this DLL. This way any other project whose source files include this file see -// Redis_API functions as being imported from a DLL, wheras this DLL sees symbols +// Redis_API functions as being imported from a DLL, whereas this DLL sees symbols // defined with this macro as being exported. // #if defined(_WIN32) && defined(POCO_DLL) diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index da63ecf1b9b..fa613043895 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -187,7 +187,7 @@ public: /// by the .ini file and the .xml file. /// /// If the application is built in debug mode (the _DEBUG preprocessor - /// macro is defined) and the base name of the appication executable + /// macro is defined) and the base name of the application executable /// ends with a 'd', a config file without the 'd' ending its base name is /// also found. /// diff --git a/base/poco/Util/include/Poco/Util/Units.h b/base/poco/Util/include/Poco/Util/Units.h index 47e8fe3c3aa..3c70e03b042 100644 --- a/base/poco/Util/include/Poco/Util/Units.h +++ b/base/poco/Util/include/Poco/Util/Units.h @@ -1260,7 +1260,6 @@ namespace Constants // Other constants: const Value pi (3.141592653589793); - const Value lightyear (9.4605284e15); const Value AU(149597871); const Value > > g (9.80665); } diff --git a/base/poco/XML/src/AbstractNode.cpp b/base/poco/XML/src/AbstractNode.cpp index d1c4912eacf..742b1aa7cbf 100644 --- a/base/poco/XML/src/AbstractNode.cpp +++ b/base/poco/XML/src/AbstractNode.cpp @@ -299,50 +299,50 @@ void AbstractNode::bubbleEvent(Event* evt) void AbstractNode::dispatchSubtreeModified() { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMSubtreeModified, this, true, false, 0); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMSubtreeModified, this, true, false, 0); + dispatchEvent(event.get()); } void AbstractNode::dispatchNodeInserted() { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMNodeInserted, this, true, false, parentNode()); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMNodeInserted, this, true, false, parentNode()); + dispatchEvent(event.get()); } void AbstractNode::dispatchNodeRemoved() { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMNodeRemoved, this, true, false, parentNode()); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMNodeRemoved, this, true, false, parentNode()); + dispatchEvent(event.get()); } void AbstractNode::dispatchNodeRemovedFromDocument() { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMNodeRemovedFromDocument, this, false, false, 0); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMNodeRemovedFromDocument, this, false, false, 0); + dispatchEvent(event.get()); } void AbstractNode::dispatchNodeInsertedIntoDocument() { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMNodeInsertedIntoDocument, this, false, false, 0); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMNodeInsertedIntoDocument, this, false, false, 0); + dispatchEvent(event.get()); } void AbstractNode::dispatchAttrModified(Attr* pAttr, MutationEvent::AttrChangeType changeType, const XMLString& prevValue, const XMLString& newValue) { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMAttrModified, this, true, false, pAttr, prevValue, newValue, pAttr->name(), changeType); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMAttrModified, this, true, false, pAttr, prevValue, newValue, pAttr->name(), changeType); + dispatchEvent(event.get()); } void AbstractNode::dispatchCharacterDataModified(const XMLString& prevValue, const XMLString& newValue) { - AutoPtr pEvent = new MutationEvent(_pOwner, MutationEvent::DOMCharacterDataModified, this, true, false, 0, prevValue, newValue, EMPTY_STRING, MutationEvent::MODIFICATION); - dispatchEvent(pEvent.get()); + AutoPtr event = new MutationEvent(_pOwner, MutationEvent::DOMCharacterDataModified, this, true, false, 0, prevValue, newValue, EMPTY_STRING, MutationEvent::MODIFICATION); + dispatchEvent(event.get()); } diff --git a/base/poco/XML/src/XMLStreamParser.cpp b/base/poco/XML/src/XMLStreamParser.cpp index e3937c327cf..20901f4922a 100644 --- a/base/poco/XML/src/XMLStreamParser.cpp +++ b/base/poco/XML/src/XMLStreamParser.cpp @@ -712,7 +712,7 @@ void XMLCALL XMLStreamParser::handleStartElement(void* v, const XML_Char* name, XML_ParsingStatus ps; XML_GetParsingStatus(p._parser, &ps); - // Expat has a (mis)-feature of a possibily calling handlers even + // Expat has a (mis)-feature of a possibly calling handlers even // after the non-resumable XML_StopParser call. // if (ps.parsing == XML_FINISHED) @@ -796,7 +796,7 @@ void XMLCALL XMLStreamParser::handleEndElement(void* v, const XML_Char* name) XML_ParsingStatus ps; XML_GetParsingStatus(p._parser, &ps); - // Expat has a (mis)-feature of a possibily calling handlers even + // Expat has a (mis)-feature of a possibly calling handlers even // after the non-resumable XML_StopParser call. // if (ps.parsing == XML_FINISHED) @@ -835,7 +835,7 @@ void XMLCALL XMLStreamParser::handleCharacters(void* v, const XML_Char* s, int n XML_ParsingStatus ps; XML_GetParsingStatus(p._parser, &ps); - // Expat has a (mis)-feature of a possibily calling handlers even + // Expat has a (mis)-feature of a possibly calling handlers even // after the non-resumable XML_StopParser call. // if (ps.parsing == XML_FINISHED) @@ -906,7 +906,7 @@ void XMLCALL XMLStreamParser::handleStartNamespaceDecl(void* v, const XML_Char* XML_ParsingStatus ps; XML_GetParsingStatus(p._parser, &ps); - // Expat has a (mis)-feature of a possibily calling handlers even + // Expat has a (mis)-feature of a possibly calling handlers even // after the non-resumable XML_StopParser call. // if (ps.parsing == XML_FINISHED) @@ -925,7 +925,7 @@ void XMLCALL XMLStreamParser::handleEndNamespaceDecl(void* v, const XML_Char* pr XML_ParsingStatus ps; XML_GetParsingStatus(p._parser, &ps); - // Expat has a (mis)-feature of a possibily calling handlers even + // Expat has a (mis)-feature of a possibly calling handlers even // after the non-resumable XML_StopParser call. // if (ps.parsing == XML_FINISHED) diff --git a/utils/check-style/check-typos b/utils/check-style/check-typos index 24f1f5d1b2d..9194a9464a7 100755 --- a/utils/check-style/check-typos +++ b/utils/check-style/check-typos @@ -9,6 +9,6 @@ codespell \ --ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \ --exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \ --quiet-level 2 \ - "$ROOT_PATH"/{src,base/base,base/glibc-compatibility,base/harmful,base/pcg-random,base/readpassphrase,base/widechar_width,programs,utils} \ # all subdirs of base/ except poco + "$ROOT_PATH"/{src,base,programs,utils} \ $@ | grep -P '.' \ && echo -e "\nFound some typos in code.\nSee the files utils/check-style/codespell* if you want to add an exception." diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index 9c26f322c8e..132f7d13ff2 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -12,6 +12,13 @@ numer ue alse nodel +keypair +ket +smove +ist +ower +onces +upto ot te fo From 06e1293909f9e9ee340284124f054c38ace4b775 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 9 Feb 2023 17:29:24 +0000 Subject: [PATCH 100/116] Fix duplicate includes in poco --- base/poco/Data/src/StatementImpl.cpp | 1 - .../Foundation/include/Poco/FPEnvironment.h | 2 +- base/poco/Foundation/src/Environment_UNIX.cpp | 25 +++++++++---------- base/poco/Foundation/src/Environment_VX.cpp | 2 -- base/poco/Foundation/src/FPEnvironment.cpp | 2 +- base/poco/Foundation/src/PipeImpl.cpp | 2 +- base/poco/Foundation/src/SharedMemory.cpp | 2 +- .../poco/Net/include/Poco/Net/WebSocketImpl.h | 1 - base/poco/Net/src/HTMLForm.cpp | 1 - base/poco/Net/src/MailMessage.cpp | 1 - base/poco/Net/src/NetworkInterface.cpp | 8 +++--- .../poco/Util/include/Poco/Util/Application.h | 1 - utils/check-style/check-duplicate-includes.sh | 2 +- 13 files changed, 21 insertions(+), 29 deletions(-) diff --git a/base/poco/Data/src/StatementImpl.cpp b/base/poco/Data/src/StatementImpl.cpp index 43d2c362c16..254f81bc1a2 100644 --- a/base/poco/Data/src/StatementImpl.cpp +++ b/base/poco/Data/src/StatementImpl.cpp @@ -23,7 +23,6 @@ #include "Poco/SharedPtr.h" #include "Poco/DateTime.h" #include "Poco/Exception.h" -#include "Poco/Data/DataException.h" using Poco::icompare; diff --git a/base/poco/Foundation/include/Poco/FPEnvironment.h b/base/poco/Foundation/include/Poco/FPEnvironment.h index 4602f57f91e..4392f04a56a 100644 --- a/base/poco/Foundation/include/Poco/FPEnvironment.h +++ b/base/poco/Foundation/include/Poco/FPEnvironment.h @@ -22,7 +22,7 @@ #if defined(POCO_NO_FPENVIRONMENT) -#include "Poco/FPEnvironment_DUMMY.h" +/// #include "Poco/FPEnvironment_DUMMY.h" #elif defined(__osf__) #include "Poco/FPEnvironment_DEC.h" #elif defined(sun) || defined(__sun) diff --git a/base/poco/Foundation/src/Environment_UNIX.cpp b/base/poco/Foundation/src/Environment_UNIX.cpp index b7e57e4314c..86800acb1f5 100644 --- a/base/poco/Foundation/src/Environment_UNIX.cpp +++ b/base/poco/Foundation/src/Environment_UNIX.cpp @@ -140,8 +140,8 @@ unsigned EnvironmentImpl::processorCountImpl() // // BSD variants // -#include -#include +/// #include +/// #include #include #include @@ -191,7 +191,6 @@ void EnvironmentImpl::nodeIdImpl(NodeId& id) #define ARPHRD_ETHER 1 /* Ethernet 10Mbps */ #endif #include -#include #include #include #include @@ -283,19 +282,19 @@ void EnvironmentImpl::nodeIdImpl(NodeId& id) // // General Unix // -#include +/// #include #if defined(sun) || defined(__sun) #include #endif -#include -#include -#include -#include -#include -#include -#include -#include -#include +/// #include +/// #include +/// #include +/// #include +/// #include +/// #include +/// #include +/// #include +/// #include namespace Poco { diff --git a/base/poco/Foundation/src/Environment_VX.cpp b/base/poco/Foundation/src/Environment_VX.cpp index f6869bab563..44dc6907688 100644 --- a/base/poco/Foundation/src/Environment_VX.cpp +++ b/base/poco/Foundation/src/Environment_VX.cpp @@ -32,8 +32,6 @@ #include #include #include -#include -#include namespace Poco { diff --git a/base/poco/Foundation/src/FPEnvironment.cpp b/base/poco/Foundation/src/FPEnvironment.cpp index 4617b7841a0..94a837f9cd6 100644 --- a/base/poco/Foundation/src/FPEnvironment.cpp +++ b/base/poco/Foundation/src/FPEnvironment.cpp @@ -18,7 +18,7 @@ #if defined(POCO_NO_FPENVIRONMENT) -#include "FPEnvironment_DUMMY.cpp" +/// #include "FPEnvironment_DUMMY.cpp" #elif defined(__osf__) #include "FPEnvironment_DEC.cpp" #elif defined(sun) || defined(__sun) diff --git a/base/poco/Foundation/src/PipeImpl.cpp b/base/poco/Foundation/src/PipeImpl.cpp index 0f43b45ba1d..bafc713a10e 100644 --- a/base/poco/Foundation/src/PipeImpl.cpp +++ b/base/poco/Foundation/src/PipeImpl.cpp @@ -17,7 +17,7 @@ #if defined(POCO_OS_FAMILY_WINDOWS) #if defined(_WIN32_WCE) -#include "PipeImpl_DUMMY.cpp" +/// #include "PipeImpl_DUMMY.cpp" #else #include "PipeImpl_WIN32.cpp" #endif diff --git a/base/poco/Foundation/src/SharedMemory.cpp b/base/poco/Foundation/src/SharedMemory.cpp index da43a421c02..154555c4fa5 100644 --- a/base/poco/Foundation/src/SharedMemory.cpp +++ b/base/poco/Foundation/src/SharedMemory.cpp @@ -15,7 +15,7 @@ #include "Poco/SharedMemory.h" #include "Poco/Exception.h" #if defined(POCO_NO_SHAREDMEMORY) -#include "SharedMemory_DUMMY.cpp" +/// #include "SharedMemory_DUMMY.cpp" #elif defined(POCO_OS_FAMILY_WINDOWS) #include "SharedMemory_WIN32.cpp" #elif defined(POCO_OS_FAMILY_UNIX) diff --git a/base/poco/Net/include/Poco/Net/WebSocketImpl.h b/base/poco/Net/include/Poco/Net/WebSocketImpl.h index a2471c3694b..e77ac7b8e72 100644 --- a/base/poco/Net/include/Poco/Net/WebSocketImpl.h +++ b/base/poco/Net/include/Poco/Net/WebSocketImpl.h @@ -21,7 +21,6 @@ #include "Poco/Net/StreamSocketImpl.h" #include "Poco/Buffer.h" #include "Poco/Random.h" -#include "Poco/Buffer.h" namespace Poco { diff --git a/base/poco/Net/src/HTMLForm.cpp b/base/poco/Net/src/HTMLForm.cpp index 3de03e6c6b7..b9614141f15 100644 --- a/base/poco/Net/src/HTMLForm.cpp +++ b/base/poco/Net/src/HTMLForm.cpp @@ -25,7 +25,6 @@ #include "Poco/StreamCopier.h" #include "Poco/URI.h" #include "Poco/String.h" -#include "Poco/CountingStream.h" #include "Poco/UTF8String.h" #include diff --git a/base/poco/Net/src/MailMessage.cpp b/base/poco/Net/src/MailMessage.cpp index 3fab8fa4688..838021d6355 100644 --- a/base/poco/Net/src/MailMessage.cpp +++ b/base/poco/Net/src/MailMessage.cpp @@ -24,7 +24,6 @@ #include "Poco/Net/NameValueCollection.h" #include "Poco/Base64Encoder.h" #include "Poco/Base64Decoder.h" -#include "Poco/StreamCopier.h" #include "Poco/DateTimeFormat.h" #include "Poco/DateTimeFormatter.h" #include "Poco/DateTimeParser.h" diff --git a/base/poco/Net/src/NetworkInterface.cpp b/base/poco/Net/src/NetworkInterface.cpp index 7bd00c0ef2d..616dea72189 100644 --- a/base/poco/Net/src/NetworkInterface.cpp +++ b/base/poco/Net/src/NetworkInterface.cpp @@ -1344,10 +1344,10 @@ NetworkInterface::NetworkInterfaceList NetworkInterface::list() // // BSD variants, QNX(?) and Solaris // -#include +/// #include #include -#include -#include +/// #include +/// #include #include #ifndef POCO_NO_NET_IFTYPES #include @@ -1526,7 +1526,7 @@ NetworkInterface::Map NetworkInterface::map(bool ipOnly, bool upOnly) #include #endif #include -#include +/// #include namespace Poco { namespace Net { diff --git a/base/poco/Util/include/Poco/Util/Application.h b/base/poco/Util/include/Poco/Util/Application.h index fa613043895..2382455cc82 100644 --- a/base/poco/Util/include/Poco/Util/Application.h +++ b/base/poco/Util/include/Poco/Util/Application.h @@ -27,7 +27,6 @@ #include "Poco/Path.h" #include "Poco/Timestamp.h" #include "Poco/Timespan.h" -#include "Poco/AutoPtr.h" #if defined(POCO_VXWORKS) #include #endif diff --git a/utils/check-style/check-duplicate-includes.sh b/utils/check-style/check-duplicate-includes.sh index 8043930cf5c..64aca4d180d 100755 --- a/utils/check-style/check-duplicate-includes.sh +++ b/utils/check-style/check-duplicate-includes.sh @@ -3,4 +3,4 @@ ROOT_PATH=$(git rev-parse --show-toplevel) # Find duplicate include directives -find $ROOT_PATH/{src,base/base,base/glibc-compatibility,base/harmful,base/pcg-random,base/readpassphrase,base/widechar_width,programs,utils} -name '*.h' -or -name '*.cpp' | while read file; do grep -P '^#include ' $file | sort | uniq -c | grep -v -P '^\s+1\s' && echo $file; done | sed '/^[[:space:]]*$/d' # all in base/ except poco +find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' | while read file; do grep -P '^#include ' $file | sort | uniq -c | grep -v -P '^\s+1\s' && echo $file; done | sed '/^[[:space:]]*$/d' From d61c665cd5603e4abe21a7736029e1a1c50f0d02 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Feb 2023 14:01:48 +0300 Subject: [PATCH 101/116] Update run.sh --- docker/test/stress/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 314be8ae0fd..ed8719fdbfd 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -651,6 +651,7 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then -e "No connection to ZooKeeper, cannot get shared table ID" \ -e "Session expired" \ -e "TOO_MANY_PARTS" \ + -e "Container already exists" \ /var/log/clickhouse-server/clickhouse-server.backward.dirty.log | rg -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e "Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)$FAIL$(trim_server_logs bc_check_error_messages.txt)" \ >> /test_output/test_results.tsv \ From 8458c7821bc990182161a364d15c85c6147b5395 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 12:05:47 +0100 Subject: [PATCH 102/116] Review comments fix --- src/Core/Field.h | 4 ++-- src/Disks/DiskSelector.h | 2 +- src/Disks/getOrCreateDiskFromAST.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 6a4581e8944..e25692c53fb 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -118,7 +118,7 @@ struct CustomType }; CustomType() = default; - explicit CustomType(std::shared_ptr impl_) : impl(impl_) {} + explicit CustomType(std::shared_ptr impl_) : impl(impl_) {} const char * getTypeName() const { return impl->getTypeName(); } String toString() const { return impl->toString(); } @@ -130,7 +130,7 @@ struct CustomType bool operator >= (const CustomType & rhs) const { return *impl >= *rhs.impl; } bool operator == (const CustomType & rhs) const { return *impl == *rhs.impl; } - std::shared_ptr impl; + std::shared_ptr impl; }; template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); diff --git a/src/Disks/DiskSelector.h b/src/Disks/DiskSelector.h index eda16d6b2e1..58adeb953db 100644 --- a/src/Disks/DiskSelector.h +++ b/src/Disks/DiskSelector.h @@ -18,7 +18,7 @@ using DiskSelectorPtr = std::shared_ptr; class DiskSelector { public: - static constexpr auto TMP_DISK_PREFIX = "__tmp_"; + static constexpr auto TMP_INTERNAL_DISK_PREFIX = "__tmp_internal_"; DiskSelector() = default; DiskSelector(const DiskSelector & from) = default; diff --git a/src/Disks/getOrCreateDiskFromAST.cpp b/src/Disks/getOrCreateDiskFromAST.cpp index 48d40864396..fc9cd7edbee 100644 --- a/src/Disks/getOrCreateDiskFromAST.cpp +++ b/src/Disks/getOrCreateDiskFromAST.cpp @@ -32,7 +32,7 @@ std::string getOrCreateDiskFromDiskAST(const ASTFunction & function, ContextPtr /// after table is reattached or server is restarted, so take a hash of the disk /// configuration serialized ast as a disk name suffix. auto disk_setting_string = serializeAST(function, true); - auto disk_name = DiskSelector::TMP_DISK_PREFIX + auto disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX + toString(sipHash128(disk_setting_string.data(), disk_setting_string.size())); LOG_TRACE( From 81e943c7f8f050083cfbc33c608643e37ed6fcf9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 10 Feb 2023 11:20:48 +0000 Subject: [PATCH 103/116] More spelling fixes --- base/poco/Data/include/Poco/Data/SQLChannel.h | 2 +- base/poco/Foundation/include/Poco/Any.h | 2 +- .../Foundation/include/Poco/DirectoryIteratorStrategy.h | 2 +- base/poco/Foundation/include/Poco/LinearHashTable.h | 2 +- base/poco/Foundation/include/Poco/PatternFormatter.h | 2 +- .../Foundation/include/Poco/RecursiveDirectoryIterator.h | 2 +- base/poco/Foundation/include/Poco/RegularExpression.h | 2 +- base/poco/Foundation/include/Poco/TaskNotification.h | 2 +- base/poco/Foundation/src/fast-dtoa.cc | 2 +- base/poco/Foundation/src/pcre_config.h | 2 +- base/poco/Foundation/src/pcre_jit_compile.c | 6 +++--- base/poco/Foundation/src/strtod.cc | 2 +- base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h | 2 +- utils/check-style/codespell-ignore-words.list | 1 + 14 files changed, 16 insertions(+), 15 deletions(-) diff --git a/base/poco/Data/include/Poco/Data/SQLChannel.h b/base/poco/Data/include/Poco/Data/SQLChannel.h index b5ad819786f..068f97f71f5 100644 --- a/base/poco/Data/include/Poco/Data/SQLChannel.h +++ b/base/poco/Data/include/Poco/Data/SQLChannel.h @@ -52,7 +52,7 @@ class Data_API SQLChannel: public Poco::Channel /// Other than DateTime filed name used for optiona time-based archiving purposes, currently the /// field names are not mandated. However, it is recommended to use names as specified above. /// - /// To provide as non-intrusive operation as possbile, the log entries are cached and + /// To provide as non-intrusive operation as possible, the log entries are cached and /// inserted into the target database asynchronously by default . The blocking, however, will occur /// before the next entry insertion with default timeout of 1 second. The default settings can be /// overridden (see async, timeout and throw properties for details). diff --git a/base/poco/Foundation/include/Poco/Any.h b/base/poco/Foundation/include/Poco/Any.h index 30f5b764177..977ea3d99a7 100644 --- a/base/poco/Foundation/include/Poco/Any.h +++ b/base/poco/Foundation/include/Poco/Any.h @@ -201,7 +201,7 @@ public: Any& swap(Any& other) /// Swaps the content of the two Anys. /// - /// When small object optimizaton is enabled, swap only + /// When small object optimization is enabled, swap only /// has no-throw guarantee when both (*this and other) /// objects are allocated on the heap. { diff --git a/base/poco/Foundation/include/Poco/DirectoryIteratorStrategy.h b/base/poco/Foundation/include/Poco/DirectoryIteratorStrategy.h index 4591f7c6077..392ccd23d5e 100644 --- a/base/poco/Foundation/include/Poco/DirectoryIteratorStrategy.h +++ b/base/poco/Foundation/include/Poco/DirectoryIteratorStrategy.h @@ -5,7 +5,7 @@ // Package: Filesystem // Module: RecursiveDirectoryIterator // -// Definitions of the RecursiveDirectoryIterator stategy classes. +// Definitions of the RecursiveDirectoryIterator strategy classes. // // Copyright (c) 2012, Applied Informatics Software Engineering GmbH. // and Contributors. diff --git a/base/poco/Foundation/include/Poco/LinearHashTable.h b/base/poco/Foundation/include/Poco/LinearHashTable.h index ac18b4b95aa..d2bc143ad68 100644 --- a/base/poco/Foundation/include/Poco/LinearHashTable.h +++ b/base/poco/Foundation/include/Poco/LinearHashTable.h @@ -35,7 +35,7 @@ class LinearHashTable /// This class implements a linear hash table. /// /// In a linear hash table, the available address space - /// grows or shrinks dynamically. A linar hash table thus + /// grows or shrinks dynamically. A linear hash table thus /// supports any number of insertions or deletions without /// lookup or insertion performance deterioration. /// diff --git a/base/poco/Foundation/include/Poco/PatternFormatter.h b/base/poco/Foundation/include/Poco/PatternFormatter.h index cdbb0f08dbc..e0aa15010ae 100644 --- a/base/poco/Foundation/include/Poco/PatternFormatter.h +++ b/base/poco/Foundation/include/Poco/PatternFormatter.h @@ -134,7 +134,7 @@ private: void parsePattern(); /// Will parse the _pattern string into the vector of PatternActions, /// which contains the message key, any text that needs to be written first - /// a proprety in case of %[] and required length. + /// a property in case of %[] and required length. std::vector _patternActions; bool _localTime; diff --git a/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h b/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h index 79b957d129d..1a0f78241c7 100644 --- a/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h +++ b/base/poco/Foundation/include/Poco/RecursiveDirectoryIterator.h @@ -51,7 +51,7 @@ class RecursiveDirectoryIterator /// The class can follow different traversal strategies: /// * depth-first strategy; /// * siblings-first strategy. - /// The stategies are set by template parameter. + /// The strategies are set by template parameter. /// There are two corresponding typedefs: /// * SimpleRecursiveDirectoryIterator; /// * SiblingsFirstRecursiveDirectoryIterator. diff --git a/base/poco/Foundation/include/Poco/RegularExpression.h b/base/poco/Foundation/include/Poco/RegularExpression.h index 74f2c7f95cc..81b28fef1bb 100644 --- a/base/poco/Foundation/include/Poco/RegularExpression.h +++ b/base/poco/Foundation/include/Poco/RegularExpression.h @@ -85,7 +85,7 @@ public: /// If study is true, the pattern is analyzed and optimized. This /// is mainly useful if the pattern is used more than once. /// For a description of the options, please see the PCRE documentation. - /// Throws a RegularExpressionException if the patter cannot be compiled. + /// Throws a RegularExpressionException if the pattern cannot be compiled. ~RegularExpression(); /// Destroys the regular expression. diff --git a/base/poco/Foundation/include/Poco/TaskNotification.h b/base/poco/Foundation/include/Poco/TaskNotification.h index 92d8a95821d..88dab508d38 100644 --- a/base/poco/Foundation/include/Poco/TaskNotification.h +++ b/base/poco/Foundation/include/Poco/TaskNotification.h @@ -120,7 +120,7 @@ class TaskCustomNotification: public TaskNotification /// This is a template for "custom" notification. /// Unlike other notifications, this notification /// is instantiated and posted by the task itself. - /// The purpose is to provide generic notifiation + /// The purpose is to provide generic notification /// mechanism between the task and its observer(s). { public: diff --git a/base/poco/Foundation/src/fast-dtoa.cc b/base/poco/Foundation/src/fast-dtoa.cc index 857a1301472..a58f4d4487a 100644 --- a/base/poco/Foundation/src/fast-dtoa.cc +++ b/base/poco/Foundation/src/fast-dtoa.cc @@ -565,7 +565,7 @@ static bool Grisu3(double v, // the difference between w and boundary_minus/plus (a power of 2) and to // compute scaled_boundary_minus/plus by subtracting/adding from // scaled_w. However the code becomes much less readable and the speed - // enhancements are not terriffic. + // enhancements are not terrific. DiyFp scaled_boundary_minus = DiyFp::Times(boundary_minus, ten_mk); DiyFp scaled_boundary_plus = DiyFp::Times(boundary_plus, ten_mk); diff --git a/base/poco/Foundation/src/pcre_config.h b/base/poco/Foundation/src/pcre_config.h index 7d95bd4f63c..8e6c11ee194 100644 --- a/base/poco/Foundation/src/pcre_config.h +++ b/base/poco/Foundation/src/pcre_config.h @@ -322,7 +322,7 @@ sure both macros are undefined; an emulation function will then be used. */ contain the relevant magic. If you do not define this macro, a suitable __declspec value is used for Windows systems; in other environments "extern" is used for a C compiler and "extern C" for a C++ compiler. - This macro apears at the start of every exported function that is part + This macro appears at the start of every exported function that is part of the external API. It does not appear on functions that are "external" in the C sense, but which are internal to the library. */ /* #undef PCRE_EXP_DEFN */ diff --git a/base/poco/Foundation/src/pcre_jit_compile.c b/base/poco/Foundation/src/pcre_jit_compile.c index 2a92feabc98..95c9c836680 100644 --- a/base/poco/Foundation/src/pcre_jit_compile.c +++ b/base/poco/Foundation/src/pcre_jit_compile.c @@ -147,7 +147,7 @@ Saved stack frames: Atomic blocks and asserts require reloading the values of private data when the backtrack mechanism performed. Because of OP_RECURSE, the data -are not necessarly known in compile time, thus we need a dynamic restore +are not necessarily known in compile time, thus we need a dynamic restore mechanism. The stack frames are stored in a chain list, and have the following format: @@ -7923,8 +7923,8 @@ return stacksize; M - Any values pushed by the current alternative. Can be empty, or anything. C - Push the previous OVECTOR(i), OVECTOR(i+1) and OVECTOR_PRIV(i) to the stack. L - Push the previous local (pointed by localptr) to the stack - () - opional values stored on the stack - ()* - optonal, can be stored multiple times + () - optional values stored on the stack + ()* - optional, can be stored multiple times The following list shows the regular expression templates, their PCRE byte codes and stack layout supported by pcre-sljit. diff --git a/base/poco/Foundation/src/strtod.cc b/base/poco/Foundation/src/strtod.cc index e74b981f17c..7c776943be5 100644 --- a/base/poco/Foundation/src/strtod.cc +++ b/base/poco/Foundation/src/strtod.cc @@ -529,7 +529,7 @@ float Strtof(Vector buffer, int exponent) { (f1 == f2 && f2 != f3 && f3 == f4) || (f1 == f2 && f2 == f3 && f3 != f4)); - // guess and next are the two possible canditates (in the same way that + // guess and next are the two possible candidates (in the same way that // double_guess was the lower candidate for a double-precision guess). float guess = f1; float next = f4; diff --git a/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h b/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h index 8f3486d07fe..1561e183447 100644 --- a/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h +++ b/base/poco/MongoDB/include/Poco/MongoDB/DeleteRequest.h @@ -28,7 +28,7 @@ namespace MongoDB { class MongoDB_API DeleteRequest: public RequestMessage - /// A DeleteRequest is used to delete one ore more documents from a database. + /// A DeleteRequest is used to delete one or more documents from a database. /// /// Specific flags for this request /// - DELETE_DEFAULT: default delete operation diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index 132f7d13ff2..27e08de80ee 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -19,6 +19,7 @@ ist ower onces upto +clen ot te fo From 9c20e42807661b1aacb8f5bf9f53e6bbf7e544a7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Feb 2023 14:25:26 +0300 Subject: [PATCH 104/116] Update run.sh --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index ed8719fdbfd..78f45667a15 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -402,7 +402,7 @@ start # NOTE Hung check is implemented in docker/tests/stress/stress rg -Fa "No queries hung" /test_output/test_results.tsv | grep -Fa "OK" \ - || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log | unts)" >> /test_output/test_results.tsv + || echo -e "Hung check failed, possible deadlock found (see hung_check.log)$FAIL$(head_escaped /test_output/hung_check.log)" >> /test_output/test_results.tsv stop mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.stress.log From 17561c3dae8d597f3da32d6ce605bf4b7d10f87c Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 10 Feb 2023 11:25:59 +0000 Subject: [PATCH 105/116] Fix build --- src/Functions/MultiMatchAnyImpl.h | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index 7a35fcebc24..672aaa95169 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -327,11 +327,14 @@ struct MultiMatchAnyImpl checkHyperscanRegexp(needles, max_hyperscan_regexp_length, max_hyperscan_regexp_total_length); - for (auto needle : needles) + if (reject_expensive_hyperscan_regexps) { - SlowWithHyperscanChecker checker; - if (checker.isSlow(needle)) - throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + for (auto needle : needles) + { + SlowWithHyperscanChecker checker; + if (checker.isSlow(needle)) + throw Exception(ErrorCodes::HYPERSCAN_CANNOT_SCAN_TEXT, "Regular expression evaluation in vectorscan will be too slow. To ignore this error, disable setting 'reject_expensive_hyperscan_regexps'."); + } } for (size_t j = 0; j < needles.size(); ++j) From deda940d870cf57dfc62abf9a3b44147150b5ff7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 10 Feb 2023 12:57:47 +0000 Subject: [PATCH 106/116] Fix possible out of bounds error while reading LowCardinality(Nullable) in Arrow format --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 3 ++- .../02563_arrow_low_cardinality_bug.reference | 2 ++ .../0_stateless/02563_arrow_low_cardinality_bug.sh | 14 ++++++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02563_arrow_low_cardinality_bug.reference create mode 100755 tests/queries/0_stateless/02563_arrow_low_cardinality_bug.sh diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 68c40527097..c7ab727e74e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -384,9 +384,10 @@ static ColumnWithTypeAndName readColumnWithIndexesDataImpl(std::shared_ptr(buffer->data()); /// Check that indexes are correct (protection against corrupted files) + /// Note that on null values index can be arbitrary value. for (int64_t i = 0; i != chunk->length(); ++i) { - if (data[i] < 0 || data[i] >= dict_size) + if (!chunk->IsNull(i) && (data[i] < 0 || data[i] >= dict_size)) throw Exception(ErrorCodes::INCORRECT_DATA, "Index {} in Dictionary column is out of bounds, dictionary size is {}", Int64(data[i]), UInt64(dict_size)); diff --git a/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.reference b/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.reference new file mode 100644 index 00000000000..416d7c7547a --- /dev/null +++ b/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.reference @@ -0,0 +1,2 @@ +0 +\N diff --git a/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.sh b/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.sh new file mode 100755 index 00000000000..efdb7288dec --- /dev/null +++ b/tests/queries/0_stateless/02563_arrow_low_cardinality_bug.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test" +$CLICKHOUSE_CLIENT -q "create table test (z LowCardinality(Nullable(String))) engine=Memory"; +$CLICKHOUSE_CLIENT -q "select CAST(number % 2 ? NULL : toString(number), 'LowCardinality(Nullable(String))') as z from numbers(2) format Arrow settings output_format_arrow_low_cardinality_as_dictionary=1" | $CLICKHOUSE_CLIENT -q "insert into test format Arrow" + +$CLICKHOUSE_CLIENT -q "select * from test" +$CLICKHOUSE_CLIENT -q "drop table test" + From 8f65dc4ff7eccafa29598951d4522aff810ab224 Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 9 Feb 2023 14:02:26 +0100 Subject: [PATCH 107/116] ASTFunction: avoid rewriting tuple function as literal when not safe It is not always safe to implicitly rewrite a `tuple` function call to a literal. Even when the tuple has more then one element e.g. `SELECT (tuple(1, 2, 3) AS x).1` is not equivalent to `SELECT ((1, 2, 3) AS x).1`, the latter is interpreted as `SELECT tuple((1, 2, 3) AS x).1`. --- src/Parsers/ASTFunction.cpp | 3 ++- tests/queries/0_stateless/02560_tuple_format.reference | 4 ++++ tests/queries/0_stateless/02560_tuple_format.sh | 10 ++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02560_tuple_format.reference create mode 100755 tests/queries/0_stateless/02560_tuple_format.sh diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 7a19cba0f75..129d3d60744 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -1019,7 +1019,8 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (!written && arguments->children.size() >= 2 && name == "tuple"sv) { - settings.ostr << (settings.hilite ? hilite_operator : "") << '(' << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_operator : "") << ((frame.need_parens && !alias.empty()) ? "tuple" : "") << '(' + << (settings.hilite ? hilite_none : ""); for (size_t i = 0; i < arguments->children.size(); ++i) { if (i != 0) diff --git a/tests/queries/0_stateless/02560_tuple_format.reference b/tests/queries/0_stateless/02560_tuple_format.reference new file mode 100644 index 00000000000..7582ad08d73 --- /dev/null +++ b/tests/queries/0_stateless/02560_tuple_format.reference @@ -0,0 +1,4 @@ +SELECT (1, 2, 3) +SELECT (1, 2, 3) AS x +SELECT (1, 2, 3).1 +SELECT (tuple(1, 2, 3) AS x).1 diff --git a/tests/queries/0_stateless/02560_tuple_format.sh b/tests/queries/0_stateless/02560_tuple_format.sh new file mode 100755 index 00000000000..8ef811c08d7 --- /dev/null +++ b/tests/queries/0_stateless/02560_tuple_format.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "select (tuple(1, 2, 3));" | "$CLICKHOUSE_FORMAT" +echo "select (tuple(1, 2, 3) as x);" | "$CLICKHOUSE_FORMAT" +echo "select (tuple(1, 2, 3)).1;" | "$CLICKHOUSE_FORMAT" +echo "select (tuple(1, 2, 3) as x).1;" | "$CLICKHOUSE_FORMAT" From dbbc577863599c94202d0f437419f1e844980746 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 10 Feb 2023 14:02:52 +0100 Subject: [PATCH 108/116] Add upcoming Events ...and split recent recordings into a new section --- README.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 75e0fa1bc4d..49703f44390 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,13 @@ ClickHouse® is an open-source column-oriented database management system that a * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. -## Events +## Upcoming Events +* [**ClickHouse Workshop**](https://staging2.clickhouse.com/company/events/2023-02-15-clickhouse-workshop?utm_source=github&utm_medium=social&utm_campaign=workshop) - Feb 15 & 16 - In this 2-day (3 hrs per day) free training, topics range from introductory content to a deep dive on interacting with and understanding your data. There will be both live training and hands-on labs. +* [**v23.2 Release Webinar**](https://clickhouse.com/company/events/v23-2-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Feb 23 - 23.2 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/291485868/) - Mar 9 - The first ClickHouse Amsterdam Meetup of 2023 is here! 🎉 Join us for short lightning talks and long discussions. Food, drinks & good times on us. +* [**ClickHouse Meetup in SF Bay Area**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/291490121/) - Mar 14 - A night to meet with ClickHouse team in the San Francisco area! Food and drink are a given...but networking is the primary focus. +* [**ClickHouse Meetup in Austin**](https://www.meetup.com/clickhouse-austin-user-group/events/291486654/) - Mar 16 - The first ClickHouse Meetup in Austin is happening soon! Interested in speaking, let us know! + +## Recent Recordings * **FOSDEM 2023**: In the "Fast and Streaming Data" room Alexey gave a talk entitled "Building Analytical Apps With ClickHouse" that looks at the landscape of data tools, an interesting data set, and how you can interact with data quickly. Check out the recording on **[YouTube](https://www.youtube.com/watch?v=JlcI2Vfz_uk)**. * **Recording available**: [**v23.1 Release Webinar**](https://www.youtube.com/watch?v=zYSZXBnTMSE) 23.1 is the ClickHouse New Year release. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. Inverted indices, query cache, and so -- very -- much more. -* **Recording available**: [**ClickHouse Meetup at the CHEQ office in Tel Aviv**](https://www.meetup.com/clickhouse-tel-aviv-user-group/events/289599423/) - We are very excited to be holding our next in-person ClickHouse meetup at the CHEQ office in Tel Aviv! Hear from CHEQ, ServiceNow and Contentsquare, as well as a deep dive presentation from ClickHouse CTO Alexey Milovidov. Join us for a fun evening of talks, food and discussion! From 44b82cf9ac4c75be3adecd02b3658443400aaf16 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 10 Feb 2023 14:04:49 +0100 Subject: [PATCH 109/116] Update README.md wrong URLs are wrong --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 49703f44390..dee60cafb33 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,7 @@ ClickHouse® is an open-source column-oriented database management system that a * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming Events -* [**ClickHouse Workshop**](https://staging2.clickhouse.com/company/events/2023-02-15-clickhouse-workshop?utm_source=github&utm_medium=social&utm_campaign=workshop) - Feb 15 & 16 - In this 2-day (3 hrs per day) free training, topics range from introductory content to a deep dive on interacting with and understanding your data. There will be both live training and hands-on labs. +* [**ClickHouse Workshop**](https://clickhouse.com/company/events/2023-02-15-clickhouse-workshop?utm_source=github&utm_medium=social&utm_campaign=workshop) - Feb 15 & 16 - In this 2-day (3 hrs per day) free training, topics range from introductory content to a deep dive on interacting with and understanding your data. There will be both live training and hands-on labs. * [**v23.2 Release Webinar**](https://clickhouse.com/company/events/v23-2-release-webinar?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-02) - Feb 23 - 23.2 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/291485868/) - Mar 9 - The first ClickHouse Amsterdam Meetup of 2023 is here! 🎉 Join us for short lightning talks and long discussions. Food, drinks & good times on us. * [**ClickHouse Meetup in SF Bay Area**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/291490121/) - Mar 14 - A night to meet with ClickHouse team in the San Francisco area! Food and drink are a given...but networking is the primary focus. From 38b6f89d68e4334de81ec6ac152f14e137601c78 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 10 Feb 2023 13:28:06 +0000 Subject: [PATCH 110/116] Fix FreeBSD build --- base/poco/Foundation/src/Environment_UNIX.cpp | 8 ++++---- base/poco/Net/src/NetworkInterface.cpp | 12 ++++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/base/poco/Foundation/src/Environment_UNIX.cpp b/base/poco/Foundation/src/Environment_UNIX.cpp index 86800acb1f5..04622dd1874 100644 --- a/base/poco/Foundation/src/Environment_UNIX.cpp +++ b/base/poco/Foundation/src/Environment_UNIX.cpp @@ -26,6 +26,8 @@ #include #endif +#include +#include namespace Poco { @@ -140,8 +142,6 @@ unsigned EnvironmentImpl::processorCountImpl() // // BSD variants // -/// #include -/// #include #include #include @@ -182,7 +182,7 @@ void EnvironmentImpl::nodeIdImpl(NodeId& id) // Linux, Cygwin // #include -#include +/// #include #include #include #ifndef __CYGWIN__ @@ -191,7 +191,7 @@ void EnvironmentImpl::nodeIdImpl(NodeId& id) #define ARPHRD_ETHER 1 /* Ethernet 10Mbps */ #endif #include -#include +/// #include #include #include #include diff --git a/base/poco/Net/src/NetworkInterface.cpp b/base/poco/Net/src/NetworkInterface.cpp index 616dea72189..e6279040e6e 100644 --- a/base/poco/Net/src/NetworkInterface.cpp +++ b/base/poco/Net/src/NetworkInterface.cpp @@ -39,6 +39,9 @@ #include #include +#include +#include +#include using Poco::NumberFormatter; using Poco::FastMutex; @@ -1344,10 +1347,7 @@ NetworkInterface::NetworkInterfaceList NetworkInterface::list() // // BSD variants, QNX(?) and Solaris // -/// #include #include -/// #include -/// #include #include #ifndef POCO_NO_NET_IFTYPES #include @@ -1517,11 +1517,11 @@ NetworkInterface::Map NetworkInterface::map(bool ipOnly, bool upOnly) // -#include +/// #include #if POCO_OS != POCO_OS_ANDROID // Android doesn't have -#include +/// #include #endif -#include +/// #include #ifndef POCO_NO_LINUX_IF_PACKET_H #include #endif From 45ee7c70814d144a6164040287c37e113519a249 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 10 Feb 2023 13:56:47 +0000 Subject: [PATCH 111/116] fix --- src/Parsers/ParserCreateQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 052bb3863b4..8cbfac91465 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -1299,13 +1299,13 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - if (!is_ordinary_view && ParserKeyword{"TO INNER UUID"}.ignore(pos, expected)) + if (is_materialized_view && ParserKeyword{"TO INNER UUID"}.ignore(pos, expected)) { ParserStringLiteral literal_p; if (!literal_p.parse(pos, to_inner_uuid, expected)) return false; } - else if (!is_ordinary_view && ParserKeyword{"TO"}.ignore(pos, expected)) + else if (is_materialized_view && ParserKeyword{"TO"}.ignore(pos, expected)) { // TO [db.]table if (!table_name_p.parse(pos, to_table, expected)) From ef133ee30d2cbf43b7c215bb2e17c7318ed400ff Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 10 Feb 2023 16:22:35 +0100 Subject: [PATCH 112/116] fix data race between check table query and background part checker thread --- src/Storages/StorageReplicatedMergeTree.cpp | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 16a2f49b8df..014fcd4e3de 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7835,18 +7835,23 @@ CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, Context else data_parts = getVisibleDataPartsVector(local_context); - for (auto & part : data_parts) { - try + auto part_check_lock = part_check_thread.pausePartsCheck(); + + for (auto & part : data_parts) { - results.push_back(part_check_thread.checkPart(part->name)); - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + try + { + results.push_back(part_check_thread.checkPart(part->name)); + } + catch (const Exception & ex) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + } } } + return results; } From 681baa10b79d6cdecef948551a650cbca7aa9c2c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Feb 2023 18:56:22 +0300 Subject: [PATCH 113/116] Update clickhouse-test --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2215982affe..035a1cc1001 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -120,6 +120,7 @@ def clickhouse_execute_http( if default_format is not None: params["default_format"] = default_format + error_message = "" for i in range(max_http_retries): try: client.request( @@ -130,8 +131,9 @@ def clickhouse_execute_http( data = res.read() break except Exception as ex: + error_message += "\tAttempt {} failed: {}\n{}\n".format(i, str(ex), traceback.format_exc()) if i == max_http_retries - 1: - raise ex + raise Exception("Failed to send a request after {} retries:\n{}".format(max_http_retries, error_message)) sleep(i + 1) From b2870d3e0c281109b49c9ccc7e9b5a866b7002c7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 10 Feb 2023 19:34:13 +0300 Subject: [PATCH 114/116] Update clickhouse-test --- tests/clickhouse-test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 035a1cc1001..7511a22f912 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -4,6 +4,7 @@ # pylint: disable=global-variable-not-assigned # pylint: disable=too-many-lines # pylint: disable=anomalous-backslash-in-string +# pylint: disable=raise-missing-from import enum from queue import Full From f3958819e1e2675ac6077fabd8c8331b8f3e4c89 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 10 Feb 2023 18:20:25 +0100 Subject: [PATCH 115/116] More reliable --- .../queries/0_stateless/01052_window_view_proc_tumble_to_now.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh index 033568b6077..44afc984fb7 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sh @@ -12,7 +12,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); EOF From 8c3f85e58d5c5a59658b3de8903c7fe8a5ee6013 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 10 Feb 2023 17:39:28 +0000 Subject: [PATCH 116/116] more review fixes --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 8 +++++++- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 11 ++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index a4242d9e458..f8ca6b9ab07 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -58,7 +58,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { try { - ResourceGuard rlock(write_settings.resource_link, cost); + ResourceGuard rlock(write_settings.resource_link, cost); // Note that zero-cost requests are ignored func(); break; } @@ -70,6 +70,12 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { handle_exception(e, i); } + catch (...) + { + if (cost) + write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure + throw; + } } } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 466dd8f04dc..293f5d31836 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -66,7 +66,16 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) { ResourceGuard rlock(write_settings.resource_link, size); - int bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + int bytes_written; + try + { + bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + } + catch (...) + { + write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure + throw; + } rlock.unlock(); if (bytes_written < 0)