From db0724dc0e232830f1f52076f3c4dc2a3af8b57f Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 29 May 2025 10:04:06 +0200 Subject: [PATCH 01/16] Merge pull request #798 from Altinity/feature/antalya-25.3/alternative_syntax 25.3 Antalya port - Alternative syntax for cluster functions --- .../FunctionSecretArgumentsFinderTreeNode.h | 10 +- src/Core/Settings.cpp | 9 + src/Core/SettingsChangesHistory.cpp | 6 + src/Databases/DataLake/DatabaseDataLake.cpp | 14 +- .../DataLake/DatabaseDataLakeSettings.cpp | 1 + src/Disks/DiskType.cpp | 60 ++- src/Disks/DiskType.h | 6 +- src/Interpreters/Cluster.cpp | 37 +- src/Interpreters/Cluster.h | 7 +- src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Parsers/FunctionSecretArgumentsFinder.h | 108 ++++- .../FunctionSecretArgumentsFinderAST.h | 9 +- src/Server/TCPHandler.cpp | 1 - src/Storages/IStorage.h | 3 + src/Storages/IStorageCluster.cpp | 38 +- src/Storages/IStorageCluster.h | 40 +- .../ObjectStorage/Azure/Configuration.cpp | 57 +-- .../ObjectStorage/Azure/Configuration.h | 4 + .../DataLakes/DataLakeConfiguration.h | 248 ++++++++++- .../ObjectStorage/DataLakes/HudiMetadata.cpp | 2 +- .../Iceberg/metadata/INFORMATION_SCHEMA.sql | 2 + .../DataLakes/Iceberg/metadata/default.sql | 2 + .../DataLakes/Iceberg/metadata/ice.sql | 3 + .../Iceberg/metadata/information_schema.sql | 2 + .../DataLakes/Iceberg/metadata/system | 1 + .../DataLakes/Iceberg/metadata/system.sql | 2 + .../Iceberg/preprocessed_configs/config.xml | 44 ++ .../ObjectStorage/DataLakes/Iceberg/status | 3 + .../ObjectStorage/DataLakes/Iceberg/uuid | 1 + .../ObjectStorage/HDFS/Configuration.cpp | 23 +- .../ObjectStorage/HDFS/Configuration.h | 2 + .../ObjectStorage/Local/Configuration.cpp | 14 +- .../ObjectStorage/ReadBufferIterator.cpp | 10 +- .../ObjectStorage/S3/Configuration.cpp | 43 +- src/Storages/ObjectStorage/S3/Configuration.h | 2 + .../ObjectStorage/StorageObjectStorage.cpp | 45 +- .../ObjectStorage/StorageObjectStorage.h | 29 +- .../StorageObjectStorageCluster.cpp | 289 ++++++++++++- .../StorageObjectStorageCluster.h | 56 ++- .../StorageObjectStorageSettings.h | 10 + .../StorageObjectStorageSink.cpp | 4 +- .../StorageObjectStorageSource.cpp | 12 +- src/Storages/ObjectStorage/Utils.cpp | 15 +- src/Storages/ObjectStorage/Utils.h | 3 +- .../registerStorageObjectStorage.cpp | 27 +- .../StorageObjectStorageQueue.cpp | 8 +- .../StorageObjectStorageQueue.h | 2 +- .../registerQueueStorage.cpp | 2 +- .../extractTableFunctionFromSelectQuery.cpp | 11 +- .../extractTableFunctionFromSelectQuery.h | 2 + src/TableFunctions/ITableFunction.h | 2 +- src/TableFunctions/ITableFunctionCluster.h | 6 +- .../TableFunctionObjectStorage.cpp | 178 ++------ .../TableFunctionObjectStorage.h | 26 +- .../TableFunctionObjectStorageCluster.cpp | 40 +- .../TableFunctionObjectStorageCluster.h | 22 +- ...leFunctionObjectStorageClusterFallback.cpp | 386 ++++++++++++++++++ ...ableFunctionObjectStorageClusterFallback.h | 49 +++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + tests/integration/test_s3_cluster/test.py | 93 +++++ .../integration/test_storage_iceberg/test.py | 21 + 62 files changed, 1792 insertions(+), 365 deletions(-) create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql create mode 120000 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/status create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/uuid create mode 100644 src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp create mode 100644 src/TableFunctions/TableFunctionObjectStorageClusterFallback.h diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 8bcb6e147420..e4f63192c95b 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -71,8 +71,14 @@ class FunctionTreeNodeImpl : public AbstractFunction { public: explicit ArgumentsTreeNode(const QueryTreeNodes * arguments_) : arguments(arguments_) {} - size_t size() const override { return arguments ? arguments->size() : 0; } - std::unique_ptr at(size_t n) const override { return std::make_unique(arguments->at(n).get()); } + size_t size() const override + { /// size withous skipped indexes + return arguments ? arguments->size() - skippedSize() : 0; + } + std::unique_ptr at(size_t n) const override + { /// n is relative index, some can be skipped + return std::make_unique(arguments->at(getRealIndex(n)).get()); + } private: const QueryTreeNodes * arguments = nullptr; }; diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9ad69a281a50..a40e7e6be865 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6829,6 +6829,15 @@ Enable PRQL - an alternative to SQL. )", EXPERIMENTAL) \ DECLARE(Bool, enable_adaptive_memory_spill_scheduler, false, R"( Trigger processor to spill data into external storage adpatively. grace join is supported at present. +)", EXPERIMENTAL) \ + DECLARE(String, object_storage_cluster, "", R"( +Cluster to make distributed requests to object storages with alternative syntax. +)", EXPERIMENTAL) \ + DECLARE(UInt64, object_storage_max_nodes, 0, R"( +Limit for hosts used for request in object storage cluster table functions - azureBlobStorageCluster, s3Cluster, hdfsCluster, etc. +Possible values: +- Positive integer. +- 0 — All hosts in cluster. )", EXPERIMENTAL) \ DECLARE(Bool, allow_experimental_delta_kernel_rs, true, R"( Allow experimental delta-kernel-rs implementation. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 17a26f20045c..a00867e83e1a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -67,6 +67,12 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() /// controls new feature and it's 'true' by default, use 'false' as previous_value). /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) /// Note: please check if the key already exists to prevent duplicate entries. + addSettingsChanges(settings_changes_history, "25.6.2.20000", + { + // Altinity Antalya modifications atop of 25.6 + {"object_storage_cluster", "", "", "New setting"}, + {"object_storage_max_nodes", 0, 0, "New setting"}, + }); addSettingsChanges(settings_changes_history, "25.6", { {"output_format_native_use_flattened_dynamic_and_json_serialization", false, false, "Add flattened Dynamic/JSON serializations to Native format"}, diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index 7a686c2bb04b..68bfdd52dc03 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -43,6 +44,7 @@ namespace DatabaseDataLakeSetting extern const DatabaseDataLakeSettingsString storage_endpoint; extern const DatabaseDataLakeSettingsString oauth_server_uri; extern const DatabaseDataLakeSettingsBool vended_credentials; + extern const DatabaseDataLakeSettingsString object_storage_cluster; extern const DatabaseDataLakeSettingsString aws_access_key_id; @@ -428,9 +430,12 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con /// with_table_structure = false: because there will be /// no table structure in table definition AST. - StorageObjectStorage::Configuration::initialize(*configuration, args, context_copy, /* with_table_structure */false); + configuration->initialize(args, context_copy, /* with_table_structure */false); - return std::make_shared( + auto cluster_name = settings[DatabaseDataLakeSetting::object_storage_cluster].value; + + return std::make_shared( + cluster_name, configuration, configuration->createObjectStorage(context_copy, /* is_readonly */ false), context_copy, @@ -440,10 +445,7 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con /* comment */"", getFormatSettings(context_copy), LoadingStrictnessLevel::CREATE, - /* distributed_processing */false, - /* partition_by */nullptr, - /* is_table_function */false, - /* lazy_init */true); + /* partition_by */nullptr); } DatabaseTablesIteratorPtr DatabaseDataLake::getTablesIterator( diff --git a/src/Databases/DataLake/DatabaseDataLakeSettings.cpp b/src/Databases/DataLake/DatabaseDataLakeSettings.cpp index a6b15bc71334..046017c2fa8f 100644 --- a/src/Databases/DataLake/DatabaseDataLakeSettings.cpp +++ b/src/Databases/DataLake/DatabaseDataLakeSettings.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes DECLARE(String, aws_secret_access_key, "", "Key for AWS connection for Glue Catalog'", 0) \ DECLARE(String, region, "", "Region for Glue catalog", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ + DECLARE(String, object_storage_cluster, "", "Cluster for distributed requests", 0) \ #define LIST_OF_DATABASE_ICEBERG_SETTINGS(M, ALIAS) \ DATABASE_ICEBERG_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index 07a7099419ba..ba767959e991 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -MetadataStorageType metadataTypeFromString(const String & type) +MetadataStorageType metadataTypeFromString(const std::string & type) { auto check_type = Poco::toLower(type); if (check_type == "local") @@ -53,23 +53,47 @@ std::string DataSourceDescription::toString() const case DataSourceType::RAM: return "memory"; case DataSourceType::ObjectStorage: - { - switch (object_storage_type) - { - case ObjectStorageType::S3: - return "s3"; - case ObjectStorageType::HDFS: - return "hdfs"; - case ObjectStorageType::Azure: - return "azure_blob_storage"; - case ObjectStorageType::Local: - return "local_blob_storage"; - case ObjectStorageType::Web: - return "web"; - case ObjectStorageType::None: - return "none"; - } - } + return DB::toString(object_storage_type); } } + +ObjectStorageType objectStorageTypeFromString(const std::string & type) +{ + auto check_type = Poco::toLower(type); + if (check_type == "s3") + return ObjectStorageType::S3; + if (check_type == "hdfs") + return ObjectStorageType::HDFS; + if (check_type == "azure_blob_storage" || check_type == "azure") + return ObjectStorageType::Azure; + if (check_type == "local_blob_storage" || check_type == "local") + return ObjectStorageType::Local; + if (check_type == "web") + return ObjectStorageType::Web; + if (check_type == "none") + return ObjectStorageType::None; + + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG, + "Unknown object storage type: {}", type); +} + +std::string toString(ObjectStorageType type) +{ + switch (type) + { + case ObjectStorageType::S3: + return "s3"; + case ObjectStorageType::HDFS: + return "hdfs"; + case ObjectStorageType::Azure: + return "azure_blob_storage"; + case ObjectStorageType::Local: + return "local_blob_storage"; + case ObjectStorageType::Web: + return "web"; + case ObjectStorageType::None: + return "none"; + } +} + } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index bf7ef3d30eb0..1aa3ea19cbbe 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -34,8 +34,10 @@ enum class MetadataStorageType : uint8_t Memory, }; -MetadataStorageType metadataTypeFromString(const String & type); -String toString(DataSourceType data_source_type); +MetadataStorageType metadataTypeFromString(const std::string & type); + +ObjectStorageType objectStorageTypeFromString(const std::string & type); +std::string toString(ObjectStorageType type); struct DataSourceDescription { diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index a0e4e4999b72..e067cf87b067 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -732,9 +732,9 @@ void Cluster::initMisc() } } -std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard) const +std::unique_ptr Cluster::getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard, size_t max_hosts) const { - return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings, max_replicas_from_shard)}; + return std::unique_ptr{ new Cluster(ReplicasAsShardsTag{}, *this, settings, max_replicas_from_shard, max_hosts)}; } std::unique_ptr Cluster::getClusterWithSingleShard(size_t index) const @@ -783,7 +783,7 @@ void shuffleReplicas(std::vector & replicas, const Settings & } -Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard) +Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard, size_t max_hosts) { if (from.addresses_with_failover.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cluster is empty"); @@ -805,6 +805,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti if (address.is_local) info.local_addresses.push_back(address); + addresses_with_failover.emplace_back(Addresses({address})); auto pool = ConnectionPoolFactory::instance().get( static_cast(settings[Setting::distributed_connections_pool_size]), @@ -828,9 +829,6 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti info.per_replica_pools = {std::move(pool)}; info.default_database = address.default_database; - addresses_with_failover.emplace_back(Addresses{address}); - - slot_to_shard.insert(std::end(slot_to_shard), info.weight, shards_info.size()); shards_info.emplace_back(std::move(info)); } }; @@ -852,10 +850,37 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti secret = from.secret; name = from.name; + constrainShardInfoAndAddressesToMaxHosts(max_hosts); + + for (size_t i = 0; i < shards_info.size(); ++i) + slot_to_shard.insert(std::end(slot_to_shard), shards_info[i].weight, i); + initMisc(); } +void Cluster::constrainShardInfoAndAddressesToMaxHosts(size_t max_hosts) +{ + if (max_hosts == 0 || shards_info.size() <= max_hosts) + return; + + pcg64_fast gen{randomSeed()}; + std::shuffle(shards_info.begin(), shards_info.end(), gen); + shards_info.resize(max_hosts); + + AddressesWithFailover addresses_with_failover_; + + UInt32 shard_num = 0; + for (auto & shard_info : shards_info) + { + addresses_with_failover_.push_back(addresses_with_failover[shard_info.shard_num - 1]); + shard_info.shard_num = ++shard_num; + } + + addresses_with_failover.swap(addresses_with_failover_); +} + + Cluster::Cluster(Cluster::SubclusterTag, const Cluster & from, const std::vector & indices) { for (size_t index : indices) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b5a4c51c11db..f9b581034ef7 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -270,7 +270,7 @@ class Cluster std::unique_ptr getClusterWithMultipleShards(const std::vector & indices) const; /// Get a new Cluster that contains all servers (all shards with all replicas) from existing cluster as independent shards. - std::unique_ptr getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard = 0) const; + std::unique_ptr getClusterWithReplicasAsShards(const Settings & settings, size_t max_replicas_from_shard = 0, size_t max_hosts = 0) const; /// Returns false if cluster configuration doesn't allow to use it for cross-replication. /// NOTE: true does not mean, that it's actually a cross-replication cluster. @@ -296,7 +296,7 @@ class Cluster /// For getClusterWithReplicasAsShards implementation struct ReplicasAsShardsTag {}; - Cluster(ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard); + Cluster(ReplicasAsShardsTag, const Cluster & from, const Settings & settings, size_t max_replicas_from_shard, size_t max_hosts); void addShard( const Settings & settings, @@ -308,6 +308,9 @@ class Cluster ShardInfoInsertPathForInternalReplication insert_paths = {}, bool internal_replication = false); + /// Reduce size of cluster to max_hosts + void constrainShardInfoAndAddressesToMaxHosts(size_t max_hosts); + /// Inter-server secret String secret; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f4ef79b1c326..83409ee0ca13 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1949,8 +1949,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, auto table_function_ast = create.as_table_function->ptr(); auto table_function = TableFunctionFactory::instance().get(table_function_ast, getContext()); - if (!table_function->canBeUsedToCreateTable()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' cannot be used to create a table", table_function->getName()); + table_function->validateUseToCreateTable(); /// In case of CREATE AS table_function() query we should use global context /// in storage creation because there will be no query context on server startup diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index b1fd227dbabd..f145760a3a26 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -3,9 +3,12 @@ #include #include #include +#include +#include #include #include #include +#include namespace DB @@ -29,6 +32,21 @@ class AbstractFunction virtual ~Arguments() = default; virtual size_t size() const = 0; virtual std::unique_ptr at(size_t n) const = 0; + void skipArgument(size_t n) { skipped_indexes.insert(n); } + void unskipArguments() { skipped_indexes.clear(); } + size_t getRealIndex(size_t n) const + { + for (auto idx : skipped_indexes) + { + if (n < idx) + break; + ++n; + } + return n; + } + size_t skippedSize() const { return skipped_indexes.size(); } + private: + std::set skipped_indexes; }; virtual ~AbstractFunction() = default; @@ -75,14 +93,15 @@ class FunctionSecretArgumentsFinder { if (index >= function->arguments->size()) return; + auto real_index = function->arguments->getRealIndex(index); if (!result.count) { - result.start = index; + result.start = real_index; result.are_named = argument_is_named; } - chassert(index >= result.start); /// We always check arguments consecutively + chassert(real_index >= result.start); /// We always check arguments consecutively chassert(result.replacement.empty()); /// We shouldn't use replacement with masking other arguments - result.count = index + 1 - result.start; + result.count = real_index + 1 - result.start; if (!argument_is_named) result.are_named = false; } @@ -100,14 +119,18 @@ class FunctionSecretArgumentsFinder { findMongoDBSecretArguments(); } + else if (function->name() == "iceberg") + { + findIcebergFunctionSecretArguments(); + } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || - (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "gcs") || (function->name() == "icebergS3")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); } - else if (function->name() == "s3Cluster") + else if ((function->name() == "s3Cluster") || (function->name() == "icebergS3Cluster")) { /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ true); @@ -117,7 +140,7 @@ class FunctionSecretArgumentsFinder /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); } - else if (function->name() == "azureBlobStorageCluster") + else if ((function->name() == "azureBlobStorageCluster") || (function->name() == "icebergAzureCluster")) { /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); @@ -236,11 +259,18 @@ class FunctionSecretArgumentsFinder findSecretNamedArgument("secret_access_key", 1); return; } + if (is_cluster_function && isNamedCollectionName(1)) + { + /// s3Cluster(cluster, named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 2); + return; + } /// We should check other arguments first because we don't need to do any replacement in case of /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) { String second_arg; @@ -305,6 +335,48 @@ class FunctionSecretArgumentsFinder markSecretArgument(url_arg_idx + 4); } + std::string findIcebergStorageType() + { + std::string storage_type = "s3"; + + size_t count = function->arguments->size(); + if (!count) + return storage_type; + + auto storage_type_idx = findNamedArgument(&storage_type, "storage_type"); + if (storage_type_idx != -1) + { + storage_type = Poco::toLower(storage_type); + function->arguments->skipArgument(storage_type_idx); + } + else if (isNamedCollectionName(0)) + { + std::string collection_name; + if (function->arguments->at(0)->tryGetString(&collection_name, true)) + { + NamedCollectionPtr collection = NamedCollectionFactory::instance().tryGet(collection_name); + if (collection && collection->has("storage_type")) + { + storage_type = Poco::toLower(collection->get("storage_type")); + } + } + } + + return storage_type; + } + + void findIcebergFunctionSecretArguments() + { + auto storage_type = findIcebergStorageType(); + + if (storage_type == "s3") + findS3FunctionSecretArguments(false); + else if (storage_type == "azure") + findAzureBlobStorageFunctionSecretArguments(false); + + function->arguments->unskipArguments(); + } + bool maskAzureConnectionString(ssize_t url_arg_idx, bool argument_is_named = false, size_t start = 0) { String url_arg; @@ -328,7 +400,7 @@ class FunctionSecretArgumentsFinder if (RE2::Replace(&url_arg, account_key_pattern, "AccountKey=[HIDDEN]\\1")) { chassert(result.count == 0); /// We shouldn't use replacement with masking other arguments - result.start = url_arg_idx; + result.start = function->arguments->getRealIndex(url_arg_idx); result.are_named = argument_is_named; result.count = 1; result.replacement = url_arg; @@ -487,6 +559,7 @@ class FunctionSecretArgumentsFinder void findTableEngineSecretArguments() { const String & engine_name = function->name(); + if (engine_name == "ExternalDistributed") { /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') @@ -504,10 +577,13 @@ class FunctionSecretArgumentsFinder { findMongoDBSecretArguments(); } + else if (engine_name == "Iceberg") + { + findIcebergTableEngineSecretArguments(); + } else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || (engine_name == "DeltaLake") || (engine_name == "Hudi") - || (engine_name == "Iceberg") || (engine_name == "IcebergS3") - || (engine_name == "S3Queue")) + || (engine_name == "IcebergS3") || (engine_name == "S3Queue")) { /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) findS3TableEngineSecretArguments(); @@ -516,7 +592,7 @@ class FunctionSecretArgumentsFinder { findURLSecretArguments(); } - else if (engine_name == "AzureBlobStorage" || engine_name == "AzureQueue") + else if (engine_name == "AzureBlobStorage" || engine_name == "AzureQueue" || engine_name == "IcebergAzure") { findAzureBlobStorageTableEngineSecretArguments(); } @@ -612,6 +688,18 @@ class FunctionSecretArgumentsFinder markSecretArgument(url_arg_idx + 4); } + void findIcebergTableEngineSecretArguments() + { + auto storage_type = findIcebergStorageType(); + + if (storage_type == "s3") + findS3TableEngineSecretArguments(); + else if (storage_type == "azure") + findAzureBlobStorageTableEngineSecretArguments(); + + function->arguments->unskipArguments(); + } + void findDatabaseEngineSecretArguments() { const String & engine_name = function->name(); diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index a260c0d58da6..3624d7a7e87b 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -54,10 +54,13 @@ class FunctionAST : public AbstractFunction { public: explicit ArgumentsAST(const ASTs * arguments_) : arguments(arguments_) {} - size_t size() const override { return arguments ? arguments->size() : 0; } + size_t size() const override + { /// size withous skipped indexes + return arguments ? arguments->size() - skippedSize() : 0; + } std::unique_ptr at(size_t n) const override - { - return std::make_unique(arguments->at(n).get()); + { /// n is relative index, some can be skipped + return std::make_unique(arguments->at(getRealIndex(n)).get()); } private: const ASTs * arguments = nullptr; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 6b16465db7bf..af94e46869b1 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -36,7 +36,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 006b9e377f4b..028fdd61a26f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -65,6 +65,9 @@ class RestorerFromBackup; class ConditionSelectivityEstimator; +class IObjectStorage; +using ObjectStoragePtr = std::shared_ptr; + class ActionsDAG; /** Storage. Describes the table. Responsible for diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 88f04e1e9c64..978b043747a3 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -40,6 +40,12 @@ namespace Setting extern const SettingsBool parallel_replicas_local_plan; extern const SettingsString cluster_for_parallel_replicas; extern const SettingsNonZeroUInt64 max_parallel_replicas; + extern const SettingsUInt64 object_storage_max_nodes; +} + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; } IStorageCluster::IStorageCluster( @@ -128,13 +134,21 @@ void IStorageCluster::read( SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum processed_stage, - size_t /*max_block_size*/, - size_t /*num_streams*/) + size_t max_block_size, + size_t num_streams) { + auto cluster_name_from_settings = getClusterName(context); + + if (cluster_name_from_settings.empty()) + { + readFallBackToPure(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); + return; + } + storage_snapshot->check(column_names); updateBeforeRead(context); - auto cluster = getCluster(context); + auto cluster = getClusterImpl(context, cluster_name_from_settings, context->getSettingsRef()[Setting::object_storage_max_nodes]); /// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*) @@ -181,6 +195,20 @@ void IStorageCluster::read( query_plan.addStep(std::move(reading)); } +SinkToStoragePtr IStorageCluster::write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) +{ + auto cluster_name_from_settings = getClusterName(context); + + if (cluster_name_from_settings.empty()) + return writeFallBackToPure(query, metadata_snapshot, context, async_insert); + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not supported by storage {}", getName()); +} + void ReadFromCluster::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; @@ -273,9 +301,9 @@ ContextPtr ReadFromCluster::updateSettings(const Settings & settings) return new_context; } -ClusterPtr IStorageCluster::getCluster(ContextPtr context) const +ClusterPtr IStorageCluster::getClusterImpl(ContextPtr context, const String & cluster_name_, size_t max_hosts) { - return context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef()); + return context->getCluster(cluster_name_)->getClusterWithReplicasAsShards(context->getSettingsRef(), /* max_replicas_from_shard */ 0, max_hosts); } } diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index 6017613c7bea..a119f5fc676d 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -30,10 +30,17 @@ class IStorageCluster : public IStorage SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum processed_stage, - size_t /*max_block_size*/, - size_t /*num_streams*/) override; + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; + + ClusterPtr getCluster(ContextPtr context) const { return getClusterImpl(context, cluster_name); } - ClusterPtr getCluster(ContextPtr context) const; /// Query is needed for pruning by virtual columns (_file, _path) virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, size_t number_of_replicas) const = 0; @@ -44,11 +51,38 @@ class IStorageCluster : public IStorage bool supportsOptimizationToSubcolumns() const override { return false; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } + const String & getOriginalClusterName() const { return cluster_name; } + virtual String getClusterName(ContextPtr /* context */) const { return getOriginalClusterName(); } + protected: virtual void updateBeforeRead(const ContextPtr &) {} virtual void updateQueryToSendIfNeeded(ASTPtr & /*query*/, const StorageSnapshotPtr & /*storage_snapshot*/, const ContextPtr & /*context*/) {} + virtual void readFallBackToPure( + QueryPlan & /* query_plan */, + const Names & /* column_names */, + const StorageSnapshotPtr & /* storage_snapshot */, + SelectQueryInfo & /* query_info */, + ContextPtr /* context */, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + size_t /* num_streams */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readFallBackToPure is not supported by storage {}", getName()); + } + + virtual SinkToStoragePtr writeFallBackToPure( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*context*/, + bool /*async_insert*/) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method writeFallBackToPure is not supported by storage {}", getName()); + } + private: + static ClusterPtr getClusterImpl(ContextPtr context, const String & cluster_name_, size_t max_hosts = 0); + LoggerPtr log; String cluster_name; }; diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index eec50b5aadb4..e4cc3cef8d8b 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -56,6 +56,7 @@ const std::unordered_set optional_configuration_keys = { "account_key", "connection_string", "storage_account_url", + "storage_type", }; void StorageAzureConfiguration::check(ContextPtr context) const @@ -128,8 +129,6 @@ void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & coll String connection_url; String container_name; - std::optional account_name; - std::optional account_key; if (collection.has("connection_string")) connection_url = collection.get("connection_string"); @@ -145,9 +144,9 @@ void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & coll if (collection.has("account_key")) account_key = collection.get("account_key"); - structure = collection.getOrDefault("structure", "auto"); - format = collection.getOrDefault("format", format); - compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + setStructure(collection.getOrDefault("structure", "auto")); + setFormat(collection.getOrDefault("format", getFormat())); + setCompressionMethod(collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto"))); blobs_paths = {blob_path}; connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, context); @@ -169,14 +168,10 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, std::unordered_map engine_args_to_idx; - String connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); String container_name = checkAndGetLiteralArgument(engine_args[1], "container"); blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - std::optional account_name; - std::optional account_key; - auto is_format_arg = [] (const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(Poco::toLower(s)); @@ -187,12 +182,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); if (is_format_arg(fourth_arg)) { - format = fourth_arg; + setFormat(fourth_arg); } else { if (with_structure) - structure = fourth_arg; + setStructure(fourth_arg); else throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -204,8 +199,8 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); if (is_format_arg(fourth_arg)) { - format = fourth_arg; - compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + setFormat(fourth_arg); + setCompressionMethod(checkAndGetLiteralArgument(engine_args[4], "compression")); } else { @@ -220,9 +215,9 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, { if (with_structure) { - format = fourth_arg; - compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - structure = checkAndGetLiteralArgument(engine_args[5], "structure"); + setFormat(fourth_arg); + setCompressionMethod(checkAndGetLiteralArgument(engine_args[4], "compression")); + setStructure(checkAndGetLiteralArgument(engine_args[5], "structure")); } else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); @@ -234,12 +229,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/structure"); if (is_format_arg(sixth_arg)) { - format = sixth_arg; + setFormat(sixth_arg); } else { if (with_structure) - structure = sixth_arg; + setStructure(sixth_arg); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); } @@ -258,8 +253,8 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - format = sixth_arg; - compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); + setFormat(sixth_arg); + setCompressionMethod(checkAndGetLiteralArgument(engine_args[6], "compression")); } else if (with_structure && engine_args.size() == 8) { @@ -269,9 +264,9 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); - format = sixth_arg; - compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - structure = checkAndGetLiteralArgument(engine_args[7], "structure"); + setFormat(sixth_arg); + setCompressionMethod (checkAndGetLiteralArgument(engine_args[6], "compression")); + setStructure(checkAndGetLiteralArgument(engine_args[7], "structure")); } blobs_paths = {blob_path}; @@ -442,6 +437,22 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded( } } +ASTPtr StorageAzureConfiguration::createArgsWithAccessData() const +{ + auto arguments = std::make_shared(); + + arguments->children.push_back(std::make_shared(connection_params.endpoint.storage_account_url)); + arguments->children.push_back(std::make_shared(connection_params.endpoint.container_name)); + arguments->children.push_back(std::make_shared(blob_path)); + if (account_name && account_key) + { + arguments->children.push_back(std::make_shared(*account_name)); + arguments->children.push_back(std::make_shared(*account_key)); + } + + return arguments; +} + } #endif diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 157da6047ce2..f764f12864cd 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -78,6 +78,8 @@ class StorageAzureConfiguration : public StorageObjectStorage::Configuration ContextPtr context, bool with_structure) override; + ASTPtr createArgsWithAccessData() const override; + protected: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; @@ -85,6 +87,8 @@ class StorageAzureConfiguration : public StorageObjectStorage::Configuration std::string blob_path; std::vector blobs_paths; AzureBlobStorage::ConnectionParams connection_params; + std::optional account_name; + std::optional account_key; }; } diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index a1a4c8a852bd..d7e64afb97b9 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -13,7 +14,11 @@ #include #include #include -#include "Storages/ColumnsDescription.h" +#include +#include +#include +#include +#include #include #include @@ -228,15 +233,252 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl using StorageS3IcebergConfiguration = DataLakeConfiguration; #endif -#if USE_AZURE_BLOB_STORAGE +# if USE_AZURE_BLOB_STORAGE using StorageAzureIcebergConfiguration = DataLakeConfiguration; #endif -#if USE_HDFS +# if USE_HDFS using StorageHDFSIcebergConfiguration = DataLakeConfiguration; #endif using StorageLocalIcebergConfiguration = DataLakeConfiguration; + +/// Class detects storage type by `storage_type` parameter if exists +/// and uses appropriate implementation - S3, Azure, HDFS or Local +class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, public std::enable_shared_from_this +{ + friend class StorageObjectStorage::Configuration; + +public: + explicit StorageIcebergConfiguration(DataLakeStorageSettingsPtr settings_) : settings(settings_) {} + + ObjectStorageType getType() const override { return getImpl().getType(); } + + std::string getTypeName() const override { return getImpl().getTypeName(); } + std::string getEngineName() const override { return getImpl().getEngineName(); } + std::string getNamespaceType() const override { return getImpl().getNamespaceType(); } + + Path getPath() const override { return getImpl().getPath(); } + void setPath(const Path & path) override { getImpl().setPath(path); } + + const Paths & getPaths() const override { return getImpl().getPaths(); } + void setPaths(const Paths & paths) override { getImpl().setPaths(paths); } + + String getDataSourceDescription() const override { return getImpl().getDataSourceDescription(); } + String getNamespace() const override { return getImpl().getNamespace(); } + + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr & context) const override + { return getImpl().getQuerySettings(context); } + + void addStructureAndFormatToArgsIfNeeded( + ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) override + { getImpl().addStructureAndFormatToArgsIfNeeded(args, structure_, format_, context, with_structure); } + + std::string getPathWithoutGlobs() const override { return getImpl().getPathWithoutGlobs(); } + + bool isArchive() const override { return getImpl().isArchive(); } + std::string getPathInArchive() const override { return getImpl().getPathInArchive(); } + + void check(ContextPtr context) const override { getImpl().check(context); } + void validateNamespace(const String & name) const override { getImpl().validateNamespace(name); } + + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override + { return getImpl().createObjectStorage(context, is_readonly); } + //StorageObjectStorage::ConfigurationPtr clone() override { return getImpl().clone(); } + bool isStaticConfiguration() const override { return getImpl().isStaticConfiguration(); } + + bool isDataLakeConfiguration() const override { return getImpl().isDataLakeConfiguration(); } + + bool hasExternalDynamicMetadata() override { return getImpl().hasExternalDynamicMetadata(); } + + std::shared_ptr getInitialSchemaByPath(const String & path) const override + { return getImpl().getInitialSchemaByPath(path); } + + std::shared_ptr getSchemaTransformer(const String & data_path) const override + { return getImpl().getSchemaTransformer(data_path); } + + //ColumnsDescription updateAndGetCurrentSchema(ObjectStoragePtr object_storage, ContextPtr context) override + // { return getImpl().updateAndGetCurrentSchema(object_storage, context); } + + ReadFromFormatInfo prepareReadingFromFormat( + ObjectStoragePtr object_storage, + const Strings & requested_columns, + const StorageSnapshotPtr & storage_snapshot, + bool supports_subset_of_columns, + ContextPtr local_context) override + { + return getImpl().prepareReadingFromFormat( + object_storage, + requested_columns, + storage_snapshot, + supports_subset_of_columns, + local_context); + } + + std::optional tryGetTableStructureFromMetadata() const override + { return getImpl().tryGetTableStructureFromMetadata(); } + + bool update( + ObjectStoragePtr object_storage_ptr, + ContextPtr context, + bool if_not_updated_before, + bool check_consistent_with_previous_metadata) override + { + return getImpl().update(object_storage_ptr, context, if_not_updated_before, check_consistent_with_previous_metadata); + } + + void initialize( + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) override + { + createDynamicConfiguration(engine_args, local_context); + getImpl().initialize(engine_args, local_context, with_table_structure); + } + + ASTPtr createArgsWithAccessData() const override + { + return getImpl().createArgsWithAccessData(); + } + + const String & getFormat() const override { return getImpl().getFormat(); } + const String & getCompressionMethod() const override { return getImpl().getCompressionMethod(); } + const String & getStructure() const override { return getImpl().getStructure(); } + + void setFormat(const String & format_) override { getImpl().setFormat(format_); } + void setCompressionMethod(const String & compression_method_) override { getImpl().setCompressionMethod(compression_method_); } + void setStructure(const String & structure_) override { getImpl().setStructure(structure_); } + +protected: + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override + { return getImpl().fromNamedCollection(collection, context); } + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override + { return getImpl().fromAST(args, context, with_structure); } + + /// Find storage_type argument and remove it from args if exists. + /// Return storage type. + ObjectStorageType extractDynamicStorageType(ASTs & args, ContextPtr context, ASTPtr * type_arg = nullptr) const override + { + static const auto storage_type_name = "storage_type"; + + if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) + { + if (named_collection->has(storage_type_name)) + { + return objectStorageTypeFromString(named_collection->get(storage_type_name)); + } + } + + auto type_it = args.end(); + + /// S3 by default for backward compatibility + /// Iceberg without storage_type == IcebergS3 + ObjectStorageType type = ObjectStorageType::S3; + + for (auto arg_it = args.begin(); arg_it != args.end(); ++arg_it) + { + const auto * type_ast_function = (*arg_it)->as(); + + if (type_ast_function && type_ast_function->name == "equals" + && type_ast_function->arguments && type_ast_function->arguments->children.size() == 2) + { + auto name = type_ast_function->arguments->children[0]->as(); + + if (name && name->name() == storage_type_name) + { + if (type_it != args.end()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "DataLake can have only one key-value argument: storage_type='type'."); + } + + auto value = type_ast_function->arguments->children[1]->as(); + + if (!value) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "DataLake parameter 'storage_type' has wrong type, string literal expected."); + } + + if (value->value.getType() != Field::Types::String) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "DataLake parameter 'storage_type' has wrong value type, string expected."); + } + + type = objectStorageTypeFromString(value->value.safeGet()); + + type_it = arg_it; + } + } + } + + if (type_it != args.end()) + { + if (type_arg) + *type_arg = *type_it; + args.erase(type_it); + } + + return type; + } + + void createDynamicConfiguration(ASTs & args, ContextPtr context) + { + ObjectStorageType type = extractDynamicStorageType(args, context); + createDynamicStorage(type); + } + +private: + inline StorageObjectStorage::Configuration & getImpl() const + { + if (!impl) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Dynamic DataLake storage not initialized"); + + return *impl; + } + + void createDynamicStorage(ObjectStorageType type) + { + if (impl) + { + if (impl->getType() == type) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't change datalake engine storage"); + } + + switch (type) + { +# if USE_AWS_S3 + case ObjectStorageType::S3: + impl = std::make_unique(settings); + break; +# endif +# if USE_AZURE_BLOB_STORAGE + case ObjectStorageType::Azure: + impl = std::make_unique(settings); + break; +# endif +# if USE_HDFS + case ObjectStorageType::HDFS: + impl = std::make_unique(settings); + break; +# endif + case ObjectStorageType::Local: + impl = std::make_unique(settings); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsuported DataLake storage {}", type); + } + } + + std::shared_ptr impl; + DataLakeStorageSettingsPtr settings; +}; #endif #if USE_PARQUET diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index a1f47e34be6e..774e39554edc 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -44,7 +44,7 @@ Strings HudiMetadata::getDataFilesImpl() const { auto configuration_ptr = configuration.lock(); auto log = getLogger("HudiMetadata"); - const auto keys = listFiles(*object_storage, *configuration_ptr, "", Poco::toLower(configuration_ptr->format)); + const auto keys = listFiles(*object_storage, *configuration_ptr, "", Poco::toLower(configuration_ptr->getFormat())); using Partition = std::string; using FileID = std::string; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql new file mode 100644 index 000000000000..291582fd1eb3 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql @@ -0,0 +1,2 @@ +ATTACH DATABASE INFORMATION_SCHEMA +ENGINE = Memory diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql new file mode 100644 index 000000000000..3b99a1d7af8f --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql @@ -0,0 +1,2 @@ +ATTACH DATABASE _ UUID 'd9e2d817-49d1-4148-9aad-73a382733129' +ENGINE = Atomic diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql new file mode 100644 index 000000000000..0a9ae99ee054 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql @@ -0,0 +1,3 @@ +ATTACH DATABASE ice +ENGINE = DataLakeCatalog('https://iceberg-catalog.aws-us-west-2.dev.altinity.cloud') +SETTINGS catalog_type = 'rest', auth_header = 'Authorization: Bearer etxkehqze7esafs9qw07lcrww5nd0iqo', warehouse = 's3://aws-st-2-fs5vug37-iceberg' diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql new file mode 100644 index 000000000000..6cea934b49d2 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql @@ -0,0 +1,2 @@ +ATTACH DATABASE information_schema +ENGINE = Memory diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system new file mode 120000 index 000000000000..86be5e92dc38 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system @@ -0,0 +1 @@ +./store/41c/41c68de0-0dd8-4d7f-8c61-df82ac3de66d/ \ No newline at end of file diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql new file mode 100644 index 000000000000..3d990f9689b2 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql @@ -0,0 +1,2 @@ +ATTACH DATABASE _ UUID '41c68de0-0dd8-4d7f-8c61-df82ac3de66d' +ENGINE = Atomic diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml b/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml new file mode 100644 index 000000000000..790297966d1b --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml @@ -0,0 +1,44 @@ + + + + + + trace + true + + + 8123 + 9000 + 9004 + + ./ + + true + + + + + + + ::/0 + + + default + default + + 1 + 1 + + + + + + + + + + + diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/status b/src/Storages/ObjectStorage/DataLakes/Iceberg/status new file mode 100644 index 000000000000..bfed01e9836c --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/status @@ -0,0 +1,3 @@ +PID: 1058703 +Started at: 2025-06-20 12:17:04 +Revision: 54498 diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid b/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid new file mode 100644 index 000000000000..2f29a5ba4076 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid @@ -0,0 +1 @@ +24fc6e4a-fd07-4ba7-88f0-e779b4571020 \ No newline at end of file diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 73fbafcb388b..065e23426727 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -103,23 +103,23 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit if (args.size() > 1) { - format = checkAndGetLiteralArgument(args[1], "format_name"); + setFormat(checkAndGetLiteralArgument(args[1], "format_name")); } if (with_structure) { if (args.size() > 2) { - structure = checkAndGetLiteralArgument(args[2], "structure"); + setStructure(checkAndGetLiteralArgument(args[2], "structure")); } if (args.size() > 3) { - compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + setCompressionMethod(checkAndGetLiteralArgument(args[3], "compression_method")); } } else if (args.size() > 2) { - compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + setCompressionMethod(checkAndGetLiteralArgument(args[2], "compression_method")); } setURL(url_str); @@ -135,10 +135,10 @@ void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & colle else url_str = collection.get("url"); - format = collection.getOrDefault("format", "auto"); - compression_method = collection.getOrDefault("compression_method", - collection.getOrDefault("compression", "auto")); - structure = collection.getOrDefault("structure", "auto"); + setFormat(collection.getOrDefault("format", "auto")); + setCompressionMethod(collection.getOrDefault("compression_method", + collection.getOrDefault("compression", "auto"))); + setStructure(collection.getOrDefault("structure", "auto")); setURL(url_str); } @@ -228,6 +228,13 @@ void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded( } } +ASTPtr StorageHDFSConfiguration::createArgsWithAccessData() const +{ + auto arguments = std::make_shared(); + arguments->children.push_back(std::make_shared(url + path)); + return arguments; +} + } #endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 698dd4a8e354..ae39665f0400 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -63,6 +63,8 @@ class StorageHDFSConfiguration : public StorageObjectStorage::Configuration ContextPtr context, bool with_structure) override; + ASTPtr createArgsWithAccessData() const override; + private: void fromNamedCollection(const NamedCollection &, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index bae84714273c..c6dfa3318331 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -24,9 +24,9 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) { path = collection.get("path"); - format = collection.getOrDefault("format", "auto"); - compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); - structure = collection.getOrDefault("structure", "auto"); + setFormat(collection.getOrDefault("format", "auto")); + setCompressionMethod(collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto"))); + setStructure(collection.getOrDefault("structure", "auto")); paths = {path}; } @@ -46,23 +46,23 @@ void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool wi if (args.size() > 1) { - format = checkAndGetLiteralArgument(args[1], "format_name"); + setFormat(checkAndGetLiteralArgument(args[1], "format_name")); } if (with_structure) { if (args.size() > 2) { - structure = checkAndGetLiteralArgument(args[2], "structure"); + setStructure(checkAndGetLiteralArgument(args[2], "structure")); } if (args.size() > 3) { - compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + setCompressionMethod(checkAndGetLiteralArgument(args[3], "compression_method")); } } else if (args.size() > 2) { - compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + setCompressionMethod(checkAndGetLiteralArgument(args[2], "compression_method")); } paths = {path}; } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index bc2a43af9fd1..92527050936c 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -39,8 +39,8 @@ ReadBufferIterator::ReadBufferIterator( , read_keys(read_keys_) , prev_read_keys_size(read_keys_.size()) { - if (configuration->format != "auto") - format = configuration->format; + if (configuration->getFormat() != "auto") + format = configuration->getFormat(); } SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const @@ -153,7 +153,7 @@ std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); auto impl = StorageObjectStorageSource::createReadBuffer(*current_object_info, object_storage, context, getLogger("ReadBufferIterator")); - const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); + const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->getCompressionMethod()); const auto zstd_window = static_cast(context->getSettingsRef()[Setting::zstd_window_log_max]); return wrapReadBufferWithCompressionMethod(std::move(impl), compression_method, zstd_window); @@ -269,13 +269,13 @@ ReadBufferIterator::Data ReadBufferIterator::next() using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; if (const auto * object_info_in_archive = dynamic_cast(current_object_info.get())) { - compression_method = chooseCompressionMethod(filename, configuration->compression_method); + compression_method = chooseCompressionMethod(filename, configuration->getCompressionMethod()); const auto & archive_reader = object_info_in_archive->archive_reader; read_buf = archive_reader->readFile(object_info_in_archive->path_in_archive, /*throw_on_not_found=*/true); } else { - compression_method = chooseCompressionMethod(filename, configuration->compression_method); + compression_method = chooseCompressionMethod(filename, configuration->getCompressionMethod()); read_buf = StorageObjectStorageSource::createReadBuffer(*current_object_info, object_storage, getContext(), getLogger("ReadBufferIterator")); } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 60050eb249f4..84c3450a8dab 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -75,7 +75,8 @@ static const std::unordered_set optional_configuration_keys = "max_single_part_upload_size", "max_connections", "expiration_window_seconds", - "no_sign_request" + "no_sign_request", + "storage_type", }; String StorageS3Configuration::getDataSourceDescription() const @@ -171,9 +172,9 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect auth_settings[S3AuthSetting::expiration_window_seconds] = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); auth_settings[S3AuthSetting::session_token] = collection.getOrDefault("session_token", ""); - format = collection.getOrDefault("format", format); - compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); - structure = collection.getOrDefault("structure", "auto"); + setFormat(collection.getOrDefault("format", getFormat())); + setCompressionMethod(collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto"))); + setStructure(collection.getOrDefault("structure", "auto")); request_settings = S3::S3RequestSettings(collection, settings, /* validate_settings */true); @@ -356,18 +357,18 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ if (engine_args_to_idx.contains("format")) { - format = checkAndGetLiteralArgument(args[engine_args_to_idx["format"]], "format"); + auto format_ = checkAndGetLiteralArgument(args[engine_args_to_idx["format"]], "format"); /// Set format to configuration only of it's not 'auto', /// because we can have default format set in configuration. - if (format != "auto") - format = format; + if (format_ != "auto") + setFormat(format_); } if (engine_args_to_idx.contains("structure")) - structure = checkAndGetLiteralArgument(args[engine_args_to_idx["structure"]], "structure"); + setStructure(checkAndGetLiteralArgument(args[engine_args_to_idx["structure"]], "structure")); if (engine_args_to_idx.contains("compression_method")) - compression_method = checkAndGetLiteralArgument(args[engine_args_to_idx["compression_method"]], "compression_method"); + setCompressionMethod(checkAndGetLiteralArgument(args[engine_args_to_idx["compression_method"]], "compression_method")); if (engine_args_to_idx.contains("access_key_id")) auth_settings[S3AuthSetting::access_key_id] = checkAndGetLiteralArgument(args[engine_args_to_idx["access_key_id"]], "access_key_id"); @@ -578,6 +579,30 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded( } } +ASTPtr StorageS3Configuration::createArgsWithAccessData() const +{ + auto arguments = std::make_shared(); + + arguments->children.push_back(std::make_shared(url.uri_str)); + if (auth_settings[S3AuthSetting::no_sign_request]) + { + arguments->children.push_back(std::make_shared("NOSIGN")); + } + else + { + arguments->children.push_back(std::make_shared(auth_settings[S3AuthSetting::access_key_id].value)); + arguments->children.push_back(std::make_shared(auth_settings[S3AuthSetting::secret_access_key].value)); + if (!auth_settings[S3AuthSetting::session_token].value.empty()) + arguments->children.push_back(std::make_shared(auth_settings[S3AuthSetting::session_token].value)); + if (getFormat() != "auto") + arguments->children.push_back(std::make_shared(getFormat())); + if (!getCompressionMethod().empty()) + arguments->children.push_back(std::make_shared(getCompressionMethod())); + } + + return arguments; +} + } #endif diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 52fcc45390c0..ea60e20eca00 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -95,6 +95,8 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration ContextPtr context, bool with_structure) override; + ASTPtr createArgsWithAccessData() const override; + private: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 3409e4c2ea8b..008f830854c2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -58,6 +58,9 @@ String StorageObjectStorage::getPathSample(ContextPtr context) if (context->getSettingsRef()[Setting::use_hive_partitioning]) local_distributed_processing = false; + if (!configuration->isArchive() && !configuration->isPathWithGlobs() && !local_distributed_processing) + return configuration->getPath(); + auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, query_settings, @@ -71,9 +74,6 @@ String StorageObjectStorage::getPathSample(ContextPtr context) {} // file_progress_callback ); - if (!configuration->isArchive() && !configuration->isPathWithGlobs() && !local_distributed_processing) - return configuration->getPath(); - if (auto file = file_iterator->next(0)) return file->getPath(); return ""; @@ -101,7 +101,7 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - const bool need_resolve_columns_or_format = columns_.empty() || (configuration->format == "auto"); + const bool need_resolve_columns_or_format = columns_.empty() || (configuration->getFormat() == "auto"); const bool need_resolve_sample_path = context->getSettingsRef()[Setting::use_hive_partitioning] && !configuration->withPartitionWildcard() && !configuration->isDataLakeConfiguration(); @@ -141,7 +141,7 @@ StorageObjectStorage::StorageObjectStorage( std::string sample_path; ColumnsDescription columns{columns_}; if (need_resolve_columns_or_format) - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context); + resolveSchemaAndFormat(columns, object_storage, configuration, format_settings, sample_path, context); else validateSupportedColumns(columns, *configuration); @@ -182,17 +182,17 @@ String StorageObjectStorage::getName() const bool StorageObjectStorage::prefersLargeBlocks() const { - return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration->format); + return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration->getFormat()); } bool StorageObjectStorage::parallelizeOutputAfterReading(ContextPtr context) const { - return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration->format, context); + return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration->getFormat(), context); } bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->getFormat(), context, format_settings); } bool StorageObjectStorage::Configuration::update( ///NOLINT @@ -332,7 +332,7 @@ class ReadFromObjectStorageStep : public SourceStepWithFilter num_streams = 1; } - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + const size_t max_parsing_threads = (distributed_processing || num_streams >= max_threads) ? 1 : (max_threads / std::max(num_streams, 1ul)); for (size_t i = 0; i < num_streams; ++i) { @@ -586,7 +586,7 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( { ObjectInfos read_keys; auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); - auto schema = readSchemaFromFormat(configuration->format, format_settings, *iterator, context); + auto schema = readSchemaFromFormat(configuration->getFormat(), format_settings, *iterator, context); sample_path = iterator->getLastFilePath(); return schema; } @@ -607,7 +607,7 @@ std::string StorageObjectStorage::resolveFormatFromData( std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + ConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context) @@ -616,13 +616,13 @@ std::pair StorageObjectStorage::resolveSchemaAn auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); sample_path = iterator->getLastFilePath(); - configuration->format = format; + configuration->setFormat(format); return std::pair(columns, format); } void StorageObjectStorage::addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const { - configuration->addStructureAndFormatToArgsIfNeeded(args, "", configuration->format, context, /*with_structure=*/false); + configuration->addStructureAndFormatToArgsIfNeeded(args, "", configuration->getFormat(), context, /*with_structure=*/false); } SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name) @@ -657,34 +657,33 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c } void StorageObjectStorage::Configuration::initialize( - Configuration & configuration_to_initialize, ASTs & engine_args, ContextPtr local_context, bool with_table_structure) { if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration_to_initialize.fromNamedCollection(*named_collection, local_context); + fromNamedCollection(*named_collection, local_context); else - configuration_to_initialize.fromAST(engine_args, local_context, with_table_structure); + fromAST(engine_args, local_context, with_table_structure); - if (configuration_to_initialize.format == "auto") + if (format == "auto") { - if (configuration_to_initialize.isDataLakeConfiguration()) + if (isDataLakeConfiguration()) { - configuration_to_initialize.format = "Parquet"; + format = "Parquet"; } else { - configuration_to_initialize.format + format = FormatFactory::instance() - .tryGetFormatFromFileName(configuration_to_initialize.isArchive() ? configuration_to_initialize.getPathInArchive() : configuration_to_initialize.getPath()) + .tryGetFormatFromFileName(isArchive() ? getPathInArchive() : getPath()) .value_or("auto"); } } else - FormatFactory::instance().checkFormatName(configuration_to_initialize.format); + FormatFactory::instance().checkFormatName(format); - configuration_to_initialize.initialized = true; + initialized = true; } void StorageObjectStorage::Configuration::check(ContextPtr) const diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 4ea70da8a638..668d268caaba 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -134,7 +134,7 @@ class StorageObjectStorage : public IStorage static std::pair resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + ConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context); @@ -189,8 +189,7 @@ class StorageObjectStorage::Configuration using Paths = std::vector; /// Initialize configuration from either AST or NamedCollection. - static void initialize( - Configuration & configuration_to_initialize, + virtual void initialize( ASTs & engine_args, ContextPtr local_context, bool with_table_structure); @@ -285,14 +284,32 @@ class StorageObjectStorage::Configuration throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataLakeSettings() is not implemented for configuration type {}", getTypeName()); } + /// Create arguments for table function with path and access parameters + virtual ASTPtr createArgsWithAccessData() const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method createArgsWithAccessData is not supported by storage {}", getEngineName()); + } + + virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; + virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + + virtual ObjectStorageType extractDynamicStorageType(ASTs & /* args */, ContextPtr /* context */, ASTPtr * /* type_arg */ = nullptr) const + { return ObjectStorageType::None; } + + virtual const String & getFormat() const { return format; } + virtual const String & getCompressionMethod() const { return compression_method; } + virtual const String & getStructure() const { return structure; } + + virtual void setFormat(const String & format_) { format = format_; } + virtual void setCompressionMethod(const String & compression_method_) { compression_method = compression_method_; } + virtual void setStructure(const String & structure_) { structure = structure_; } + +private: String format = "auto"; String compression_method = "auto"; String structure = "auto"; protected: - virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; - virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; - void assertInitialized() const; bool initialized = false; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 87d32c0c0a44..f844d082a5c2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -7,8 +7,15 @@ #include #include +#include +#include +#include +#include +#include #include #include +#include +#include #include #include @@ -16,24 +23,31 @@ #include #include - namespace DB { namespace Setting { extern const SettingsBool use_hive_partitioning; + extern const SettingsString object_storage_cluster; } namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int UNKNOWN_FUNCTION; + extern const int NOT_IMPLEMENTED; } + String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metadata, ContextPtr context) { auto query_settings = configuration->getQuerySettings(context); /// We don't want to throw an exception if there are no files with specified path. query_settings.throw_on_zero_files_match = false; + + if (!configuration->isArchive() && !configuration->isPathWithGlobs()) + return configuration->getPath(); + auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, query_settings, @@ -49,6 +63,7 @@ String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metada if (auto file = file_iterator->next(0)) return file->getPath(); + return ""; } @@ -56,14 +71,20 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( const String & cluster_name_, ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, + ContextPtr context_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - ContextPtr context_) + const String & comment_, + std::optional format_settings_, + LoadingStrictnessLevel mode_, + ASTPtr partition_by_ +) : IStorageCluster( cluster_name_, table_id_, getLogger(fmt::format("{}({})", configuration_->getEngineName(), table_id_.table_name))) , configuration{configuration_} , object_storage(object_storage_) + , cluster_name_in_settings(false) { /// We allow exceptions to be thrown on update(), /// because Cluster engine can only be used as table function, @@ -76,7 +97,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( ColumnsDescription columns{columns_}; std::string sample_path; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, sample_path, context_); + resolveSchemaAndFormat(columns, object_storage, configuration, {}, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata metadata; @@ -89,6 +110,24 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage( metadata.columns, context_, sample_path, std::nullopt, configuration->isDataLakeConfiguration())); setInMemoryMetadata(metadata); + + pure_storage = std::make_shared( + configuration, + object_storage, + context_, + getStorageID(), + getInMemoryMetadata().getColumns(), + getInMemoryMetadata().getConstraints(), + comment_, + format_settings_, + mode_, + /* distributed_processing */false, + partition_by_); + + auto virtuals_ = getVirtualsPtr(); + if (virtuals_) + pure_storage->setVirtuals(*virtuals_); + pure_storage->setInMemoryMetadata(getInMemoryMetadata()); } std::string StorageObjectStorageCluster::getName() const @@ -116,11 +155,133 @@ std::optional StorageObjectStorageCluster::totalBytes(ContextPtr query_c return configuration->totalBytes(); } +void StorageObjectStorageCluster::updateQueryForDistributedEngineIfNeeded(ASTPtr & query, ContextPtr context) +{ + // Change table engine on table function for distributed request + // CREATE TABLE t (...) ENGINE=IcebergS3(...) + // SELECT * FROM t + // change on + // SELECT * FROM icebergS3(...) + // to execute on cluster nodes + + auto * select_query = query->as(); + if (!select_query || !select_query->tables()) + return; + + auto * tables = select_query->tables()->as(); + + if (tables->children.empty()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected SELECT query from table with engine {}, got '{}'", + configuration->getEngineName(), query->formatForLogging()); + + auto * table_expression = tables->children[0]->as()->table_expression->as(); + + if (!table_expression) + return; + + if (!table_expression->database_and_table_name) + return; + + auto & table_identifier_typed = table_expression->database_and_table_name->as(); + + auto table_alias = table_identifier_typed.tryGetAlias(); + + auto storage_engine_name = configuration->getEngineName(); + if (storage_engine_name == "Iceberg") + { + switch (configuration->getType()) + { + case ObjectStorageType::S3: + storage_engine_name = "IcebergS3"; + break; + case ObjectStorageType::Azure: + storage_engine_name = "IcebergAzure"; + break; + case ObjectStorageType::HDFS: + storage_engine_name = "IcebergHDFS"; + break; + default: + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Can't find table function for engine {}", + storage_engine_name + ); + } + } + + static std::unordered_map engine_to_function = { + {"S3", "s3"}, + {"Azure", "azureBlobStorage"}, + {"HDFS", "hdfs"}, + {"Iceberg", "iceberg"}, + {"IcebergS3", "icebergS3"}, + {"IcebergAzure", "icebergAzure"}, + {"IcebergHDFS", "icebergHDFS"}, + {"DeltaLake", "deltaLake"}, + {"Hudi", "hudi"} + }; + + auto p = engine_to_function.find(storage_engine_name); + if (p == engine_to_function.end()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Can't find table function for engine {}", + storage_engine_name + ); + } + + std::string table_function_name = p->second; + + auto function_ast = std::make_shared(); + function_ast->name = table_function_name; + + auto cluster_name = getClusterName(context); + + if (cluster_name.empty()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Can't be here without cluster name, no cluster name in query {}", + query->formatForLogging()); + } + + function_ast->arguments = configuration->createArgsWithAccessData(); + function_ast->children.push_back(function_ast->arguments); + function_ast->setAlias(table_alias); + + ASTPtr function_ast_ptr(function_ast); + + table_expression->database_and_table_name = nullptr; + table_expression->table_function = function_ast_ptr; + table_expression->children[0] = function_ast_ptr; + + auto settings = select_query->settings(); + if (settings) + { + auto & settings_ast = settings->as(); + settings_ast.changes.insertSetting("object_storage_cluster", cluster_name); + } + else + { + auto settings_ast_ptr = std::make_shared(); + settings_ast_ptr->is_standalone = false; + settings_ast_ptr->changes.setSetting("object_storage_cluster", cluster_name); + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(settings_ast_ptr)); + } + + cluster_name_in_settings = true; +} + void StorageObjectStorageCluster::updateQueryToSendIfNeeded( ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) { + updateQueryForDistributedEngineIfNeeded(query, context); + auto * table_function = extractTableFunctionFromSelectQuery(query); if (!table_function) { @@ -149,6 +310,8 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( configuration->getEngineName()); } + ASTPtr object_storage_type_arg; + configuration->extractDynamicStorageType(args, context, &object_storage_type_arg); ASTPtr settings_temporary_storage = nullptr; for (auto * it = args.begin(); it != args.end(); ++it) { @@ -161,19 +324,70 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( } } - if (!endsWith(table_function->name, "Cluster")) - configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context, /*with_structure=*/true); + if (cluster_name_in_settings || !endsWith(table_function->name, "Cluster")) + { + configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->getFormat(), context, /*with_structure=*/true); + + /// Convert to old-stype *Cluster table function. + /// This allows to use old clickhouse versions in cluster. + static std::unordered_map function_to_cluster_function = { + {"s3", "s3Cluster"}, + {"azureBlobStorage", "azureBlobStorageCluster"}, + {"hdfs", "hdfsCluster"}, + {"iceberg", "icebergS3Cluster"}, + {"icebergS3", "icebergS3Cluster"}, + {"icebergAzure", "icebergAzureCluster"}, + {"icebergHDFS", "icebergHDFSCluster"}, + {"deltaLake", "deltaLakeCluster"}, + {"hudi", "hudiCluster"}, + }; + + auto p = function_to_cluster_function.find(table_function->name); + if (p == function_to_cluster_function.end()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Can't find cluster name for table function {}", + table_function->name); + } + + table_function->name = p->second; + + auto cluster_name = getClusterName(context); + auto cluster_name_arg = std::make_shared(cluster_name); + args.insert(args.begin(), cluster_name_arg); + + auto * select_query = query->as(); + if (!select_query) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected SELECT query from table function {}", + configuration->getEngineName()); + + auto settings = select_query->settings(); + if (settings) + { + auto & settings_ast = settings->as(); + if (settings_ast.changes.removeSetting("object_storage_cluster") && settings_ast.changes.empty()) + { + select_query->setExpression(ASTSelectQuery::Expression::SETTINGS, {}); + } + /// No throw if not found - `object_storage_cluster` can be global setting. + } + } else { ASTPtr cluster_name_arg = args.front(); args.erase(args.begin()); - configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context, /*with_structure=*/true); + configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->getFormat(), context, /*with_structure=*/true); args.insert(args.begin(), cluster_name_arg); } if (settings_temporary_storage) { args.insert(args.end(), std::move(settings_temporary_storage)); } + if (object_storage_type_arg) + args.insert(args.end(), object_storage_type_arg); } RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExtension( @@ -193,4 +407,67 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } +void StorageObjectStorageCluster::readFallBackToPure( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) +{ + pure_storage->read(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); +} + +SinkToStoragePtr StorageObjectStorageCluster::writeFallBackToPure( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) +{ + return pure_storage->write(query, metadata_snapshot, context, async_insert); +} + +String StorageObjectStorageCluster::getClusterName(ContextPtr context) const +{ + /// StorageObjectStorageCluster is always created for cluster or non-cluster variants. + /// User can specify cluster name in table definition or in setting `object_storage_cluster` + /// only for several queries. When it specified in both places, priority is given to the query setting. + /// When it is empty, non-cluster realization is used. + auto cluster_name_from_settings = context->getSettingsRef()[Setting::object_storage_cluster].value; + if (cluster_name_from_settings.empty()) + cluster_name_from_settings = getOriginalClusterName(); + return cluster_name_from_settings; +} + +QueryProcessingStage::Enum StorageObjectStorageCluster::getQueryProcessingStage( + ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info) const +{ + /// Full query if fall back to pure storage. + if (getClusterName(context).empty()) + return QueryProcessingStage::Enum::FetchColumns; + + /// Distributed storage. + return IStorageCluster::getQueryProcessingStage(context, to_stage, storage_snapshot, query_info); +} + +void StorageObjectStorageCluster::truncate( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + TableExclusiveLockHolder & lock_holder) +{ + /// Full query if fall back to pure storage. + if (getClusterName(local_context).empty()) + return pure_storage->truncate(query, metadata_snapshot, local_context, lock_holder); + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); +} + +void StorageObjectStorageCluster::addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const +{ + configuration->addStructureAndFormatToArgsIfNeeded(args, "", configuration->getFormat(), context, /*with_structure=*/false); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 4f4d541008f1..228c2779cda0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -16,10 +16,15 @@ class StorageObjectStorageCluster : public IStorageCluster const String & cluster_name_, ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, + ContextPtr context_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - ContextPtr context_); + const String & comment_, + std::optional format_settings_, + LoadingStrictnessLevel mode_, + ASTPtr partition_by_ = nullptr + ); std::string getName() const override; @@ -30,6 +35,19 @@ class StorageObjectStorageCluster : public IStorageCluster std::optional totalRows(ContextPtr query_context) const override; std::optional totalBytes(ContextPtr query_context) const override; + void setClusterNameInSettings(bool cluster_name_in_settings_) { cluster_name_in_settings = cluster_name_in_settings_; } + + String getClusterName(ContextPtr context) const override; + + QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; + + void truncate( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + TableExclusiveLockHolder &) override; + + void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override; private: void updateQueryToSendIfNeeded( @@ -37,10 +55,44 @@ class StorageObjectStorageCluster : public IStorageCluster const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; + void readFallBackToPure( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr writeFallBackToPure( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; + + /* + In case the table was created with `object_storage_cluster` setting, + modify the AST query object so that it uses the table function implementation + by mapping the engine name to table function name and setting `object_storage_cluster`. + For table like + CREATE TABLE table ENGINE=S3(...) SETTINGS object_storage_cluster='cluster' + coverts request + SELECT * FROM table + to + SELECT * FROM s3(...) SETTINGS object_storage_cluster='cluster' + to make distributed request over cluster 'cluster'. + */ + void updateQueryForDistributedEngineIfNeeded(ASTPtr & query, ContextPtr context); + const String engine_name; - const StorageObjectStorage::ConfigurationPtr configuration; + StorageObjectStorage::ConfigurationPtr configuration; const ObjectStoragePtr object_storage; NamesAndTypesList virtual_columns; + bool cluster_name_in_settings; + + /// non-clustered storage to fall back on pure realisation if needed + std::shared_ptr pure_storage; }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSettings.h b/src/Storages/ObjectStorage/StorageObjectStorageSettings.h index 1314b7d87c3d..180d71ea0c8a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSettings.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSettings.h @@ -68,7 +68,17 @@ struct StorageObjectStorageSettings using StorageObjectStorageSettingsPtr = std::shared_ptr; +// clang-format off + +#define STORAGE_OBJECT_STORAGE_RELATED_SETTINGS(DECLARE, ALIAS) \ + DECLARE(String, object_storage_cluster, "", R"( +Cluster for distributed requests +)", 0) \ + +// clang-format on + #define LIST_OF_STORAGE_OBJECT_STORAGE_SETTINGS(M, ALIAS) \ + STORAGE_OBJECT_STORAGE_RELATED_SETTINGS(M, ALIAS) \ LIST_OF_ALL_FORMAT_SETTINGS(M, ALIAS) } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 6bb10710c342..b5bb5b563e9d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -33,7 +33,7 @@ StorageObjectStorageSink::StorageObjectStorageSink( , sample_block(sample_block_) { const auto & settings = context->getSettingsRef(); - const auto chosen_compression_method = chooseCompressionMethod(path, configuration->compression_method); + const auto chosen_compression_method = chooseCompressionMethod(path, configuration->getCompressionMethod()); auto buffer = object_storage->writeObject( StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); @@ -45,7 +45,7 @@ StorageObjectStorageSink::StorageObjectStorageSink( static_cast(settings[Setting::output_format_compression_zstd_window_log])); writer = FormatFactory::instance().getOutputFormatParallelIfPossible( - configuration->format, *write_buf, sample_block, context, format_settings_); + configuration->getFormat(), *write_buf, sample_block, context, format_settings_); } void StorageObjectStorageSink::consume(Chunk & chunk) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index ee6c78f7eb27..1eb24610a9cc 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -151,6 +151,8 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); + configuration->update(object_storage, local_context, true, true); + std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { @@ -366,7 +368,7 @@ Chunk StorageObjectStorageSource::generate() void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows) { const auto cache_key = getKeyForSchemaCache( - getUniqueStoragePathIdentifier(*configuration, object_info), configuration->format, format_settings, read_context); + getUniqueStoragePathIdentifier(*configuration, object_info), configuration->getFormat(), format_settings, read_context); schema_cache.addNumRows(cache_key, num_rows); } @@ -442,7 +444,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade const auto cache_key = getKeyForSchemaCache( getUniqueStoragePathIdentifier(*configuration, *object_info), - configuration->format, + configuration->getFormat(), format_settings, context_); @@ -473,13 +475,13 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { - compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); + compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->getCompressionMethod()); const auto & archive_reader = object_info_in_archive->archive_reader; read_buf = archive_reader->readFile(object_info_in_archive->path_in_archive, /*throw_on_not_found=*/true); } else { - compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); + compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->getCompressionMethod()); read_buf = createReadBuffer(*object_info, object_storage, context_, log); } @@ -497,7 +499,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade auto input_format = FormatFactory::instance().getInput( - configuration->format, + configuration->getFormat(), *read_buf, initial_header, context_, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 5fefd53b37ee..6e68b3e6302f 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -46,14 +46,13 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( void resolveSchemaAndFormat( ColumnsDescription & columns, - std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorage::ConfigurationPtr & configuration, + StorageObjectStorage::ConfigurationPtr configuration, std::optional format_settings, std::string & sample_path, const ContextPtr & context) { - if (format == "auto") + if (configuration->getFormat() == "auto") { if (configuration->isDataLakeConfiguration()) { @@ -75,21 +74,23 @@ void resolveSchemaAndFormat( if (columns.empty()) { - if (format == "auto") + if (configuration->getFormat() == "auto") { + std::string format; std::tie(columns, format) = StorageObjectStorage::resolveSchemaAndFormatFromData( object_storage, configuration, format_settings, sample_path, context); + configuration->setFormat(format); } else { - chassert(!format.empty()); + chassert(!configuration->getFormat().empty()); columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, sample_path, context); } } } - else if (format == "auto") + else if (configuration->getFormat() == "auto") { - format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, sample_path, context); + configuration->setFormat(StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, sample_path, context)); } validateSupportedColumns(columns, *configuration); diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 8a3dfaad0287..7631d92173db 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -15,9 +15,8 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( void resolveSchemaAndFormat( ColumnsDescription & columns, - std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorage::ConfigurationPtr & configuration, + StorageObjectStorage::ConfigurationPtr configuration, std::optional format_settings, std::string & sample_path, const ContextPtr & context); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index ac873743a81a..58d7a40ae11f 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -21,13 +22,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace StorageObjectStorageSetting +{ + extern const StorageObjectStorageSettingsString object_storage_cluster; +} + namespace { // LocalObjectStorage is only supported for Iceberg Datalake operations where Avro format is required. For regular file access, use FileStorage instead. #if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS || USE_AVRO -std::shared_ptr +StoragePtr createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObjectStorage::ConfigurationPtr configuration) { auto & engine_args = args.engine_args; @@ -35,7 +41,14 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); const auto context = args.getLocalContext(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, context, false); + auto storage_settings = std::make_shared(); + + if (args.storage_def->settings) + storage_settings->loadFromQuery(*args.storage_def->settings); + + auto cluster_name = (*storage_settings)[StorageObjectStorageSetting::object_storage_cluster].value; + + configuration->initialize(args.engine_args, context, false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current @@ -59,7 +72,8 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - return std::make_shared( + return std::make_shared( + cluster_name, configuration, // We only want to perform write actions (e.g. create a container in Azure) when the table is being created, // and we want to avoid it when we load the table after a server restart. @@ -71,7 +85,6 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject args.comment, format_settings, args.mode, - /* distributed_processing */ false, partition_by); } @@ -176,22 +189,22 @@ static DataLakeStorageSettingsPtr getDataLakeStorageSettings(const ASTStorage & void registerStorageIceberg(StorageFactory & factory) { -#if USE_AWS_S3 factory.registerStorage( "Iceberg", [&](const StorageFactory::Arguments & args) { const auto storage_settings = getDataLakeStorageSettings(*args.storage_def); - auto configuration = std::make_shared(storage_settings); + auto configuration = std::make_shared(storage_settings); return createStorageObjectStorage(args, configuration); }, { .supports_settings = true, .supports_schema_inference = true, - .source_access_type = AccessType::S3, + .source_access_type = AccessType::NONE, .has_builtin_setting_fn = DataLakeStorageSettings::hasBuiltin, }); +# if USE_AWS_S3 factory.registerStorage( "IcebergS3", [&](const StorageFactory::Arguments & args) diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 27a9d804110d..22c65c110775 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -226,12 +226,12 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( validateSettings(*queue_settings_, is_attach); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); - FormatFactory::instance().checkFormatName(configuration->format); + FormatFactory::instance().checkFormatName(configuration->getFormat()); configuration->check(context_); ColumnsDescription columns{columns_}; std::string sample_path; - resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, sample_path, context_); + resolveSchemaAndFormat(columns, object_storage, configuration, format_settings, sample_path, context_); configuration->check(context_); StorageInMemoryMetadata storage_metadata; @@ -246,7 +246,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); auto table_metadata = ObjectStorageQueueMetadata::syncWithKeeper( - zk_path, *queue_settings_, storage_metadata.getColumns(), configuration_->format, context_, is_attach, log); + zk_path, *queue_settings_, storage_metadata.getColumns(), configuration_->getFormat(), context_, is_attach, log); ObjectStorageType storage_type = engine_name == "S3Queue" ? ObjectStorageType::S3 : ObjectStorageType::Azure; @@ -317,7 +317,7 @@ void StorageObjectStorageQueue::shutdown(bool is_drop) bool StorageObjectStorageQueue::supportsSubsetOfColumns(const ContextPtr & context_) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context_, format_settings); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->getFormat(), context_, format_settings); } class ReadFromObjectStorageQueue : public SourceStepWithFilter diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h index 579ea0680853..673c51ac5ae7 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h @@ -56,7 +56,7 @@ class StorageObjectStorageQueue : public IStorage, WithContext ContextPtr local_context, AlterLockHolder & table_lock_holder) override; - const auto & getFormatName() const { return configuration->format; } + const auto & getFormatName() const { return configuration->getFormat(); } const fs::path & getZooKeeperPath() const { return zk_path; } diff --git a/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp b/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp index da6cba2d7ea4..df884a5f82af 100644 --- a/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp +++ b/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp @@ -34,7 +34,7 @@ StoragePtr createQueueStorage(const StorageFactory::Arguments & args) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getContext(), false); + configuration->initialize(args.engine_args, args.getContext(), false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current diff --git a/src/Storages/extractTableFunctionFromSelectQuery.cpp b/src/Storages/extractTableFunctionFromSelectQuery.cpp index 57302036c889..c7f60240b3c7 100644 --- a/src/Storages/extractTableFunctionFromSelectQuery.cpp +++ b/src/Storages/extractTableFunctionFromSelectQuery.cpp @@ -20,7 +20,16 @@ ASTFunction * extractTableFunctionFromSelectQuery(ASTPtr & query) if (!table_expression->table_function) return nullptr; - return table_expression->table_function->as(); + auto * table_function = table_expression->table_function->as(); + return table_function; +} + +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) +{ + auto * table_function = extractTableFunctionFromSelectQuery(query); + if (!table_function) + return nullptr; + return table_function->arguments->as(); } } diff --git a/src/Storages/extractTableFunctionFromSelectQuery.h b/src/Storages/extractTableFunctionFromSelectQuery.h index c69cc7ce6c52..0511d59f6230 100644 --- a/src/Storages/extractTableFunctionFromSelectQuery.h +++ b/src/Storages/extractTableFunctionFromSelectQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -8,5 +9,6 @@ namespace DB { ASTFunction * extractTableFunctionFromSelectQuery(ASTPtr & query); +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); } diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index f24580608c5e..184c942022fc 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -78,7 +78,7 @@ class ITableFunction : public std::enable_shared_from_this virtual bool supportsReadingSubsetOfColumns(const ContextPtr &) { return true; } - virtual bool canBeUsedToCreateTable() const { return true; } + virtual void validateUseToCreateTable() const {} /// Create storage according to the query. StoragePtr diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index 2550e10f2702..ab2e3f809ba6 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int CLUSTER_DOESNT_EXIST; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } /// Base class for *Cluster table functions that require cluster_name for the first argument. @@ -46,7 +47,10 @@ class ITableFunctionCluster : public Base throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected table function name: {}", table_function->name); } - bool canBeUsedToCreateTable() const override { return false; } + void validateUseToCreateTable() const override + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' cannot be used to create a table", getName()); + } protected: void parseArguments(const ASTPtr & ast, ContextPtr context) override diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 29a7c5c04de3..11606f0be866 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -125,7 +125,7 @@ template ColumnsDescription TableFunctionObjectStorage< Definition, Configuration, is_data_lake>::getActualTableStructure(ContextPtr context, bool is_insert_query) const { - if (configuration->structure == "auto") + if (configuration->getStructure() == "auto") { context->checkAccess(getSourceAccessType()); @@ -140,7 +140,6 @@ ColumnsDescription TableFunctionObjectStorage< ColumnsDescription columns; resolveSchemaAndFormat( columns, - configuration->format, std::move(storage), configuration, /* format_settings */std::nullopt, @@ -149,7 +148,7 @@ ColumnsDescription TableFunctionObjectStorage< return columns; } - return parseColumnsListFromString(configuration->structure, context); + return parseColumnsListFromString(configuration->getStructure(), context); } template @@ -163,8 +162,8 @@ StoragePtr TableFunctionObjectStorage:: chassert(configuration); ColumnsDescription columns; - if (configuration->structure != "auto") - columns = parseColumnsListFromString(configuration->structure, context); + if (configuration->getStructure() != "auto") + columns = parseColumnsListFromString(configuration->getStructure(), context); else if (!structure_hint.empty()) columns = structure_hint; else if (!cached_columns.empty()) @@ -187,10 +186,14 @@ StoragePtr TableFunctionObjectStorage:: parallel_replicas_cluster_name, configuration, getObjectStorage(context, !is_insert_query), + context, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, - context); + /* comment */ String{}, + /* format_settings */ std::nullopt, /// No format_settings + /* mode */ LoadingStrictnessLevel::CREATE, + /* partition_by */ nullptr); storage->startup(); return storage; @@ -218,18 +221,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AWS_S3 - factory.registerFunction>( - { - .documentation = - { - .description=R"(The table function can be used to read the data stored on AWS S3.)", - .examples{{"s3", "SELECT * FROM s3(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction - }, - .allow_readonly = false - }); - - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -240,7 +232,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -251,7 +243,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -262,60 +254,30 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif - -#if USE_AZURE_BLOB_STORAGE - factory.registerFunction>( - { - .documentation = - { - .description=R"(The table function can be used to read the data stored on Azure Blob Storage.)", - .examples{ - { - "azureBlobStorage", - "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " - "[account_name, account_key, format, compression, structure])", "" - }}, - .category = FunctionDocumentation::Category::TableFunction - }, - .allow_readonly = false - }); -#endif -#if USE_HDFS - factory.registerFunction>( - { - .documentation = - { - .description=R"(The table function can be used to read the data stored on HDFS virtual filesystem.)", - .examples{ - { - "hdfs", - "SELECT * FROM hdfs(url, format, compression, structure])", "" - }}, - .category = FunctionDocumentation::Category::TableFunction - }, - .allow_readonly = false - }); -#endif } #if USE_AZURE_BLOB_STORAGE -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; #endif #if USE_AWS_S3 -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; #endif #if USE_HDFS -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +#endif +template class TableFunctionObjectStorage; + +#if USE_AVRO +template class TableFunctionObjectStorage; #endif -template class TableFunctionObjectStorage; #if USE_AVRO && USE_AWS_S3 template class TableFunctionObjectStorage; @@ -340,37 +302,6 @@ template class TableFunctionObjectStorage( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", - .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); - -#endif -#if USE_AZURE_BLOB_STORAGE - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", - .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -#endif -#if USE_HDFS - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Iceberg table stored on HDFS virtual filesystem.)", - .examples{{"icebergHDFS", "SELECT * FROM icebergHDFS(url)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -#endif factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", @@ -381,68 +312,11 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) #endif -#if USE_PARQUET && USE_DELTA_KERNEL_RS -void registerTableFunctionDeltaLake(TableFunctionFactory & factory) -{ -#if USE_AWS_S3 - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the DeltaLake table stored on S3, alias of deltaLakeS3.)", - .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); - - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the DeltaLake table stored on S3.)", - .examples{{"deltaLakeS3", "SELECT * FROM deltaLakeS3(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -#endif - -#if USE_AZURE_BLOB_STORAGE - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the DeltaLake table stored on Azure object store.)", - .examples{{"deltaLakeAzure", "SELECT * FROM deltaLakeAzure(connection_string|storage_account_url, container_name, blobpath, \"\n" - " \"[account_name, account_key, format, compression, structure])", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -#endif - // Register the new local Delta Lake table function - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the DeltaLake table stored locally.)", - .examples{{"deltaLakeLocal", "SELECT * FROM deltaLakeLocal(path)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -} -#endif - -#if USE_AWS_S3 -void registerTableFunctionHudi(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description = R"(The table function can be used to read the Hudi table stored on object store.)", - .examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)", ""}}, - .category = FunctionDocumentation::Category::TableFunction}, - .allow_readonly = false}); -} -#endif - void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AVRO registerTableFunctionIceberg(factory); #endif - -#if USE_PARQUET && USE_DELTA_KERNEL_RS - registerTableFunctionDeltaLake(factory); -#endif -#if USE_AWS_S3 - registerTableFunctionHudi(factory); -#endif } } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index d10781e8f0ec..81e6fc59fd38 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -69,7 +69,7 @@ struct LocalDefinition struct IcebergDefinition { static constexpr auto name = "iceberg"; - static constexpr auto storage_type_name = "S3"; + static constexpr auto storage_type_name = "UNDEFINED"; }; struct IcebergS3Definition @@ -138,15 +138,16 @@ class TableFunctionObjectStorage : public ITableFunction String getName() const override { return name; } - bool hasStaticStructure() const override { return configuration->structure != "auto"; } + bool hasStaticStructure() const override { return configuration->getStructure() != "auto"; } - bool needStructureHint() const override { return configuration->structure == "auto"; } + bool needStructureHint() const override { return configuration->getStructure() == "auto"; } void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } bool supportsReadingSubsetOfColumns(const ContextPtr & context) override { - return configuration->format != "auto" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); + return configuration->getFormat() != "auto" + && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->getFormat(), context); } std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override @@ -156,7 +157,7 @@ class TableFunctionObjectStorage : public ITableFunction virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context) { - StorageObjectStorage::Configuration::initialize(*getConfiguration(), args, context, true); + getConfiguration()->initialize(args, context, true); } static void updateStructureAndFormatArgumentsIfNeeded( @@ -168,8 +169,8 @@ class TableFunctionObjectStorage : public ITableFunction if constexpr (is_data_lake) { Configuration configuration(createEmptySettings()); - if (configuration.format == "auto") - configuration.format = "Parquet"; /// Default format of data lakes. + if (configuration.getFormat() == "auto") + configuration.setFormat("Parquet"); /// Default format of data lakes. configuration.addStructureAndFormatToArgsIfNeeded(args, structure, format, context, /*with_structure=*/true); } @@ -206,23 +207,24 @@ class TableFunctionObjectStorage : public ITableFunction }; #if USE_AWS_S3 -using TableFunctionS3 = TableFunctionObjectStorage; +using TableFunctionS3 = TableFunctionObjectStorage; #endif #if USE_AZURE_BLOB_STORAGE -using TableFunctionAzureBlob = TableFunctionObjectStorage; +using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif #if USE_HDFS -using TableFunctionHDFS = TableFunctionObjectStorage; +using TableFunctionHDFS = TableFunctionObjectStorage; #endif -using TableFunctionLocal = TableFunctionObjectStorage; +using TableFunctionLocal = TableFunctionObjectStorage; #if USE_AVRO +using TableFunctionIceberg = TableFunctionObjectStorage; + # if USE_AWS_S3 -using TableFunctionIceberg = TableFunctionObjectStorage; using TableFunctionIcebergS3 = TableFunctionObjectStorage; # endif # if USE_AZURE_BLOB_STORAGE diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 6083d9a915f0..6f3d61ef54bd 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -22,8 +22,9 @@ StoragePtr TableFunctionObjectStorageClusterstructure != "auto") - columns = parseColumnsListFromString(configuration->structure, context); + + if (configuration->getStructure() != "auto") + columns = parseColumnsListFromString(configuration->getStructure(), context); else if (!Base::structure_hint.empty()) columns = Base::structure_hint; else if (!cached_columns.empty()) @@ -54,10 +55,14 @@ StoragePtr TableFunctionObjectStorageCluster::cluster_name, configuration, object_storage, + context, StorageID(Base::getDatabaseName(), table_name), columns, ConstraintsDescription{}, - context); + /* comment */ String{}, + /* format_settings */ std::nullopt, /// No format_settings + /* mode */ LoadingStrictnessLevel::CREATE, + /* partition_by */ nullptr); } storage->startup(); @@ -118,32 +123,49 @@ void registerTableFunctionIcebergCluster(TableFunctionFactory & factory) { UNUSED(factory); -#if USE_AWS_S3 + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on any object store in parallel for many nodes in a specified cluster.)", + .examples{ +# if USE_AWS_S3 + {"icebergCluster", "SELECT * FROM icebergCluster(cluster, url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression], storage_type='s3')", ""}, +# endif +# if USE_AZURE_BLOB_STORAGE + {"icebergCluster", "SELECT * FROM icebergCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression], storage_type='azure')", ""}, +# endif +# if USE_HDFS + {"icebergCluster", "SELECT * FROM icebergCluster(cluster, uri, [format], [structure], [compression_method], storage_type='hdfs')", ""}, +# endif + }, + .category = FunctionDocumentation::Category::TableFunction}, + .allow_readonly = false}); + +# if USE_AWS_S3 factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store in parallel for many nodes in a specified cluster.)", .examples{{"icebergS3Cluster", "SELECT * FROM icebergS3Cluster(cluster, url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression])", ""}}, .category = FunctionDocumentation::Category::TableFunction}, .allow_readonly = false}); -#endif +# endif -#if USE_AZURE_BLOB_STORAGE +# if USE_AZURE_BLOB_STORAGE factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store in parallel for many nodes in a specified cluster.)", .examples{{"icebergAzureCluster", "SELECT * FROM icebergAzureCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])", ""}}, .category = FunctionDocumentation::Category::TableFunction}, .allow_readonly = false}); -#endif +# endif -#if USE_HDFS +# if USE_HDFS factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on HDFS virtual filesystem in parallel for many nodes in a specified cluster.)", .examples{{"icebergHDFSCluster", "SELECT * FROM icebergHDFSCluster(cluster, uri, [format], [structure], [compression_method])", ""}}, .category = FunctionDocumentation::Category::TableFunction}, .allow_readonly = false}); -#endif +# endif } #endif diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index f8477abac18c..6ca66b4fc174 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -10,8 +10,6 @@ namespace DB class Context; -class StorageS3Settings; -class StorageAzureBlobSettings; class StorageS3Configuration; class StorageAzureConfiguration; @@ -33,6 +31,12 @@ struct HDFSClusterDefinition static constexpr auto storage_type_name = "HDFSCluster"; }; +struct IcebergClusterDefinition +{ + static constexpr auto name = "icebergCluster"; + static constexpr auto storage_type_name = "UNDEFINED"; +}; + struct IcebergS3ClusterDefinition { static constexpr auto name = "icebergS3Cluster"; @@ -91,23 +95,27 @@ class TableFunctionObjectStorageCluster : public ITableFunctionClusterstructure != "auto"; } + bool hasStaticStructure() const override { return Base::getConfiguration()->getStructure() != "auto"; } - bool needStructureHint() const override { return Base::getConfiguration()->structure == "auto"; } + bool needStructureHint() const override { return Base::getConfiguration()->getStructure() == "auto"; } void setStructureHint(const ColumnsDescription & structure_hint_) override { Base::structure_hint = structure_hint_; } }; #if USE_AWS_S3 -using TableFunctionS3Cluster = TableFunctionObjectStorageCluster; +using TableFunctionS3Cluster = TableFunctionObjectStorageCluster; #endif #if USE_AZURE_BLOB_STORAGE -using TableFunctionAzureBlobCluster = TableFunctionObjectStorageCluster; +using TableFunctionAzureBlobCluster = TableFunctionObjectStorageCluster; #endif #if USE_HDFS -using TableFunctionHDFSCluster = TableFunctionObjectStorageCluster; +using TableFunctionHDFSCluster = TableFunctionObjectStorageCluster; +#endif + +#if USE_AVRO +using TableFunctionIcebergCluster = TableFunctionObjectStorageCluster; #endif #if USE_AVRO && USE_AWS_S3 diff --git a/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp new file mode 100644 index 000000000000..ea31d246b173 --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp @@ -0,0 +1,386 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace Setting +{ + extern const SettingsString object_storage_cluster; +} + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +struct S3ClusterFallbackDefinition +{ + static constexpr auto name = "s3"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto storage_type_cluster_name = "S3Cluster"; +}; + +struct AzureClusterFallbackDefinition +{ + static constexpr auto name = "azureBlobStorage"; + static constexpr auto storage_type_name = "Azure"; + static constexpr auto storage_type_cluster_name = "AzureBlobStorageCluster"; +}; + +struct HDFSClusterFallbackDefinition +{ + static constexpr auto name = "hdfs"; + static constexpr auto storage_type_name = "HDFS"; + static constexpr auto storage_type_cluster_name = "HDFSCluster"; +}; + +struct IcebergClusterFallbackDefinition +{ + static constexpr auto name = "iceberg"; + static constexpr auto storage_type_name = "UNDEFINED"; + static constexpr auto storage_type_cluster_name = "IcebergCluster"; +}; + +struct IcebergS3ClusterFallbackDefinition +{ + static constexpr auto name = "icebergS3"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto storage_type_cluster_name = "IcebergS3Cluster"; +}; + +struct IcebergAzureClusterFallbackDefinition +{ + static constexpr auto name = "icebergAzure"; + static constexpr auto storage_type_name = "Azure"; + static constexpr auto storage_type_cluster_name = "IcebergAzureCluster"; +}; + +struct IcebergHDFSClusterFallbackDefinition +{ + static constexpr auto name = "icebergHDFS"; + static constexpr auto storage_type_name = "HDFS"; + static constexpr auto storage_type_cluster_name = "IcebergHDFSCluster"; +}; + +struct DeltaLakeClusterFallbackDefinition +{ + static constexpr auto name = "deltaLake"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto storage_type_cluster_name = "DeltaLakeS3Cluster"; +}; + +struct HudiClusterFallbackDefinition +{ + static constexpr auto name = "hudi"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto storage_type_cluster_name = "HudiS3Cluster"; +}; + +template +void TableFunctionObjectStorageClusterFallback::parseArgumentsImpl(ASTs & args, const ContextPtr & context) +{ + if (args.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "The function {} should have arguments. The first argument must be the cluster name and the rest are the arguments of " + "corresponding table function", + getName()); + + const auto & settings = context->getSettingsRef(); + + is_cluster_function = !settings[Setting::object_storage_cluster].value.empty(); + + if (is_cluster_function) + { + ASTPtr cluster_name_arg = std::make_shared(settings[Setting::object_storage_cluster].value); + args.insert(args.begin(), cluster_name_arg); + BaseCluster::parseArgumentsImpl(args, context); + args.erase(args.begin()); + } + else + BaseSimple::parseArgumentsImpl(args, context); +} + +template +StoragePtr TableFunctionObjectStorageClusterFallback::executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const +{ + if (is_cluster_function) + { + auto result = BaseCluster::executeImpl(ast_function, context, table_name, cached_columns, is_insert_query); + if (auto storage = typeid_cast>(result)) + storage->setClusterNameInSettings(true); + return result; + } + else + return BaseSimple::executeImpl(ast_function, context, table_name, cached_columns, is_insert_query); +} + +template +void TableFunctionObjectStorageClusterFallback::validateUseToCreateTable() const +{ + if (is_cluster_function) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Table function '{}' cannot be used to create a table in cluster mode", + getName()); +} + +#if USE_AWS_S3 +using TableFunctionS3ClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AZURE_BLOB_STORAGE +using TableFunctionAzureClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_HDFS +using TableFunctionHDFSClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AVRO +using TableFunctionIcebergClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AVRO && USE_AWS_S3 +using TableFunctionIcebergS3ClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AVRO && USE_AZURE_BLOB_STORAGE +using TableFunctionIcebergAzureClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AVRO && USE_HDFS +using TableFunctionIcebergHDFSClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AWS_S3 && USE_PARQUET && USE_DELTA_KERNEL_RS +using TableFunctionDeltaLakeClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +#if USE_AWS_S3 +using TableFunctionHudiClusterFallback = TableFunctionObjectStorageClusterFallback; +#endif + +void registerTableFunctionObjectStorageClusterFallback(TableFunctionFactory & factory) +{ + UNUSED(factory); +#if USE_AWS_S3 + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on S3 in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + {"s3", "SELECT * FROM s3(url, format, structure)", ""}, + {"s3", "SELECT * FROM s3(url, format, structure) SETTINGS object_storage_cluster='cluster'", ""} + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on Azure Blob Storage in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "azureBlobStorage", + "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure])", "" + }, + { + "azureBlobStorage", + "SELECT * FROM azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure]) " + "SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_HDFS + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on HDFS virtual filesystem in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "hdfs", + "SELECT * FROM hdfs(url, format, compression, structure])", "" + }, + { + "hdfs", + "SELECT * FROM hdfs(url, format, compression, structure]) " + "SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AVRO + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the Iceberg table stored on different object store in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "iceberg", + "SELECT * FROM iceberg(url, access_key_id, secret_access_key, storage_type='s3')", "" + }, + { + "iceberg", + "SELECT * FROM iceberg(url, access_key_id, secret_access_key, storage_type='s3') " + "SETTINGS object_storage_cluster='cluster'", "" + }, + { + "iceberg", + "SELECT * FROM iceberg(url, access_key_id, secret_access_key, storage_type='azure')", "" + }, + { + "iceberg", + "SELECT * FROM iceberg(url, storage_type='hdfs') SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AVRO && USE_AWS_S3 + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the Iceberg table stored on S3 object store in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "icebergS3", + "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", "" + }, + { + "icebergS3", + "SELECT * FROM icebergS3(url, access_key_id, secret_access_key) " + "SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AVRO && USE_AZURE_BLOB_STORAGE + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the Iceberg table stored on Azure object store in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "icebergAzure", + "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", "" + }, + { + "icebergAzure", + "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key) " + "SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AVRO && USE_HDFS + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the Iceberg table stored on HDFS virtual filesystem in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "icebergHDFS", + "SELECT * FROM icebergHDFS(url)", "" + }, + { + "icebergHDFS", + "SELECT * FROM icebergHDFS(url) SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AWS_S3 && USE_PARQUET && USE_DELTA_KERNEL_RS + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the DeltaLake table stored on object store in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "deltaLake", + "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", "" + }, + { + "deltaLake", + "SELECT * FROM deltaLake(url, access_key_id, secret_access_key) " + "SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif + +#if USE_AWS_S3 + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the Hudi table stored on object store in parallel for many nodes in a specified cluster or from single node.)", + .examples{ + { + "hudi", + "SELECT * FROM hudi(url, access_key_id, secret_access_key)", "" + }, + { + "hudi", + "SELECT * FROM hudi(url, access_key_id, secret_access_key) SETTINGS object_storage_cluster='cluster'", "" + }, + }, + .category = FunctionDocumentation::Category::TableFunction + }, + .allow_readonly = false + } + ); +#endif +} + +} diff --git a/src/TableFunctions/TableFunctionObjectStorageClusterFallback.h b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.h new file mode 100644 index 000000000000..28dfe22143dc --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.h @@ -0,0 +1,49 @@ +#pragma once +#include "config.h" +#include + +namespace DB +{ + +/** +* Class implementing s3/hdfs/azureBlobStorage(...) table functions, +* which allow to use simple or distributed function variant based on settings. +* If setting `object_storage_cluster` is empty, +* simple single-host variant is used, if setting not empty, cluster variant is used. +* `SELECT * FROM s3('s3://...', ...) SETTINGS object_storage_cluster='cluster'` +* is equal to +* `SELECT * FROM s3Cluster('cluster', 's3://...', ...)` +*/ + +template +class TableFunctionObjectStorageClusterFallback : public Base +{ +public: + using BaseCluster = Base; + using BaseSimple = BaseCluster::Base; + + static constexpr auto name = Definition::name; + + String getName() const override { return name; } + + void validateUseToCreateTable() const override; + +private: + const char * getStorageTypeName() const override + { + return is_cluster_function ? Definition::storage_type_cluster_name : Definition::storage_type_name; + } + + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override; + + bool is_cluster_function = false; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 039e0eb7e54a..c224350e7bf6 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -63,6 +63,7 @@ void registerTableFunctions() registerTableFunctionObjectStorage(factory); registerTableFunctionObjectStorageCluster(factory); + registerTableFunctionObjectStorageClusterFallback(factory); registerDataLakeTableFunctions(factory); registerDataLakeClusterTableFunctions(factory); } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 9b2306974bb2..e0431fd8d56d 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -61,6 +61,7 @@ void registerTableFunctionExplain(TableFunctionFactory & factory); void registerTableFunctionObjectStorage(TableFunctionFactory & factory); void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); +void registerTableFunctionObjectStorageClusterFallback(TableFunctionFactory & factory); void registerDataLakeTableFunctions(TableFunctionFactory & factory); void registerDataLakeClusterTableFunctions(TableFunctionFactory & factory); diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 3573acb008b0..9489de0eb145 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -234,6 +234,21 @@ def test_wrong_cluster(started_cluster): assert "not found" in error + error = node.query_and_get_error( + f""" + SELECT count(*) from s3( + 'http://minio1:9001/root/data/{{clickhouse,database}}/*', + 'minio', '{minio_secret_key}', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + UNION ALL + SELECT count(*) from s3( + 'http://minio1:9001/root/data/{{clickhouse,database}}/*', + 'minio', '{minio_secret_key}', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + SETTINGS object_storage_cluster = 'non_existing_cluster' + """ + ) + + assert "not found" in error + def test_ambiguous_join(started_cluster): node = started_cluster.instances["s0_0_0"] @@ -252,6 +267,20 @@ def test_ambiguous_join(started_cluster): ) assert "AMBIGUOUS_COLUMN_NAME" not in result + result = node.query( + f""" + SELECT l.name, r.value from s3( + 'http://minio1:9001/root/data/{{clickhouse,database}}/*', 'minio', '{minio_secret_key}', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as l + JOIN s3( + 'http://minio1:9001/root/data/{{clickhouse,database}}/*', 'minio', '{minio_secret_key}', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') as r + ON l.name = r.name + SETTINGS object_storage_cluster = 'cluster_simple' + """ + ) + assert "AMBIGUOUS_COLUMN_NAME" not in result + def test_skip_unavailable_shards(started_cluster): node = started_cluster.instances["s0_0_0"] @@ -267,6 +296,17 @@ def test_skip_unavailable_shards(started_cluster): assert result == "10\n" + result = node.query( + f""" + SELECT count(*) from s3( + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', '{minio_secret_key}', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + SETTINGS skip_unavailable_shards = 1, object_storage_cluster = 'cluster_non_existent_port' + """ + ) + + assert result == "10\n" + def test_unset_skip_unavailable_shards(started_cluster): # Although skip_unavailable_shards is not set, cluster table functions should always skip unavailable shards. @@ -282,6 +322,17 @@ def test_unset_skip_unavailable_shards(started_cluster): assert result == "10\n" + result = node.query( + f""" + SELECT count(*) from s3( + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', '{minio_secret_key}', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + SETTINGS object_storage_cluster = 'cluster_non_existent_port' + """ + ) + + assert result == "10\n" + def test_distributed_insert_select_with_replicated(started_cluster): first_replica_first_shard = started_cluster.instances["s0_0_0"] @@ -490,6 +541,18 @@ def test_cluster_format_detection(started_cluster): assert result == expected_result + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/generated/*', 'minio', '{minio_secret_key}') order by c1, c2 SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/generated/*', 'minio', '{minio_secret_key}', auto, 'a String, b UInt64') order by a, b SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + def test_cluster_default_expression(started_cluster): node = started_cluster.instances["s0_0_0"] @@ -537,3 +600,33 @@ def test_cluster_default_expression(started_cluster): ) assert result == expected_result + + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/data{{1,2,3}}', 'minio', '{minio_secret_key}', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262') order by id SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/data{{1,2,3}}', 'minio', '{minio_secret_key}', 'auto', 'id UInt32, date Date DEFAULT 18262') order by id SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/data{{1,2,3}}', 'minio', '{minio_secret_key}', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + + result = node.query( + f"SELECT * FROM s3('http://minio1:9001/root/data/data{{1,2,3}}', 'minio', '{minio_secret_key}', 'auto', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result + + result = node.query( + "SELECT * FROM s3(test_s3_with_default) order by id SETTINGS object_storage_cluster = 'cluster_simple'" + ) + + assert result == expected_result diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 93f668811268..e3a54a45aec5 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -551,6 +551,27 @@ def test_types(started_cluster, format_version, storage_type): ) +def count_secondary_subqueries(started_cluster, query_id, expected, comment): + for node_name, replica in started_cluster.instances.items(): + cluster_secondary_queries = ( + replica.query( + f""" + SELECT count(*) FROM system.query_log + WHERE + type = 'QueryFinish' + AND NOT is_initial_query + AND initial_query_id='{query_id}' + """ + ) + .strip() + ) + + logging.info( + f"[{node_name}] cluster_secondary_queries {comment}: {cluster_secondary_queries}" + ) + assert int(cluster_secondary_queries) == expected + + @pytest.mark.parametrize("format_version", ["1", "2"]) @pytest.mark.parametrize("storage_type", ["s3", "azure"]) def test_cluster_table_function(started_cluster, format_version, storage_type): From 25ce92f38ab36d15160d6fb02ff4c910f58ffb95 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 16 Jul 2025 19:56:49 +0200 Subject: [PATCH 02/16] Fix test test_storage_s3/test.py::test_partition_by --- src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index f844d082a5c2..febf45e684f1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -58,7 +58,9 @@ String StorageObjectStorageCluster::getPathSample(StorageInMemoryMetadata metada {}, metadata.getColumns().getAll(), // virtual_columns nullptr, // read_keys - {} // file_progress_callback + {}, // file_progress_callback + false, // ignore_archive_globs + true // skip_object_metadata ); if (auto file = file_iterator->next(0)) From 910165a88893d14de65cf9ebb6470ed690dd4cb6 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 16 Jul 2025 20:15:32 +0200 Subject: [PATCH 03/16] Remove wrong files --- .../Iceberg/metadata/INFORMATION_SCHEMA.sql | 2 - .../DataLakes/Iceberg/metadata/default.sql | 2 - .../DataLakes/Iceberg/metadata/ice.sql | 3 -- .../Iceberg/metadata/information_schema.sql | 2 - .../DataLakes/Iceberg/metadata/system | 1 - .../DataLakes/Iceberg/metadata/system.sql | 2 - .../Iceberg/preprocessed_configs/config.xml | 44 ------------------- .../ObjectStorage/DataLakes/Iceberg/status | 3 -- .../ObjectStorage/DataLakes/Iceberg/uuid | 1 - 9 files changed, 60 deletions(-) delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql delete mode 120000 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/status delete mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/uuid diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql deleted file mode 100644 index 291582fd1eb3..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/INFORMATION_SCHEMA.sql +++ /dev/null @@ -1,2 +0,0 @@ -ATTACH DATABASE INFORMATION_SCHEMA -ENGINE = Memory diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql deleted file mode 100644 index 3b99a1d7af8f..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/default.sql +++ /dev/null @@ -1,2 +0,0 @@ -ATTACH DATABASE _ UUID 'd9e2d817-49d1-4148-9aad-73a382733129' -ENGINE = Atomic diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql deleted file mode 100644 index 0a9ae99ee054..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/ice.sql +++ /dev/null @@ -1,3 +0,0 @@ -ATTACH DATABASE ice -ENGINE = DataLakeCatalog('https://iceberg-catalog.aws-us-west-2.dev.altinity.cloud') -SETTINGS catalog_type = 'rest', auth_header = 'Authorization: Bearer etxkehqze7esafs9qw07lcrww5nd0iqo', warehouse = 's3://aws-st-2-fs5vug37-iceberg' diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql deleted file mode 100644 index 6cea934b49d2..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/information_schema.sql +++ /dev/null @@ -1,2 +0,0 @@ -ATTACH DATABASE information_schema -ENGINE = Memory diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system deleted file mode 120000 index 86be5e92dc38..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system +++ /dev/null @@ -1 +0,0 @@ -./store/41c/41c68de0-0dd8-4d7f-8c61-df82ac3de66d/ \ No newline at end of file diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql b/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql deleted file mode 100644 index 3d990f9689b2..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/metadata/system.sql +++ /dev/null @@ -1,2 +0,0 @@ -ATTACH DATABASE _ UUID '41c68de0-0dd8-4d7f-8c61-df82ac3de66d' -ENGINE = Atomic diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml b/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml deleted file mode 100644 index 790297966d1b..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/preprocessed_configs/config.xml +++ /dev/null @@ -1,44 +0,0 @@ - - - - - - trace - true - - - 8123 - 9000 - 9004 - - ./ - - true - - - - - - - ::/0 - - - default - default - - 1 - 1 - - - - - - - - - - - diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/status b/src/Storages/ObjectStorage/DataLakes/Iceberg/status deleted file mode 100644 index bfed01e9836c..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/status +++ /dev/null @@ -1,3 +0,0 @@ -PID: 1058703 -Started at: 2025-06-20 12:17:04 -Revision: 54498 diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid b/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid deleted file mode 100644 index 2f29a5ba4076..000000000000 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/uuid +++ /dev/null @@ -1 +0,0 @@ -24fc6e4a-fd07-4ba7-88f0-e779b4571020 \ No newline at end of file From 5cc54a0b370ee3e4bfb41f95503e927a62ad73da Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 10 Jul 2025 13:31:24 +0200 Subject: [PATCH 04/16] Lazy init for StorageObjectStorageCluster --- src/Databases/DataLake/DatabaseDataLake.cpp | 3 +- .../StorageObjectStorageCluster.cpp | 70 +++++++++++++++---- .../StorageObjectStorageCluster.h | 3 +- .../TableFunctionObjectStorageCluster.cpp | 6 +- 4 files changed, 65 insertions(+), 17 deletions(-) diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index 68bfdd52dc03..f5827feb02bc 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -445,7 +445,8 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con /* comment */"", getFormatSettings(context_copy), LoadingStrictnessLevel::CREATE, - /* partition_by */nullptr); + /* partition_by */nullptr, + /* lazy_init */true); } DatabaseTablesIteratorPtr DatabaseDataLake::getTablesIterator( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index febf45e684f1..27d9501694c9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -80,7 +80,8 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( const String & comment_, std::optional format_settings_, LoadingStrictnessLevel mode_, - ASTPtr partition_by_ + ASTPtr partition_by_, + bool lazy_init ) : IStorageCluster( cluster_name_, table_id_, getLogger(fmt::format("{}({})", configuration_->getEngineName(), table_id_.table_name))) @@ -88,26 +89,67 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( , object_storage(object_storage_) , cluster_name_in_settings(false) { - /// We allow exceptions to be thrown on update(), - /// because Cluster engine can only be used as table function, - /// so no lazy initialization is allowed. - configuration->update( - object_storage, - context_, - /* if_not_updated_before */false, - /* check_consistent_with_previous_metadata */true); + const bool need_resolve_columns_or_format = columns_.empty() || (configuration->getFormat() == "auto"); + const bool need_resolve_sample_path = context_->getSettingsRef()[Setting::use_hive_partitioning] + && !configuration->isDataLakeConfiguration(); + const bool do_lazy_init = lazy_init && !need_resolve_columns_or_format && !need_resolve_sample_path; + + auto log_ = getLogger("StorageObjectStorageCluster"); + + bool updated_configuration = false; + try + { + if (!do_lazy_init) + { + /// We allow exceptions to be thrown on update(), + /// because Cluster engine can only be used as table function, + /// so no lazy initialization is allowed. + configuration->update( + object_storage, + context_, + /* if_not_updated_before */false, + /* check_consistent_with_previous_metadata */true); + updated_configuration = true; + } + } + catch (...) + { + // If we don't have format or schema yet, we can't ignore failed configuration update, + // because relevant configuration is crucial for format and schema inference + if (mode_ <= LoadingStrictnessLevel::CREATE || need_resolve_columns_or_format) + { + throw; + } + tryLogCurrentException(log_); + } ColumnsDescription columns{columns_}; std::string sample_path; - resolveSchemaAndFormat(columns, object_storage, configuration, {}, sample_path, context_); + if (need_resolve_columns_or_format) + resolveSchemaAndFormat(columns, object_storage, configuration, {}, sample_path, context_); + else + validateSupportedColumns(columns, *configuration); configuration->check(context_); StorageInMemoryMetadata metadata; metadata.setColumns(columns); metadata.setConstraints(constraints_); - if (sample_path.empty() && context_->getSettingsRef()[Setting::use_hive_partitioning] && !configuration->isDataLakeConfiguration()) - sample_path = getPathSample(metadata, context_); + if (updated_configuration && sample_path.empty() && need_resolve_sample_path) + { + try + { + sample_path = getPathSample(metadata, context_); + } + catch (...) + { + LOG_WARNING( + log_, + "Failed to list object storage, cannot use hive partitioning. " + "Error: {}", + getCurrentExceptionMessage(true)); + } + } setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage( metadata.columns, context_, sample_path, std::nullopt, configuration->isDataLakeConfiguration())); @@ -124,7 +166,9 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( format_settings_, mode_, /* distributed_processing */false, - partition_by_); + partition_by_, + /* is_table_function */false, + /* lazy_init */lazy_init); auto virtuals_ = getVirtualsPtr(); if (virtuals_) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 228c2779cda0..2b4e634a0288 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -23,7 +23,8 @@ class StorageObjectStorageCluster : public IStorageCluster const String & comment_, std::optional format_settings_, LoadingStrictnessLevel mode_, - ASTPtr partition_by_ = nullptr + ASTPtr partition_by_ = nullptr, + bool lazy_init = false ); std::string getName() const override; diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 6f3d61ef54bd..6b153b376a3f 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -47,7 +47,8 @@ StoragePtr TableFunctionObjectStorageClusterstartup(); From 51ab6256f4959be2653c6e4ea719231ff780cb90 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 9 Jul 2025 21:23:50 +0200 Subject: [PATCH 05/16] Fix metadata update --- src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp | 7 +++++++ src/Storages/ObjectStorage/StorageObjectStorageCluster.h | 2 ++ 2 files changed, 9 insertions(+) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 27d9501694c9..41d698443c10 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -516,4 +516,11 @@ void StorageObjectStorageCluster::addInferredEngineArgsToCreateQuery(ASTs & args configuration->addStructureAndFormatToArgsIfNeeded(args, "", configuration->getFormat(), context, /*with_structure=*/false); } +bool StorageObjectStorageCluster::updateExternalDynamicMetadataIfExists(ContextPtr context) +{ + if (pure_storage) + return pure_storage->updateExternalDynamicMetadataIfExists(context); + return false; +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 2b4e634a0288..87dce34d91e6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -50,6 +50,8 @@ class StorageObjectStorageCluster : public IStorageCluster void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override; + bool updateExternalDynamicMetadataIfExists(ContextPtr context) override; + private: void updateQueryToSendIfNeeded( ASTPtr & query, From baffd1f6dfdfe5367665adea2ccf22489465238d Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 10 Jul 2025 12:29:44 +0200 Subject: [PATCH 06/16] More fix --- .../ObjectStorage/StorageObjectStorageCluster.cpp | 9 ++++++++- src/Storages/ObjectStorage/StorageObjectStorageCluster.h | 2 ++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 41d698443c10..52bd978467b4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -520,7 +520,14 @@ bool StorageObjectStorageCluster::updateExternalDynamicMetadataIfExists(ContextP { if (pure_storage) return pure_storage->updateExternalDynamicMetadataIfExists(context); - return false; + return IStorageCluster::updateExternalDynamicMetadataIfExists(context); +} + +StorageMetadataPtr StorageObjectStorageCluster::getInMemoryMetadataPtr() const +{ + if (pure_storage) + return pure_storage->getInMemoryMetadataPtr(); + return IStorageCluster::getInMemoryMetadataPtr(); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 87dce34d91e6..79227bf32644 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -52,6 +52,8 @@ class StorageObjectStorageCluster : public IStorageCluster bool updateExternalDynamicMetadataIfExists(ContextPtr context) override; + StorageMetadataPtr getInMemoryMetadataPtr() const override; + private: void updateQueryToSendIfNeeded( ASTPtr & query, From 2e55e90ff1077e08cd9cc4e6ada6928de0032186 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 16 Jul 2025 21:44:10 +0200 Subject: [PATCH 07/16] Fix object_storage_cluster setting and system.iceberg_history table --- src/Databases/DataLake/DatabaseDataLake.cpp | 2 +- .../ObjectStorage/StorageObjectStorageCluster.cpp | 14 ++++++++++++++ .../ObjectStorage/StorageObjectStorageCluster.h | 2 ++ .../registerStorageObjectStorage.cpp | 15 ++++++--------- .../System/StorageSystemIcebergHistory.cpp | 6 +++--- 5 files changed, 26 insertions(+), 13 deletions(-) diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index f5827feb02bc..16bb2ec4000a 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -480,7 +480,7 @@ DatabaseTablesIteratorPtr DatabaseDataLake::getLightweightTablesIterator( const FilterByNameFunction & filter_by_table_name, bool skip_not_loaded) const { - Tables tables; + Tables tables; auto catalog = getCatalog(); const auto iceberg_tables = catalog->getTables(); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 52bd978467b4..fe3492300f80 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -530,4 +530,18 @@ StorageMetadataPtr StorageObjectStorageCluster::getInMemoryMetadataPtr() const return IStorageCluster::getInMemoryMetadataPtr(); } +IDataLakeMetadata * StorageObjectStorageCluster::getExternalMetadata(ContextPtr query_context) +{ + if (pure_storage) + return pure_storage->getExternalMetadata(query_context); + + configuration->update( + object_storage, + query_context, + /* if_not_updated_before */false, + /* check_consistent_with_previous_metadata */false); + + return configuration->getExternalMetadata(); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 79227bf32644..59d19345de15 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -50,6 +50,8 @@ class StorageObjectStorageCluster : public IStorageCluster void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override; + IDataLakeMetadata * getExternalMetadata(ContextPtr query_context); + bool updateExternalDynamicMetadataIfExists(ContextPtr context) override; StorageMetadataPtr getInMemoryMetadataPtr() const override; diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 58d7a40ae11f..3bc4b4a63d33 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -22,11 +22,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace StorageObjectStorageSetting -{ - extern const StorageObjectStorageSettingsString object_storage_cluster; -} - namespace { @@ -41,12 +36,14 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); const auto context = args.getLocalContext(); - auto storage_settings = std::make_shared(); + + std::string cluster_name = ""; if (args.storage_def->settings) - storage_settings->loadFromQuery(*args.storage_def->settings); - - auto cluster_name = (*storage_settings)[StorageObjectStorageSetting::object_storage_cluster].value; + { + if (const auto * value = args.storage_def->settings->changes.tryGet("object_storage_cluster")) + cluster_name = value->safeGet(); + } configuration->initialize(args.engine_args, context, false); diff --git a/src/Storages/System/StorageSystemIcebergHistory.cpp b/src/Storages/System/StorageSystemIcebergHistory.cpp index c567d226cdc8..c82b6b0d0533 100644 --- a/src/Storages/System/StorageSystemIcebergHistory.cpp +++ b/src/Storages/System/StorageSystemIcebergHistory.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include @@ -51,7 +51,7 @@ void StorageSystemIcebergHistory::fillData([[maybe_unused]] MutableColumns & res #if USE_AVRO const auto access = context->getAccess(); - auto add_history_record = [&](const DatabaseTablesIteratorPtr & it, StorageObjectStorage * object_storage) + auto add_history_record = [&](const DatabaseTablesIteratorPtr & it, StorageObjectStorageCluster * object_storage) { if (!access->isGranted(AccessType::SHOW_TABLES, it->databaseName(), it->name())) return; @@ -100,7 +100,7 @@ void StorageSystemIcebergHistory::fillData([[maybe_unused]] MutableColumns & res // Table was dropped while acquiring the lock, skipping table continue; - if (auto * object_storage_table = dynamic_cast(storage.get())) + if (auto * object_storage_table = dynamic_cast(storage.get())) { add_history_record(iterator, object_storage_table); } From 1b8a417529db2be581b241e9b0529278d6e0906c Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 16 Jul 2025 22:50:03 +0200 Subject: [PATCH 08/16] Merge pull request #592 from Altinity/project-antalya-24.12.2-cluster-alt-syntax Alternative syntax for object storage cluster functions --- .../integration/test_storage_iceberg/test.py | 51 ++++++++++++++++--- 1 file changed, 45 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index e3a54a45aec5..bd8b9b85a319 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -634,16 +634,37 @@ def add_df(mode): table_function=True, run_on_cluster=True, ) + query_id_cluster = str(uuid.uuid4()) select_cluster = ( - instance.query(f"SELECT * FROM {table_function_expr_cluster}").strip().split() + instance.query( + f"SELECT * FROM {table_function_expr_cluster}", query_id=query_id_cluster + ) + .strip() + .split() + ) + + # Cluster Query with node1 as coordinator with alternative syntax + query_id_cluster_alt_syntax = str(uuid.uuid4()) + select_cluster_alt_syntax = ( + instance.query( + f""" + SELECT * FROM {table_function_expr} + SETTINGS object_storage_cluster='cluster_simple' + """, + query_id=query_id_cluster_alt_syntax, + ) + .strip() + .split() ) # Simple size check assert len(select_regular) == 600 assert len(select_cluster) == 600 + assert len(select_cluster_alt_syntax) == 600 # Actual check assert select_cluster == select_regular + assert select_cluster_alt_syntax == select_regular # Check query_log for replica in started_cluster.instances.values(): @@ -655,11 +676,29 @@ def add_df(mode): f""" SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log WHERE - type = 'QueryStart' AND - positionCaseInsensitive(query, '{storage_type}Cluster') != 0 AND - position(query, '{TABLE_NAME}') != 0 AND - position(query, 'system.query_log') = 0 AND - NOT is_initial_query + type = 'QueryStart' + AND NOT is_initial_query + AND initial_query_id='{query_id_cluster}' + """ + ) + .strip() + .split("\n") + ) + + logging.info( + f"[{node_name}] cluster_secondary_queries: {cluster_secondary_queries}" + ) + assert len(cluster_secondary_queries) == 1 + + for node_name, replica in started_cluster.instances.items(): + cluster_secondary_queries = ( + replica.query( + f""" + SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log + WHERE + type = 'QueryStart' + AND NOT is_initial_query + AND initial_query_id='{query_id_cluster_alt_syntax}' """ ) .strip() From 4d3862dc3e5f94c3f43ba2605f01db85bc882689 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 16 Jul 2025 22:59:05 +0200 Subject: [PATCH 09/16] Merge pull request #615 from Altinity/project-antalya-24.12.2-distributed-table-engine Distributed request to tables with Object Storage Engines --- tests/integration/test_s3_cluster/test.py | 69 +++++++++++ .../integration/test_storage_iceberg/test.py | 116 +++++++++++------- 2 files changed, 143 insertions(+), 42 deletions(-) diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 9489de0eb145..ce46c8c416c3 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -630,3 +630,72 @@ def test_cluster_default_expression(started_cluster): ) assert result == expected_result + + +def test_distributed_s3_table_engine(started_cluster): + node = started_cluster.instances["s0_0_0"] + + resp_def = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + """ + ) + + node.query("DROP TABLE IF EXISTS single_node"); + node.query( + """ + CREATE TABLE single_node + (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) + ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + """ + ) + query_id_engine_single_node = str(uuid.uuid4()) + resp_engine_single_node = node.query( + """ + SELECT * FROM single_node ORDER BY (name, value, polygon) + """, + query_id = query_id_engine_single_node + ) + assert resp_def == resp_engine_single_node + + node.query("DROP TABLE IF EXISTS distributed"); + node.query( + """ + CREATE TABLE distributed + (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) + ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + SETTINGS object_storage_cluster='cluster_simple' + """ + ) + query_id_engine_distributed = str(uuid.uuid4()) + resp_engine_distributed = node.query( + """ + SELECT * FROM distributed ORDER BY (name, value, polygon) + """, + query_id = query_id_engine_distributed + ) + assert resp_def == resp_engine_distributed + + node.query("SYSTEM FLUSH LOGS ON CLUSTER 'cluster_simple'") + + hosts_engine_single_node = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_engine_single_node}' + """ + ) + assert int(hosts_engine_single_node) == 1 + hosts_engine_distributed = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_engine_distributed}' + """ + ) + assert int(hosts_engine_distributed) == 3 + + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index bd8b9b85a319..2d142da68b39 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -202,6 +202,7 @@ def get_creation_expression( allow_dynamic_metadata_for_data_lakes=False, use_version_hint=False, run_on_cluster=False, + object_storage_cluster=False, explicit_metadata_path="", **kwargs, ): @@ -215,6 +216,9 @@ def get_creation_expression( if use_version_hint: settings_array.append("iceberg_use_version_hint = true") + if object_storage_cluster: + settings_array.append(f"object_storage_cluster = '{object_storage_cluster}'") + if settings_array: settings_expression = " SETTINGS " + ",".join(settings_array) else: @@ -313,10 +317,18 @@ def create_iceberg_table( table_name, cluster, format="Parquet", + object_storage_cluster=False, **kwargs, ): node.query( - get_creation_expression(storage_type, table_name, cluster, format, **kwargs) + get_creation_expression( + storage_type, + table_name, + cluster, + format, + object_storage_cluster=object_storage_cluster, + **kwargs, + ) ) @@ -657,61 +669,81 @@ def add_df(mode): .split() ) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, object_storage_cluster='cluster_simple') + query_id_cluster_table_engine = str(uuid.uuid4()) + select_cluster_table_engine = ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} + """, + query_id=query_id_cluster_table_engine, + ) + .strip() + .split() + ) + + # write 3 times + assert int(instance.query(f"SELECT count() FROM {table_function_expr_cluster}")) == 100 * 3 + + select_remote_cluster = ( + instance.query(f"SELECT * FROM remote('node2',{table_function_expr_cluster})") + .strip() + .split() + ) + + instance.query(f"DROP TABLE IF EXISTS `{TABLE_NAME}` SYNC") + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) + query_id_pure_table_engine = str(uuid.uuid4()) + select_pure_table_engine = ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} + """, + query_id=query_id_pure_table_engine, + ) + .strip() + .split() + ) + query_id_pure_table_engine_cluster = str(uuid.uuid4()) + select_pure_table_engine_cluster = ( + instance.query( + f""" + SELECT * FROM {TABLE_NAME} + SETTINGS object_storage_cluster='cluster_simple' + """, + query_id=query_id_pure_table_engine_cluster, + ) + .strip() + .split() + ) + # Simple size check assert len(select_regular) == 600 assert len(select_cluster) == 600 assert len(select_cluster_alt_syntax) == 600 + assert len(select_cluster_table_engine) == 600 + assert len(select_remote_cluster) == 600 + assert len(select_pure_table_engine) == 600 + assert len(select_pure_table_engine_cluster) == 600 # Actual check assert select_cluster == select_regular assert select_cluster_alt_syntax == select_regular + assert select_cluster_table_engine == select_regular + assert select_remote_cluster == select_regular + assert select_pure_table_engine == select_regular + assert select_pure_table_engine_cluster == select_regular # Check query_log for replica in started_cluster.instances.values(): replica.query("SYSTEM FLUSH LOGS") - for node_name, replica in started_cluster.instances.items(): - cluster_secondary_queries = ( - replica.query( - f""" - SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log - WHERE - type = 'QueryStart' - AND NOT is_initial_query - AND initial_query_id='{query_id_cluster}' - """ - ) - .strip() - .split("\n") - ) - - logging.info( - f"[{node_name}] cluster_secondary_queries: {cluster_secondary_queries}" - ) - assert len(cluster_secondary_queries) == 1 - - for node_name, replica in started_cluster.instances.items(): - cluster_secondary_queries = ( - replica.query( - f""" - SELECT query, type, is_initial_query, read_rows, read_bytes FROM system.query_log - WHERE - type = 'QueryStart' - AND NOT is_initial_query - AND initial_query_id='{query_id_cluster_alt_syntax}' - """ - ) - .strip() - .split("\n") - ) - - logging.info( - f"[{node_name}] cluster_secondary_queries: {cluster_secondary_queries}" - ) - assert len(cluster_secondary_queries) == 1 + count_secondary_subqueries(started_cluster, query_id_cluster, 1, "table function") + count_secondary_subqueries(started_cluster, query_id_cluster_alt_syntax, 1, "table function alt syntax") + count_secondary_subqueries(started_cluster, query_id_cluster_table_engine, 1, "cluster table engine") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine, 0, "table engine") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine_cluster, 1, "table engine with cluster setting") - # write 3 times - assert int(instance.query(f"SELECT count() FROM {table_function_expr_cluster}")) == 100 * 3 @pytest.mark.parametrize("format_version", ["1", "2"]) @pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) From 1b8f21e177c5d6cf91b8d75594c3eec6b35411ab Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 17 Jul 2025 10:14:12 +0200 Subject: [PATCH 10/16] Fix password in tests and object_storage_cluster for table engines --- .../DataLake/DatabaseDataLakeSettings.cpp | 1 - .../DataLakes/DataLakeStorageSettings.h | 3 +++ .../configs/named_collections.xml | 2 +- tests/integration/test_s3_cache_locality/test.py | 9 +++++---- tests/integration/test_s3_cluster/test.py | 15 +++++++-------- .../configs/config.d/named_collections.xml | 2 +- tests/integration/test_storage_iceberg/test.py | 14 +++++++------- 7 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/Databases/DataLake/DatabaseDataLakeSettings.cpp b/src/Databases/DataLake/DatabaseDataLakeSettings.cpp index 046017c2fa8f..a6b15bc71334 100644 --- a/src/Databases/DataLake/DatabaseDataLakeSettings.cpp +++ b/src/Databases/DataLake/DatabaseDataLakeSettings.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes DECLARE(String, aws_secret_access_key, "", "Key for AWS connection for Glue Catalog'", 0) \ DECLARE(String, region, "", "Region for Glue catalog", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ - DECLARE(String, object_storage_cluster, "", "Cluster for distributed requests", 0) \ #define LIST_OF_DATABASE_ICEBERG_SETTINGS(M, ALIAS) \ DATABASE_ICEBERG_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeStorageSettings.h b/src/Storages/ObjectStorage/DataLakes/DataLakeStorageSettings.h index a87a0a43c935..e63f80088a84 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeStorageSettings.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeStorageSettings.h @@ -60,6 +60,9 @@ If enabled, the engine would use the metadata file with the most recent last_upd DECLARE(Bool, iceberg_use_version_hint, false, R"( Get latest metadata path from version-hint.text file. )", 0) \ + DECLARE(String, object_storage_cluster, "", R"( +Cluster for distributed requests +)", 0) #define OBSOLETE_SETTINGS(M, ALIAS) \ MAKE_OBSOLETE(M, Bool, allow_experimental_delta_kernel_rs, true) \ diff --git a/tests/integration/test_s3_cache_locality/configs/named_collections.xml b/tests/integration/test_s3_cache_locality/configs/named_collections.xml index 511078d6f0d9..6994aa3f5e77 100644 --- a/tests/integration/test_s3_cache_locality/configs/named_collections.xml +++ b/tests/integration/test_s3_cache_locality/configs/named_collections.xml @@ -3,7 +3,7 @@ http://minio1:9001/root/data/* minio - minio123 + ClickHouse_Minio_P@ssw0rd CSV> diff --git a/tests/integration/test_s3_cache_locality/test.py b/tests/integration/test_s3_cache_locality/test.py index a2020d7e0568..da667fad35b9 100644 --- a/tests/integration/test_s3_cache_locality/test.py +++ b/tests/integration/test_s3_cache_locality/test.py @@ -7,6 +7,7 @@ import pytest from helpers.cluster import ClickHouseCluster +from helpers.config_cluster import minio_secret_key logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) @@ -81,7 +82,7 @@ def check_s3_gets(cluster, node, expected_result, cluster_first, cluster_second, result_first = node.query( f""" SELECT count(*) - FROM s3Cluster('{cluster_first}', 'http://minio1:9001/root/data/generated/*', 'minio', 'minio123', 'CSV', 'a String, b UInt64') + FROM s3Cluster('{cluster_first}', 'http://minio1:9001/root/data/generated/*', 'minio', '{minio_secret_key}', 'CSV', 'a String, b UInt64') WHERE b=42 SETTINGS enable_filesystem_cache={enable_filesystem_cache}, @@ -95,7 +96,7 @@ def check_s3_gets(cluster, node, expected_result, cluster_first, cluster_second, result_second = node.query( f""" SELECT count(*) - FROM s3Cluster('{cluster_second}', 'http://minio1:9001/root/data/generated/*', 'minio', 'minio123', 'CSV', 'a String, b UInt64') + FROM s3Cluster('{cluster_second}', 'http://minio1:9001/root/data/generated/*', 'minio', '{minio_secret_key}', 'CSV', 'a String, b UInt64') WHERE b=42 SETTINGS enable_filesystem_cache={enable_filesystem_cache}, @@ -148,9 +149,9 @@ def test_cache_locality(started_cluster): node = started_cluster.instances["clickhouse0"] expected_result = node.query( - """ + f""" SELECT count(*) - FROM s3('http://minio1:9001/root/data/generated/*', 'minio', 'minio123', 'CSV', 'a String, b UInt64') + FROM s3('http://minio1:9001/root/data/generated/*', 'minio', '{minio_secret_key}', 'CSV', 'a String, b UInt64') WHERE b=42 """ ) diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index ce46c8c416c3..3f2f788e9327 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -2,8 +2,7 @@ import logging import os import shutil -import time -from email.errors import HeaderParseError +import uuid import pytest @@ -636,20 +635,20 @@ def test_distributed_s3_table_engine(started_cluster): node = started_cluster.instances["s0_0_0"] resp_def = node.query( - """ + f""" SELECT * from s3Cluster( 'cluster_simple', - 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'http://minio1:9001/root/data/{{clickhouse,database}}/*', 'minio', '{minio_secret_key}', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) """ ) node.query("DROP TABLE IF EXISTS single_node"); node.query( - """ + f""" CREATE TABLE single_node (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) - ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + ENGINE=S3('http://minio1:9001/root/data/{{clickhouse,database}}/*', 'minio', '{minio_secret_key}', 'CSV') """ ) query_id_engine_single_node = str(uuid.uuid4()) @@ -663,10 +662,10 @@ def test_distributed_s3_table_engine(started_cluster): node.query("DROP TABLE IF EXISTS distributed"); node.query( - """ + f""" CREATE TABLE distributed (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) - ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + ENGINE=S3('http://minio1:9001/root/data/{{clickhouse,database}}/*', 'minio', '{minio_secret_key}', 'CSV') SETTINGS object_storage_cluster='cluster_simple' """ ) diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index 892665d3934d..77f9e7e4b17b 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -14,7 +14,7 @@ http://minio1:9001/root/ minio - minio123 + ClickHouse_Minio_P@ssw0rd s3 diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 2d142da68b39..e7d23ce649be 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -685,11 +685,11 @@ def add_df(mode): # write 3 times assert int(instance.query(f"SELECT count() FROM {table_function_expr_cluster}")) == 100 * 3 - select_remote_cluster = ( - instance.query(f"SELECT * FROM remote('node2',{table_function_expr_cluster})") - .strip() - .split() - ) + #select_remote_cluster = ( + # instance.query(f"SELECT * FROM remote('node2',{table_function_expr_cluster})") + # .strip() + # .split() + #) instance.query(f"DROP TABLE IF EXISTS `{TABLE_NAME}` SYNC") create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -722,7 +722,7 @@ def add_df(mode): assert len(select_cluster) == 600 assert len(select_cluster_alt_syntax) == 600 assert len(select_cluster_table_engine) == 600 - assert len(select_remote_cluster) == 600 + #assert len(select_remote_cluster) == 600 assert len(select_pure_table_engine) == 600 assert len(select_pure_table_engine_cluster) == 600 @@ -730,7 +730,7 @@ def add_df(mode): assert select_cluster == select_regular assert select_cluster_alt_syntax == select_regular assert select_cluster_table_engine == select_regular - assert select_remote_cluster == select_regular + #assert select_remote_cluster == select_regular assert select_pure_table_engine == select_regular assert select_pure_table_engine_cluster == select_regular From ee2230fdf506844eb4eaa24655e8c8dab44aeb75 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 17 Jul 2025 15:30:42 +0200 Subject: [PATCH 11/16] Fix iceberg iterator, fix iceberg remote call, fix tests --- .../DataLakes/DataLakeConfiguration.h | 12 + .../StorageObjectStorageCluster.cpp | 2 +- .../test_mask_sensitive_info/test.py | 205 ++++++++++---- .../integration/test_storage_iceberg/test.py | 251 ++++++++++++------ 4 files changed, 346 insertions(+), 124 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index d7e64afb97b9..602ccdcefa63 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -349,6 +349,18 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, void setCompressionMethod(const String & compression_method_) override { getImpl().setCompressionMethod(compression_method_); } void setStructure(const String & structure_) override { getImpl().setStructure(structure_); } + bool supportsFileIterator() const override { return getImpl().supportsFileIterator(); } + + ObjectIterator iterate( + const ActionsDAG * filter_dag, + IDataLakeMetadata::FileProgressCallback callback, + size_t list_batch_size, + ContextPtr context) override + { + return getImpl().iterate(filter_dag, callback, list_batch_size, context); + } + + protected: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override { return getImpl().fromNamedCollection(collection, context); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index fe3492300f80..5469228702d4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -380,7 +380,7 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( {"s3", "s3Cluster"}, {"azureBlobStorage", "azureBlobStorageCluster"}, {"hdfs", "hdfsCluster"}, - {"iceberg", "icebergS3Cluster"}, + {"iceberg", "icebergCluster"}, {"icebergS3", "icebergS3Cluster"}, {"icebergAzure", "icebergAzureCluster"}, {"icebergHDFS", "icebergHDFSCluster"}, diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 3b068b8d8b0a..2a1ea1db5a66 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -280,9 +280,7 @@ def test_create_table(): f"AzureBlobStorage('{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", f"AzureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_2.csv', '{azure_account_name}', '{azure_account_key}')", - f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '{azure_account_key}', 'CSV')", - f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none')", + f"AzureQueue('{azure_conn_string}', 'cont', '*', 'CSV') SETTINGS mode = 'unordered'", f"AzureQueue('{azure_conn_string}', 'cont', '*', 'CSV', 'none') SETTINGS mode = 'unordered'", f"AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '{azure_account_key}', 'CSV') SETTINGS mode = 'unordered'", @@ -291,6 +289,44 @@ def test_create_table(): f"AzureBlobStorage('BlobEndpoint=https://my-endpoint/;SharedAccessSignature=sp=r&st=2025-09-29T14:58:11Z&se=2025-09-29T00:00:00Z&spr=https&sv=2022-11-02&sr=c&sig=SECRET%SECRET%SECRET%SECRET', 'exampledatasets', 'example.csv')", "STD_EXCEPTION", ) + + f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '{azure_account_key}')", + f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '{azure_account_key}', 'CSV')", + f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none')", + f"AzureBlobStorage(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"AzureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + ( + f"IcebergAzure('{azure_conn_string}', 'cont', 'test_simple.csv')", + "FILE_DOESNT_EXIST", + ), + ( + f"IcebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", + "FILE_DOESNT_EXIST", + ), + ( + f"IcebergAzure(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + "FILE_DOESNT_EXIST", + ), + ( + f"IcebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + "FILE_DOESNT_EXIST", + ), + ( + f"Iceberg(storage_type='azure', '{azure_conn_string}', 'cont', 'test_simple.csv')", + "FILE_DOESNT_EXIST", + ), + ( + f"Iceberg(storage_type='azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", + "FILE_DOESNT_EXIST", + ), + ( + f"Iceberg(storage_type='azure', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + "FILE_DOESNT_EXIST", + ), + ( + f"Iceberg(storage_type='azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + "FILE_DOESNT_EXIST", + ), ] def make_test_case(i): @@ -357,16 +393,28 @@ def make_test_case(i): "CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip') SETTINGS mode = 'ordered'", "CREATE TABLE table21 (`x` int) ENGINE = Iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", "CREATE TABLE table22 (`x` int) ENGINE = IcebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE table23 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", - f"CREATE TABLE table24 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"CREATE TABLE table25 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_2.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE table26 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", - f"CREATE TABLE table27 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", - f"CREATE TABLE table28 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV') SETTINGS mode = 'unordered'", - f"CREATE TABLE table29 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV', 'none') SETTINGS mode = 'unordered'", - f"CREATE TABLE table30 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV') SETTINGS mode = 'unordered'", - f"CREATE TABLE table31 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none') SETTINGS mode = 'unordered'", - f"CREATE TABLE table32 (`x` int) ENGINE = AzureBlobStorage('{masked_sas_conn_string}', 'exampledatasets', 'example.csv')", + f"CREATE TABLE table23 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV') SETTINGS mode = 'unordered'", + f"CREATE TABLE table24 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV', 'none') SETTINGS mode = 'unordered'", + f"CREATE TABLE table25 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV') SETTINGS mode = 'unordered'", + f"CREATE TABLE table26 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none') SETTINGS mode = 'unordered'", + f"CREATE TABLE table27 (`x` int) ENGINE = AzureBlobStorage('{masked_sas_conn_string}', 'exampledatasets', 'example.csv')", + + "CREATE TABLE table23 (`x` int) ENGINE = Iceberg(storage_type = 's3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + f"CREATE TABLE table24 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", + f"CREATE TABLE table25 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", + f"CREATE TABLE table26 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table27 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", + f"CREATE TABLE table28 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", + f"CREATE TABLE table29 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table30 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE table31 (`x` int) ENGINE = IcebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE table32 (`x` int) ENGINE = IcebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table33 (`x` int) ENGINE = IcebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table34 (`x` int) ENGINE = IcebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE table35 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE table36 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table37 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table38 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", ], must_not_contain=[password], ) @@ -452,6 +500,9 @@ def test_table_functions(): f"s3('http://minio1:9001/root/data/test6.csv', 'minio', '{password}', 'CSV')", f"s3('http://minio1:9001/root/data/test7.csv', 'minio', '{password}', 'CSV', 'x int')", f"s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', '{password}', 'CSV', 'x int', 'gzip')", + f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', '{password}')", + f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')", + f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', '{password}', 'CSV')", f"remote('127.{{2..11}}', default.remote_table)", f"remote('127.{{2..11}}', default.remote_table, rand())", f"remote('127.{{2..11}}', default.remote_table, 'remote_user')", @@ -480,10 +531,40 @@ def test_table_functions(): f"azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", f"azureBlobStorage(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", f"azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '{azure_account_key}')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '{azure_account_key}', 'CSV')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none')", + f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", + f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", f"iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", - f"gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + f"iceberg(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + f"icebergS3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", + f"icebergAzure('{azure_conn_string}', 'cont', 'test_simple.csv')", + f"icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", f"icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", + f"icebergAzure(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"icebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + f"icebergS3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + f"icebergS3Cluster('test_shard_localhost', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", + f"icebergAzureCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple.csv')", + f"icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", + f"icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", + f"icebergAzureCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"icebergAzureCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + f"iceberg(storage_type='s3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + f"iceberg(storage_type='s3', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", + f"iceberg(storage_type='azure', '{azure_conn_string}', 'cont', 'test_simple.csv')", + f"iceberg(storage_type='azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", + f"iceberg(storage_type='azure', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", + f"iceberg(storage_type='azure', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"iceberg(storage_type='azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", + f"gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", + ] def make_test_case(i): @@ -535,38 +616,70 @@ def make_test_case(i): "CREATE TABLE tablefunc8 (`x` int) AS s3('http://minio1:9001/root/data/test6.csv', 'minio', '[HIDDEN]', 'CSV')", "CREATE TABLE tablefunc9 (`x` int) AS s3('http://minio1:9001/root/data/test7.csv', 'minio', '[HIDDEN]', 'CSV', 'x int')", "CREATE TABLE tablefunc10 (`x` int) AS s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'x int', 'gzip')", - "CREATE TABLE tablefunc11 (x int) AS remote('127.{2..11}', default.remote_table)", - "CREATE TABLE tablefunc12 (x int) AS remote('127.{2..11}', default.remote_table, rand())", - "CREATE TABLE tablefunc13 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user')", - "CREATE TABLE tablefunc14 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]')", - "CREATE TABLE tablefunc15 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user', rand())", - "CREATE TABLE tablefunc16 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc17 (`x` int) AS remote('127.{2..11}', 'default.remote_table', 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc18 (`x` int) AS remote('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc19 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc20 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')", - "CREATE TABLE tablefunc21 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", - "CREATE TABLE tablefunc22 (`x` int) AS mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', `table` = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", - "CREATE TABLE tablefunc23 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', `table` = 'postgres_table', user = 'postgres_user')", - "CREATE TABLE tablefunc24 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - "CREATE TABLE tablefunc25 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", - "CREATE TABLE tablefunc26 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", - "CREATE TABLE tablefunc27 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", - "CREATE TABLE tablefunc28 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc29 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE tablefunc30 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", - f"CREATE TABLE tablefunc31 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"CREATE TABLE tablefunc32 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", - f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", - f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - "CREATE TABLE tablefunc39 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc40 (`x` int) AS gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc41 (`x` int) AS icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE tablefunc42 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + "CREATE TABLE tablefunc11 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc12 (x int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')", + "CREATE TABLE tablefunc13 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', '[HIDDEN]', 'CSV')", + "CREATE TABLE tablefunc14 (x int) AS remote('127.{2..11}', default.remote_table)", + "CREATE TABLE tablefunc15 (x int) AS remote('127.{2..11}', default.remote_table, rand())", + "CREATE TABLE tablefunc16 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user')", + "CREATE TABLE tablefunc17 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]')", + "CREATE TABLE tablefunc18 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user', rand())", + "CREATE TABLE tablefunc19 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc20 (`x` int) AS remote('127.{2..11}', 'default.remote_table', 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc21 (`x` int) AS remote('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc22 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc23 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')", + "CREATE TABLE tablefunc24 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", + "CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', `table` = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', `table` = 'postgres_table', user = 'postgres_user')", + "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", + "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc30 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", + "CREATE TABLE tablefunc31 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc32 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", + f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", + f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", + f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", + f"CREATE TABLE tablefunc39 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc40 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc41 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc42 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')", + f"CREATE TABLE tablefunc43 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')", + f"CREATE TABLE tablefunc44 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc45 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_12.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc46 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_13.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", + f"CREATE TABLE tablefunc47 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_14.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", + f"CREATE TABLE tablefunc48 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc49 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", + f"CREATE TABLE tablefunc50 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc51 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc52 (`x` int) AS iceberg(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + "CREATE TABLE tablefunc53 (`x` int) AS icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc54 (`x` int) AS icebergS3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc55 (`x` int) AS icebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE tablefunc56 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc57 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc58 (`x` int) AS icebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc59 (`x` int) AS icebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc60 (`x` int) AS icebergS3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc61 (`x` int) AS icebergS3Cluster('test_shard_localhost', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc62 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE tablefunc63 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc64 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc65 (`x` int) AS icebergAzureCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc66 (`x` int) AS icebergAzureCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc67 (`x` int) AS iceberg(storage_type = 's3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc68 (`x` int) AS iceberg(storage_type = 's3', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc69 (`x` int) AS iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE tablefunc70 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc71 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc72 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc73 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc74 (`x` int) AS gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index e7d23ce649be..f7ea16e4eb3b 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -1,21 +1,16 @@ -import glob -import json import logging import os -import time import uuid +import time from datetime import datetime, timezone import pyspark import pytest -from azure.storage.blob import BlobServiceClient from minio.deleteobjects import DeleteObject from pyspark.sql.functions import ( - current_timestamp, monotonically_increasing_id, row_number, ) -from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.types import ( ArrayType, BooleanType, @@ -24,17 +19,14 @@ StringType, StructField, StructType, - TimestampType, ) from pyspark.sql.window import Window -import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm from helpers.s3_tools import ( AzureUploader, LocalUploader, S3Uploader, - get_file_contents, list_s3_objects, prepare_s3_bucket, ) @@ -204,6 +196,8 @@ def get_creation_expression( run_on_cluster=False, object_storage_cluster=False, explicit_metadata_path="", + storage_type_as_arg=False, + storage_type_in_named_collection=False, **kwargs, ): settings_array = [] @@ -224,6 +218,22 @@ def get_creation_expression( else: settings_expression = "" + storage_arg = storage_type + engine_part = "" + if (storage_type_in_named_collection): + storage_arg += "_with_type" + elif (storage_type_as_arg): + storage_arg += f", storage_type='{storage_type}'" + else: + if (storage_type == "s3"): + engine_part = "S3" + elif (storage_type == "azure"): + engine_part = "Azure" + elif (storage_type == "hdfs"): + engine_part = "HDFS" + elif (storage_type == "local"): + engine_part = "Local" + if storage_type == "s3": if "bucket" in kwargs: bucket = kwargs["bucket"] @@ -232,16 +242,16 @@ def get_creation_expression( if run_on_cluster: assert table_function - return f"icebergS3Cluster('cluster_simple', s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" + return f"iceberg{engine_part}Cluster('cluster_simple', {storage_arg}, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" else: if table_function: - return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" + return f"iceberg{engine_part}({storage_arg}, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" else: return ( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ENGINE=Iceberg{engine_part}({storage_arg}, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + settings_expression ) @@ -249,19 +259,19 @@ def get_creation_expression( if run_on_cluster: assert table_function return f""" - icebergAzureCluster('cluster_simple', azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) + iceberg{engine_part}Cluster('cluster_simple', {storage_arg}, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) """ else: if table_function: return f""" - icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) + iceberg{engine_part}({storage_arg}, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) """ else: return ( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" + ENGINE=Iceberg{engine_part}({storage_arg}, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" + settings_expression ) @@ -270,14 +280,14 @@ def get_creation_expression( if table_function: return f""" - icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) + iceberg{engine_part}({storage_arg}, path = '/iceberg_data/default/{table_name}/', format={format}) """ else: return ( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format})""" + ENGINE=Iceberg{engine_part}({storage_arg}, path = '/iceberg_data/default/{table_name}/', format={format})""" + settings_expression ) @@ -320,6 +330,7 @@ def create_iceberg_table( object_storage_cluster=False, **kwargs, ): + node.query(f"DROP TABLE IF EXISTS {table_name} SYNC") node.query( get_creation_expression( storage_type, @@ -562,6 +573,52 @@ def test_types(started_cluster, format_version, storage_type): ] ) + # Test storage type as function argument + table_function_expr = get_creation_expression( + storage_type, + TABLE_NAME, + started_cluster, + table_function=True, + storage_type_as_arg=True, + ) + assert ( + instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() + == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" + ) + + assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV( + [ + ["a", "Nullable(Int32)"], + ["b", "Nullable(String)"], + ["c", "Nullable(Date)"], + ["d", "Array(Nullable(String))"], + ["e", "Nullable(Bool)"], + ] + ) + + # Test storage type as field in named collection + table_function_expr = get_creation_expression( + storage_type, + TABLE_NAME, + started_cluster, + table_function=True, + storage_type_in_named_collection=True, + ) + assert ( + instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() + == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" + ) + + assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV( + [ + ["a", "Nullable(Int32)"], + ["b", "Nullable(String)"], + ["c", "Nullable(Date)"], + ["d", "Array(Nullable(String))"], + ["e", "Nullable(Bool)"], + ] + ) + def count_secondary_subqueries(started_cluster, query_id, expected, comment): for node_name, replica in started_cluster.instances.items(): @@ -638,52 +695,80 @@ def add_df(mode): instance.query(f"SELECT * FROM {table_function_expr}").strip().split() ) + def make_query_from_function( + run_on_cluster=False, + alt_syntax=False, + remote=False, + storage_type_as_arg=False, + storage_type_in_named_collection=False, + ): + expr = get_creation_expression( + storage_type, + TABLE_NAME, + started_cluster, + table_function=True, + run_on_cluster=run_on_cluster, + storage_type_as_arg=storage_type_as_arg, + storage_type_in_named_collection=storage_type_in_named_collection, + ) + query_id = str(uuid.uuid4()) + settings = "SETTINGS object_storage_cluster='cluster_simple'" if alt_syntax else "" + if remote: + query = f"SELECT * FROM remote('node2', {expr}) {settings}" + else: + query = f"SELECT * FROM {expr} {settings}" + responce = instance.query(query, query_id=query_id).strip().split() + return responce, query_id + # Cluster Query with node1 as coordinator - table_function_expr_cluster = get_creation_expression( - storage_type, - TABLE_NAME, - started_cluster, - table_function=True, + select_cluster, query_id_cluster = make_query_from_function(run_on_cluster=True) + + # Cluster Query with node1 as coordinator with alternative syntax + select_cluster_alt_syntax, query_id_cluster_alt_syntax = make_query_from_function( + run_on_cluster=True, + alt_syntax=True) + + # Cluster Query with node1 as coordinator and storage type as arg + select_cluster_with_type_arg, query_id_cluster_with_type_arg = make_query_from_function( run_on_cluster=True, + storage_type_as_arg=True, ) - query_id_cluster = str(uuid.uuid4()) - select_cluster = ( - instance.query( - f"SELECT * FROM {table_function_expr_cluster}", query_id=query_id_cluster - ) - .strip() - .split() + + # Cluster Query with node1 as coordinator and storage type in named collection + select_cluster_with_type_in_nc, query_id_cluster_with_type_in_nc = make_query_from_function( + run_on_cluster=True, + storage_type_in_named_collection=True, ) - # Cluster Query with node1 as coordinator with alternative syntax - query_id_cluster_alt_syntax = str(uuid.uuid4()) - select_cluster_alt_syntax = ( - instance.query( - f""" - SELECT * FROM {table_function_expr} - SETTINGS object_storage_cluster='cluster_simple' - """, - query_id=query_id_cluster_alt_syntax, - ) - .strip() - .split() + # Cluster Query with node1 as coordinator and storage type as arg, alternative syntax + select_cluster_with_type_arg_alt_syntax, query_id_cluster_with_type_arg_alt_syntax = make_query_from_function( + storage_type_as_arg=True, + alt_syntax=True, ) - create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, object_storage_cluster='cluster_simple') - query_id_cluster_table_engine = str(uuid.uuid4()) - select_cluster_table_engine = ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} - """, - query_id=query_id_cluster_table_engine, - ) - .strip() - .split() + # Cluster Query with node1 as coordinator and storage type in named collection, alternative syntax + select_cluster_with_type_in_nc_alt_syntax, query_id_cluster_with_type_in_nc_alt_syntax = make_query_from_function( + storage_type_in_named_collection=True, + alt_syntax=True, ) - # write 3 times - assert int(instance.query(f"SELECT count() FROM {table_function_expr_cluster}")) == 100 * 3 + #select_remote_cluster, _ = make_query_from_function(run_on_cluster=True, remote=True) + + def make_query_from_table(alt_syntax=False): + query_id = str(uuid.uuid4()) + settings = "SETTINGS object_storage_cluster='cluster_simple'" if alt_syntax else "" + responce = ( + instance.query( + f"SELECT * FROM {TABLE_NAME} {settings}", + query_id=query_id, + ) + .strip() + .split() + ) + return responce, query_id + + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, object_storage_cluster='cluster_simple') + select_cluster_table_engine, query_id_cluster_table_engine = make_query_from_table() #select_remote_cluster = ( # instance.query(f"SELECT * FROM remote('node2',{table_function_expr_cluster})") @@ -692,30 +777,18 @@ def add_df(mode): #) instance.query(f"DROP TABLE IF EXISTS `{TABLE_NAME}` SYNC") + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) - query_id_pure_table_engine = str(uuid.uuid4()) - select_pure_table_engine = ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} - """, - query_id=query_id_pure_table_engine, - ) - .strip() - .split() - ) - query_id_pure_table_engine_cluster = str(uuid.uuid4()) - select_pure_table_engine_cluster = ( - instance.query( - f""" - SELECT * FROM {TABLE_NAME} - SETTINGS object_storage_cluster='cluster_simple' - """, - query_id=query_id_pure_table_engine_cluster, - ) - .strip() - .split() - ) + select_pure_table_engine, query_id_pure_table_engine = make_query_from_table() + select_pure_table_engine_cluster, query_id_pure_table_engine_cluster = make_query_from_table(alt_syntax=True) + + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, storage_type_as_arg=True) + select_pure_table_engine_with_type_arg, query_id_pure_table_engine_with_type_arg = make_query_from_table() + select_pure_table_engine_cluster_with_type_arg, query_id_pure_table_engine_cluster_with_type_arg = make_query_from_table(alt_syntax=True) + + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster, storage_type_in_named_collection=True) + select_pure_table_engine_with_type_in_nc, query_id_pure_table_engine_with_type_in_nc = make_query_from_table() + select_pure_table_engine_cluster_with_type_in_nc, query_id_pure_table_engine_cluster_with_type_in_nc = make_query_from_table(alt_syntax=True) # Simple size check assert len(select_regular) == 600 @@ -723,16 +796,32 @@ def add_df(mode): assert len(select_cluster_alt_syntax) == 600 assert len(select_cluster_table_engine) == 600 #assert len(select_remote_cluster) == 600 + assert len(select_cluster_with_type_arg) == 600 + assert len(select_cluster_with_type_in_nc) == 600 + assert len(select_cluster_with_type_arg_alt_syntax) == 600 + assert len(select_cluster_with_type_in_nc_alt_syntax) == 600 assert len(select_pure_table_engine) == 600 assert len(select_pure_table_engine_cluster) == 600 + assert len(select_pure_table_engine_with_type_arg) == 600 + assert len(select_pure_table_engine_cluster_with_type_arg) == 600 + assert len(select_pure_table_engine_with_type_in_nc) == 600 + assert len(select_pure_table_engine_cluster_with_type_in_nc) == 600 # Actual check assert select_cluster == select_regular assert select_cluster_alt_syntax == select_regular assert select_cluster_table_engine == select_regular #assert select_remote_cluster == select_regular + assert select_cluster_with_type_arg == select_regular + assert select_cluster_with_type_in_nc == select_regular + assert select_cluster_with_type_arg_alt_syntax == select_regular + assert select_cluster_with_type_in_nc_alt_syntax == select_regular assert select_pure_table_engine == select_regular assert select_pure_table_engine_cluster == select_regular + assert select_pure_table_engine_with_type_arg == select_regular + assert select_pure_table_engine_cluster_with_type_arg == select_regular + assert select_pure_table_engine_with_type_in_nc == select_regular + assert select_pure_table_engine_cluster_with_type_in_nc == select_regular # Check query_log for replica in started_cluster.instances.values(): @@ -741,8 +830,16 @@ def add_df(mode): count_secondary_subqueries(started_cluster, query_id_cluster, 1, "table function") count_secondary_subqueries(started_cluster, query_id_cluster_alt_syntax, 1, "table function alt syntax") count_secondary_subqueries(started_cluster, query_id_cluster_table_engine, 1, "cluster table engine") + count_secondary_subqueries(started_cluster, query_id_cluster_with_type_arg, 1, "table function with storage type in args") + count_secondary_subqueries(started_cluster, query_id_cluster_with_type_in_nc, 1, "table function with storage type in named collection") + count_secondary_subqueries(started_cluster, query_id_cluster_with_type_arg_alt_syntax, 1, "table function with storage type in args alt syntax") + count_secondary_subqueries(started_cluster, query_id_cluster_with_type_in_nc_alt_syntax, 1, "table function with storage type in named collection alt syntax") count_secondary_subqueries(started_cluster, query_id_pure_table_engine, 0, "table engine") count_secondary_subqueries(started_cluster, query_id_pure_table_engine_cluster, 1, "table engine with cluster setting") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine_with_type_arg, 0, "table engine with storage type in args") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine_cluster_with_type_arg, 1, "table engine with cluster setting with storage type in args") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine_with_type_in_nc, 0, "table engine with storage type in named collection") + count_secondary_subqueries(started_cluster, query_id_pure_table_engine_cluster_with_type_in_nc, 1, "table engine with cluster setting with storage type in named collection") @pytest.mark.parametrize("format_version", ["1", "2"]) From 2e607658034b0b942d7cd9c637265063752b516d Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 17 Jul 2025 15:43:01 +0200 Subject: [PATCH 12/16] Merge pull request #677 from Altinity/feature/object_storage_max_nodes Setting object_storage_max_nodes --- tests/integration/test_s3_cluster/test.py | 160 ++++++++++++++++++++++ 1 file changed, 160 insertions(+) diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 3f2f788e9327..e4c0e56f1e1e 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -698,3 +698,163 @@ def test_distributed_s3_table_engine(started_cluster): assert int(hosts_engine_distributed) == 3 +def test_distributed_s3_table_engine(started_cluster): + node = started_cluster.instances["s0_0_0"] + + resp_def = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + """ + ) + + node.query("DROP TABLE IF EXISTS single_node"); + node.query( + """ + CREATE TABLE single_node + (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) + ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + """ + ) + query_id_engine_single_node = str(uuid.uuid4()) + resp_engine_single_node = node.query( + """ + SELECT * FROM single_node ORDER BY (name, value, polygon) + """, + query_id = query_id_engine_single_node + ) + assert resp_def == resp_engine_single_node + + node.query("DROP TABLE IF EXISTS distributed"); + node.query( + """ + CREATE TABLE distributed + (name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))) + ENGINE=S3('http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV') + SETTINGS object_storage_cluster='cluster_simple' + """ + ) + query_id_engine_distributed = str(uuid.uuid4()) + resp_engine_distributed = node.query( + """ + SELECT * FROM distributed ORDER BY (name, value, polygon) + """, + query_id = query_id_engine_distributed + ) + assert resp_def == resp_engine_distributed + + node.query("SYSTEM FLUSH LOGS ON CLUSTER 'cluster_simple'") + + hosts_engine_single_node = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_engine_single_node}' + """ + ) + assert int(hosts_engine_single_node) == 1 + hosts_engine_distributed = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_engine_distributed}' + """ + ) + assert int(hosts_engine_distributed) == 3 + + +def test_cluster_hosts_limit(started_cluster): + node = started_cluster.instances["s0_0_0"] + + query_id_def = str(uuid.uuid4()) + resp_def = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + """, + query_id = query_id_def + ) + + # object_storage_max_nodes is greater than number of hosts in cluster + query_id_4_hosts = str(uuid.uuid4()) + resp_4_hosts = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + SETTINGS object_storage_max_nodes=4 + """, + query_id = query_id_4_hosts + ) + assert resp_def == resp_4_hosts + + # object_storage_max_nodes is equal number of hosts in cluster + query_id_3_hosts = str(uuid.uuid4()) + resp_3_hosts = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + SETTINGS object_storage_max_nodes=3 + """, + query_id = query_id_3_hosts + ) + assert resp_def == resp_3_hosts + + # object_storage_max_nodes is less than number of hosts in cluster + query_id_2_hosts = str(uuid.uuid4()) + resp_2_hosts = node.query( + """ + SELECT * from s3Cluster( + 'cluster_simple', + 'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV', + 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon) + SETTINGS object_storage_max_nodes=2 + """, + query_id = query_id_2_hosts + ) + assert resp_def == resp_2_hosts + + node.query("SYSTEM FLUSH LOGS ON CLUSTER 'cluster_simple'") + + hosts_def = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_def}' AND query_id!='{query_id_def}' + """ + ) + assert int(hosts_def) == 3 + + hosts_4 = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_4_hosts}' AND query_id!='{query_id_4_hosts}' + """ + ) + assert int(hosts_4) == 3 + + hosts_3 = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_3_hosts}' AND query_id!='{query_id_3_hosts}' + """ + ) + assert int(hosts_3) == 3 + + hosts_2 = node.query( + f""" + SELECT uniq(hostname) + FROM clusterAllReplicas('cluster_simple', system.query_log) + WHERE type='QueryFinish' AND initial_query_id='{query_id_2_hosts}' AND query_id!='{query_id_2_hosts}' + """ + ) + assert int(hosts_2) == 2 From 06348488cbe61c37742d4808bfef0d5c8fed9584 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 17 Jul 2025 16:24:32 +0200 Subject: [PATCH 13/16] Fix mask sensitive info test --- .../test_mask_sensitive_info/test.py | 167 +++++++----------- 1 file changed, 64 insertions(+), 103 deletions(-) diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 2a1ea1db5a66..09ce7d8d8167 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -288,7 +288,7 @@ def test_create_table(): ( f"AzureBlobStorage('BlobEndpoint=https://my-endpoint/;SharedAccessSignature=sp=r&st=2025-09-29T14:58:11Z&se=2025-09-29T00:00:00Z&spr=https&sv=2022-11-02&sr=c&sig=SECRET%SECRET%SECRET%SECRET', 'exampledatasets', 'example.csv')", "STD_EXCEPTION", - ) + ), f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '{azure_account_key}')", f"AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '{azure_account_key}', 'CSV')", @@ -393,28 +393,27 @@ def make_test_case(i): "CREATE TABLE table20 (`x` int) ENGINE = S3Queue('http://minio1:9001/root/data/', 'minio', '[HIDDEN]', 'CSV', 'gzip') SETTINGS mode = 'ordered'", "CREATE TABLE table21 (`x` int) ENGINE = Iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", "CREATE TABLE table22 (`x` int) ENGINE = IcebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE table23 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV') SETTINGS mode = 'unordered'", - f"CREATE TABLE table24 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV', 'none') SETTINGS mode = 'unordered'", - f"CREATE TABLE table25 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV') SETTINGS mode = 'unordered'", - f"CREATE TABLE table26 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none') SETTINGS mode = 'unordered'", - f"CREATE TABLE table27 (`x` int) ENGINE = AzureBlobStorage('{masked_sas_conn_string}', 'exampledatasets', 'example.csv')", - "CREATE TABLE table23 (`x` int) ENGINE = Iceberg(storage_type = 's3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", f"CREATE TABLE table24 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", f"CREATE TABLE table25 (`x` int) ENGINE = AzureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"CREATE TABLE table26 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE table27 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", - f"CREATE TABLE table28 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", - f"CREATE TABLE table29 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE table30 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - f"CREATE TABLE table31 (`x` int) ENGINE = IcebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", - f"CREATE TABLE table32 (`x` int) ENGINE = IcebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE table33 (`x` int) ENGINE = IcebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE table34 (`x` int) ENGINE = IcebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - f"CREATE TABLE table35 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", - f"CREATE TABLE table36 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE table37 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE table38 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE table26 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV') SETTINGS mode = 'unordered'", + f"CREATE TABLE table27 (`x` int) ENGINE = AzureQueue('{masked_azure_conn_string}', 'cont', '*', 'CSV', 'none') SETTINGS mode = 'unordered'", + f"CREATE TABLE table28 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV') SETTINGS mode = 'unordered'", + f"CREATE TABLE table29 (`x` int) ENGINE = AzureQueue('{azure_storage_account_url}', 'cont', '*', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none') SETTINGS mode = 'unordered'", + f"CREATE TABLE table30 (`x` int) ENGINE = AzureBlobStorage('{masked_sas_conn_string}', 'exampledatasets', 'example.csv')", + f"CREATE TABLE table31 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table32 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", + f"CREATE TABLE table33 (`x` int) ENGINE = AzureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", + f"CREATE TABLE table34 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table35 (`x` int) ENGINE = AzureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE table36 (`x` int) ENGINE = IcebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE table37 (`x` int) ENGINE = IcebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table38 (`x` int) ENGINE = IcebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table39 (`x` int) ENGINE = IcebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + f"CREATE TABLE table40 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE table41 (`x` int) ENGINE = Iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE table42 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE table43 (`x` int) ENGINE = Iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", ], must_not_contain=[password], ) @@ -500,9 +499,6 @@ def test_table_functions(): f"s3('http://minio1:9001/root/data/test6.csv', 'minio', '{password}', 'CSV')", f"s3('http://minio1:9001/root/data/test7.csv', 'minio', '{password}', 'CSV', 'x int')", f"s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', '{password}', 'CSV', 'x int', 'gzip')", - f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', '{password}')", - f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')", - f"s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', '{password}', 'CSV')", f"remote('127.{{2..11}}', default.remote_table)", f"remote('127.{{2..11}}', default.remote_table, rand())", f"remote('127.{{2..11}}', default.remote_table, 'remote_user')", @@ -531,15 +527,6 @@ def test_table_functions(): f"azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", f"azureBlobStorage(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", f"azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '{azure_account_key}')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '{azure_account_key}', 'CSV')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none')", - f"azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", - f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", f"iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", f"iceberg(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", @@ -549,13 +536,6 @@ def test_table_functions(): f"icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", f"icebergAzure(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", f"icebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", - f"icebergS3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", - f"icebergS3Cluster('test_shard_localhost', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", - f"icebergAzureCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple.csv')", - f"icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '{azure_account_key}')", - f"icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '{azure_account_key}', 'CSV', 'none', 'auto')", - f"icebergAzureCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"icebergAzureCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '{azure_account_key}')", f"iceberg(storage_type='s3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '{password}')", f"iceberg(storage_type='s3', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '{password}')", f"iceberg(storage_type='azure', '{azure_conn_string}', 'cont', 'test_simple.csv')", @@ -616,70 +596,51 @@ def make_test_case(i): "CREATE TABLE tablefunc8 (`x` int) AS s3('http://minio1:9001/root/data/test6.csv', 'minio', '[HIDDEN]', 'CSV')", "CREATE TABLE tablefunc9 (`x` int) AS s3('http://minio1:9001/root/data/test7.csv', 'minio', '[HIDDEN]', 'CSV', 'x int')", "CREATE TABLE tablefunc10 (`x` int) AS s3('http://minio1:9001/root/data/test8.csv.gz', 'minio', '[HIDDEN]', 'CSV', 'x int', 'gzip')", - "CREATE TABLE tablefunc11 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test1.csv', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc12 (x int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test2.csv', 'CSV', 'x int')", - "CREATE TABLE tablefunc13 (`x` int) AS s3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test3.csv', 'minio', '[HIDDEN]', 'CSV')", - "CREATE TABLE tablefunc14 (x int) AS remote('127.{2..11}', default.remote_table)", - "CREATE TABLE tablefunc15 (x int) AS remote('127.{2..11}', default.remote_table, rand())", - "CREATE TABLE tablefunc16 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user')", - "CREATE TABLE tablefunc17 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]')", - "CREATE TABLE tablefunc18 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user', rand())", - "CREATE TABLE tablefunc19 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc20 (`x` int) AS remote('127.{2..11}', 'default.remote_table', 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc21 (`x` int) AS remote('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc22 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())", - "CREATE TABLE tablefunc23 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')", - "CREATE TABLE tablefunc24 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", - "CREATE TABLE tablefunc25 (`x` int) AS mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', `table` = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", - "CREATE TABLE tablefunc26 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', `table` = 'postgres_table', user = 'postgres_user')", - "CREATE TABLE tablefunc27 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - "CREATE TABLE tablefunc28 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", - "CREATE TABLE tablefunc29 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", - "CREATE TABLE tablefunc30 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", - "CREATE TABLE tablefunc31 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc32 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", - f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", - f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", - f"CREATE TABLE tablefunc39 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc40 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE tablefunc41 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - f"CREATE TABLE tablefunc42 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')", - f"CREATE TABLE tablefunc43 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')", - f"CREATE TABLE tablefunc44 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc45 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_12.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc46 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_13.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", - f"CREATE TABLE tablefunc47 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_14.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", - f"CREATE TABLE tablefunc48 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc49 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", - f"CREATE TABLE tablefunc50 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - "CREATE TABLE tablefunc51 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc52 (`x` int) AS iceberg(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - "CREATE TABLE tablefunc53 (`x` int) AS icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc54 (`x` int) AS icebergS3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - f"CREATE TABLE tablefunc55 (`x` int) AS icebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", - f"CREATE TABLE tablefunc56 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc57 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc58 (`x` int) AS icebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE tablefunc59 (`x` int) AS icebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - "CREATE TABLE tablefunc60 (`x` int) AS icebergS3Cluster('test_shard_localhost', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc61 (`x` int) AS icebergS3Cluster('test_shard_localhost', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - f"CREATE TABLE tablefunc62 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", - f"CREATE TABLE tablefunc63 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc64 (`x` int) AS icebergAzureCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc65 (`x` int) AS icebergAzureCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE tablefunc66 (`x` int) AS icebergAzureCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - "CREATE TABLE tablefunc67 (`x` int) AS iceberg(storage_type = 's3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - "CREATE TABLE tablefunc68 (`x` int) AS iceberg(storage_type = 's3', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", - f"CREATE TABLE tablefunc69 (`x` int) AS iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", - f"CREATE TABLE tablefunc70 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", - f"CREATE TABLE tablefunc71 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc72 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", - f"CREATE TABLE tablefunc73 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - "CREATE TABLE tablefunc74 (`x` int) AS gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc11 (x int) AS remote('127.{2..11}', default.remote_table)", + "CREATE TABLE tablefunc12 (x int) AS remote('127.{2..11}', default.remote_table, rand())", + "CREATE TABLE tablefunc13 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user')", + "CREATE TABLE tablefunc14 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]')", + "CREATE TABLE tablefunc15 (x int) AS remote('127.{2..11}', default.remote_table, 'remote_user', rand())", + "CREATE TABLE tablefunc16 (`x` int) AS remote('127.{2..11}', default.remote_table, 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc17 (`x` int) AS remote('127.{2..11}', 'default.remote_table', 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc18 (`x` int) AS remote('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc19 (`x` int) AS remote('127.{2..11}', numbers(10), 'remote_user', '[HIDDEN]', rand())", + "CREATE TABLE tablefunc20 (`x` int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', '[HIDDEN]')", + "CREATE TABLE tablefunc21 (x int) AS remoteSecure('127.{2..11}', 'default', 'remote_table', 'remote_user', rand())", + "CREATE TABLE tablefunc22 (`x` int) AS mysql(named_collection_1, host = 'mysql80', port = 3306, database = 'mysql_db', `table` = 'mysql_table', user = 'mysql_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc23 (`x` int) AS postgresql(named_collection_2, password = '[HIDDEN]', host = 'postgres1', port = 5432, database = 'postgres_db', `table` = 'postgres_table', user = 'postgres_user')", + "CREATE TABLE tablefunc24 (`x` int) AS s3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + "CREATE TABLE tablefunc25 (`x` int) AS remote(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]', sharding_key = rand())", + "CREATE TABLE tablefunc26 (`x` int) AS remoteSecure(named_collection_6, addresses_expr = '127.{2..11}', database = 'default', `table` = 'remote_table', user = 'remote_user', password = '[HIDDEN]')", + "CREATE TABLE tablefunc27 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", + "CREATE TABLE tablefunc28 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc29 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + f"CREATE TABLE tablefunc30 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", + f"CREATE TABLE tablefunc31 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", + f"CREATE TABLE tablefunc32 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", + f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", + f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc39 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc40 (`x` int) AS iceberg(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + "CREATE TABLE tablefunc41 (`x` int) AS icebergS3('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc42 (`x` int) AS icebergS3(named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc43 (`x` int) AS icebergAzure('{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE tablefunc44 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc45 (`x` int) AS icebergAzure('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc46 (`x` int) AS icebergAzure(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc47 (`x` int) AS icebergAzure(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc48 (`x` int) AS iceberg(storage_type = 's3', 'http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", + "CREATE TABLE tablefunc49 (`x` int) AS iceberg(storage_type = 's3', named_collection_2, url = 'http://minio1:9001/root/data/test4.csv', access_key_id = 'minio', secret_access_key = '[HIDDEN]')", + f"CREATE TABLE tablefunc50 (`x` int) AS iceberg(storage_type = 'azure', '{masked_azure_conn_string}', 'cont', 'test_simple.csv')", + f"CREATE TABLE tablefunc51 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple.csv', '{azure_account_name}', '[HIDDEN]')", + f"CREATE TABLE tablefunc52 (`x` int) AS iceberg(storage_type = 'azure', '{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc53 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc54 (`x` int) AS iceberg(storage_type = 'azure', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", + "CREATE TABLE tablefunc55 (`x` int) AS gcs('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], must_not_contain=[password], ) From 491947cbc8f919604d9d2d15e115198cde4dfda6 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 17 Jul 2025 20:27:50 +0200 Subject: [PATCH 14/16] Fix tidy build --- src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h | 4 ++-- src/Storages/ObjectStorage/StorageObjectStorage.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 602ccdcefa63..35321df7d626 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -369,7 +369,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, /// Find storage_type argument and remove it from args if exists. /// Return storage type. - ObjectStorageType extractDynamicStorageType(ASTs & args, ContextPtr context, ASTPtr * type_arg = nullptr) const override + ObjectStorageType extractDynamicStorageType(ASTs & args, ContextPtr context, ASTPtr * type_arg) const override { static const auto storage_type_name = "storage_type"; @@ -440,7 +440,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, void createDynamicConfiguration(ASTs & args, ContextPtr context) { - ObjectStorageType type = extractDynamicStorageType(args, context); + ObjectStorageType type = extractDynamicStorageType(args, context, nullptr); createDynamicStorage(type); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 668d268caaba..21967d2e1773 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -293,7 +293,7 @@ class StorageObjectStorage::Configuration virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; - virtual ObjectStorageType extractDynamicStorageType(ASTs & /* args */, ContextPtr /* context */, ASTPtr * /* type_arg */ = nullptr) const + virtual ObjectStorageType extractDynamicStorageType(ASTs & /* args */, ContextPtr /* context */, ASTPtr * /* type_arg */) const { return ObjectStorageType::None; } virtual const String & getFormat() const { return format; } From c66c37d21bb55f9654d698f450478a3959ee07c1 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sat, 3 May 2025 18:36:12 +0200 Subject: [PATCH 15/16] Merge pull request #761 from Altinity/feature/fix_integration_tests Feature/fix integration tests --- .../DataLakes/DataLakeConfiguration.h | 11 ++++++++++- src/Storages/ObjectStorage/StorageObjectStorage.h | 14 +++++++------- .../ObjectStorage/StorageObjectStorageCluster.cpp | 4 ++++ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 35321df7d626..8ca0a0e9887a 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -172,8 +172,9 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl LoggerPtr log = getLogger("DataLakeConfiguration"); const DataLakeStorageSettingsPtr settings; - void assertInitialized() const + void assertInitialized() const override { + BaseStorageConfiguration::assertInitialized(); if (!current_metadata) throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata is not initialized"); } @@ -258,6 +259,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, std::string getEngineName() const override { return getImpl().getEngineName(); } std::string getNamespaceType() const override { return getImpl().getNamespaceType(); } + Path getFullPath() const override { return getImpl().getFullPath(); } Path getPath() const override { return getImpl().getPath(); } void setPath(const Path & path) override { getImpl().setPath(path); } @@ -274,9 +276,14 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) override { getImpl().addStructureAndFormatToArgsIfNeeded(args, structure_, format_, context, with_structure); } + bool withPartitionWildcard() const override { return getImpl().withPartitionWildcard(); } + bool withGlobsIgnorePartitionWildcard() const override { return getImpl().withGlobsIgnorePartitionWildcard(); } + bool isPathWithGlobs() const override { return getImpl().isPathWithGlobs(); } + bool isNamespaceWithGlobs() const override { return getImpl().isNamespaceWithGlobs(); } std::string getPathWithoutGlobs() const override { return getImpl().getPathWithoutGlobs(); } bool isArchive() const override { return getImpl().isArchive(); } + bool isPathInArchiveWithGlobs() const override { return getImpl().isPathInArchiveWithGlobs(); } std::string getPathInArchive() const override { return getImpl().getPathInArchive(); } void check(ContextPtr context) const override { getImpl().check(context); } @@ -444,6 +451,8 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, createDynamicStorage(type); } + void assertInitialized() const override { return getImpl().assertInitialized(); } + private: inline StorageObjectStorage::Configuration & getImpl() const { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 21967d2e1773..bc9549ee4bff 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -219,15 +219,15 @@ class StorageObjectStorage::Configuration virtual void addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) = 0; - bool withPartitionWildcard() const; + virtual bool withPartitionWildcard() const; bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } - bool withGlobsIgnorePartitionWildcard() const; - bool isPathWithGlobs() const; - bool isNamespaceWithGlobs() const; + virtual bool withGlobsIgnorePartitionWildcard() const; + virtual bool isPathWithGlobs() const; + virtual bool isNamespaceWithGlobs() const; virtual std::string getPathWithoutGlobs() const; virtual bool isArchive() const { return false; } - bool isPathInArchiveWithGlobs() const; + virtual bool isPathInArchiveWithGlobs() const; virtual std::string getPathInArchive() const; virtual void check(ContextPtr context) const; @@ -304,14 +304,14 @@ class StorageObjectStorage::Configuration virtual void setCompressionMethod(const String & compression_method_) { compression_method = compression_method_; } virtual void setStructure(const String & structure_) { structure = structure_; } + virtual void assertInitialized() const; + private: String format = "auto"; String compression_method = "auto"; String structure = "auto"; protected: - void assertInitialized() const; - bool initialized = false; }; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 5469228702d4..d02406196b58 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -183,6 +183,8 @@ std::string StorageObjectStorageCluster::getName() const std::optional StorageObjectStorageCluster::totalRows(ContextPtr query_context) const { + if (pure_storage) + return pure_storage->totalRows(query_context); configuration->update( object_storage, query_context, @@ -193,6 +195,8 @@ std::optional StorageObjectStorageCluster::totalRows(ContextPtr query_co std::optional StorageObjectStorageCluster::totalBytes(ContextPtr query_context) const { + if (pure_storage) + return pure_storage->totalBytes(query_context); configuration->update( object_storage, query_context, From f5340acb970128d37b9966c96496c25823b80816 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Mon, 4 Aug 2025 18:24:03 +0200 Subject: [PATCH 16/16] Fix tidy build --- .../DataLakes/DataLakeConfiguration.h | 16 ++++++++-------- .../StorageObjectStorageCluster.cpp | 5 ++++- .../registerStorageObjectStorage.cpp | 2 +- .../extractTableFunctionFromSelectQuery.h | 1 - ...TableFunctionObjectStorageClusterFallback.cpp | 4 ++-- 5 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 8ca0a0e9887a..e1f56b0d0feb 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -370,15 +370,15 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, protected: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override - { return getImpl().fromNamedCollection(collection, context); } + { getImpl().fromNamedCollection(collection, context); } void fromAST(ASTs & args, ContextPtr context, bool with_structure) override - { return getImpl().fromAST(args, context, with_structure); } + { getImpl().fromAST(args, context, with_structure); } /// Find storage_type argument and remove it from args if exists. /// Return storage type. ObjectStorageType extractDynamicStorageType(ASTs & args, ContextPtr context, ASTPtr * type_arg) const override { - static const auto storage_type_name = "storage_type"; + static const auto * const storage_type_name = "storage_type"; if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) { @@ -388,20 +388,20 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, } } - auto type_it = args.end(); + auto * type_it = args.end(); /// S3 by default for backward compatibility /// Iceberg without storage_type == IcebergS3 ObjectStorageType type = ObjectStorageType::S3; - for (auto arg_it = args.begin(); arg_it != args.end(); ++arg_it) + for (auto * arg_it = args.begin(); arg_it != args.end(); ++arg_it) { const auto * type_ast_function = (*arg_it)->as(); if (type_ast_function && type_ast_function->name == "equals" && type_ast_function->arguments && type_ast_function->arguments->children.size() == 2) { - auto name = type_ast_function->arguments->children[0]->as(); + auto * name = type_ast_function->arguments->children[0]->as(); if (name && name->name() == storage_type_name) { @@ -412,7 +412,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, "DataLake can have only one key-value argument: storage_type='type'."); } - auto value = type_ast_function->arguments->children[1]->as(); + auto * value = type_ast_function->arguments->children[1]->as(); if (!value) { @@ -451,7 +451,7 @@ class StorageIcebergConfiguration : public StorageObjectStorage::Configuration, createDynamicStorage(type); } - void assertInitialized() const override { return getImpl().assertInitialized(); } + void assertInitialized() const override { getImpl().assertInitialized(); } private: inline StorageObjectStorage::Configuration & getImpl() const diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index d02406196b58..f22a8f6e7f63 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -510,7 +510,10 @@ void StorageObjectStorageCluster::truncate( { /// Full query if fall back to pure storage. if (getClusterName(local_context).empty()) - return pure_storage->truncate(query, metadata_snapshot, local_context, lock_holder); + { + pure_storage->truncate(query, metadata_snapshot, local_context, lock_holder); + return; + } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 3bc4b4a63d33..5f42a415adb9 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -37,7 +37,7 @@ createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObject const auto context = args.getLocalContext(); - std::string cluster_name = ""; + std::string cluster_name; if (args.storage_def->settings) { diff --git a/src/Storages/extractTableFunctionFromSelectQuery.h b/src/Storages/extractTableFunctionFromSelectQuery.h index 0511d59f6230..87edf01c1c82 100644 --- a/src/Storages/extractTableFunctionFromSelectQuery.h +++ b/src/Storages/extractTableFunctionFromSelectQuery.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { diff --git a/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp index ea31d246b173..26faabe952ef 100644 --- a/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageClusterFallback.cpp @@ -103,7 +103,7 @@ void TableFunctionObjectStorageClusterFallback::parseArguments args.erase(args.begin()); } else - BaseSimple::parseArgumentsImpl(args, context); + BaseSimple::parseArgumentsImpl(args, context); // NOLINT(bugprone-parent-virtual-call) } template @@ -122,7 +122,7 @@ StoragePtr TableFunctionObjectStorageClusterFallback::executeI return result; } else - return BaseSimple::executeImpl(ast_function, context, table_name, cached_columns, is_insert_query); + return BaseSimple::executeImpl(ast_function, context, table_name, cached_columns, is_insert_query); // NOLINT(bugprone-parent-virtual-call) } template